github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/filter/sql_event_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  	"github.com/pingcap/errors"
    18  	"github.com/pingcap/log"
    19  	tfilter "github.com/pingcap/tidb/pkg/util/table-filter"
    20  	"github.com/pingcap/tiflow/cdc/model"
    21  	bf "github.com/pingcap/tiflow/pkg/binlog-filter"
    22  	"github.com/pingcap/tiflow/pkg/config"
    23  	cerror "github.com/pingcap/tiflow/pkg/errors"
    24  	"go.uber.org/zap"
    25  )
    26  
    27  const (
    28  	// binlogFilterSchemaPlaceholder is a place holder for schema name in binlog filter.
    29  	// Since we use table filter in rule as a matcher to match a dml/ddl event's schema and table,
    30  	// so we don't need to care about schema name when we calling binlog filter's method,
    31  	// we just use this place holder to call binlog filter's method whenever we need pass a schema.
    32  	binlogFilterSchemaPlaceholder = "binlogFilterSchema"
    33  	// binlogFilterTablePlaceholder is a place holder for table name in binlog filter.
    34  	// The reason we need it is the same as binlogFilterSchemaPlaceholder.
    35  	binlogFilterTablePlaceholder = "binlogFilterTable"
    36  	// dmlQuery is a place holder to call binlog filter to filter dml event.
    37  	dmlQuery = ""
    38  	// caseSensitive is use to create bf.BinlogEvent.
    39  	caseSensitive = false
    40  )
    41  
    42  // sqlEventRule only be used by sqlEventFilter.
    43  type sqlEventRule struct {
    44  	// we use table filter to match a dml/ddl event's schema and table.
    45  	// since binlog filter does not support syntax like `!test.t1`,
    46  	// which means not match `test.t1`.
    47  	tf tfilter.Filter
    48  	bf *bf.BinlogEvent
    49  }
    50  
    51  func newSQLEventFilterRule(cfg *config.EventFilterRule) (*sqlEventRule, error) {
    52  	tf, err := tfilter.Parse(cfg.Matcher)
    53  	if err != nil {
    54  		return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg.Matcher)
    55  	}
    56  
    57  	res := &sqlEventRule{
    58  		tf: tf,
    59  	}
    60  
    61  	if err := verifyIgnoreEvents(cfg.IgnoreEvent); err != nil {
    62  		return nil, err
    63  	}
    64  
    65  	bfRule := &bf.BinlogEventRule{
    66  		SchemaPattern: binlogFilterSchemaPlaceholder,
    67  		TablePattern:  binlogFilterTablePlaceholder,
    68  		Events:        cfg.IgnoreEvent,
    69  		SQLPattern:    cfg.IgnoreSQL,
    70  		Action:        bf.Ignore,
    71  	}
    72  
    73  	res.bf, err = bf.NewBinlogEvent(caseSensitive, []*bf.BinlogEventRule{bfRule})
    74  	if err != nil {
    75  		return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, "failed to create binlog event filter")
    76  	}
    77  
    78  	return res, nil
    79  }
    80  
    81  func verifyIgnoreEvents(types []bf.EventType) error {
    82  	typesMap := make(map[bf.EventType]struct{}, len(SupportedEventTypes()))
    83  	for _, et := range SupportedEventTypes() {
    84  		typesMap[et] = struct{}{}
    85  	}
    86  	for _, et := range types {
    87  		if _, ok := typesMap[et]; !ok {
    88  			return cerror.ErrInvalidIgnoreEventType.GenWithStackByArgs(string(et))
    89  		}
    90  	}
    91  	return nil
    92  }
    93  
    94  // sqlEventFilter is a filter that filters DDL/DML event by its type or query.
    95  type sqlEventFilter struct {
    96  	rules []*sqlEventRule
    97  }
    98  
    99  func newSQLEventFilter(cfg *config.FilterConfig) (*sqlEventFilter, error) {
   100  	res := &sqlEventFilter{}
   101  	for _, rule := range cfg.EventFilters {
   102  		if err := res.addRule(rule); err != nil {
   103  			return nil, errors.Trace(err)
   104  		}
   105  	}
   106  	return res, nil
   107  }
   108  
   109  func (f *sqlEventFilter) addRule(cfg *config.EventFilterRule) error {
   110  	rule, err := newSQLEventFilterRule(cfg)
   111  	if err != nil {
   112  		return errors.Trace(err)
   113  	}
   114  	f.rules = append(f.rules, rule)
   115  	return nil
   116  }
   117  
   118  func (f *sqlEventFilter) getRules(schema, table string) []*sqlEventRule {
   119  	res := make([]*sqlEventRule, 0)
   120  	for _, rule := range f.rules {
   121  		if len(table) == 0 {
   122  			if rule.tf.MatchSchema(schema) {
   123  				res = append(res, rule)
   124  			}
   125  		} else {
   126  			if rule.tf.MatchTable(schema, table) {
   127  				res = append(res, rule)
   128  			}
   129  		}
   130  	}
   131  	return res
   132  }
   133  
   134  // skipDDLEvent skips ddl event by its type and query.
   135  func (f *sqlEventFilter) shouldSkipDDL(ddl *model.DDLEvent) (skip bool, err error) {
   136  	if len(f.rules) == 0 {
   137  		return false, nil
   138  	}
   139  	schema := ddl.TableInfo.TableName.Schema
   140  	table := ddl.TableInfo.TableName.Table
   141  	evenType := ddlToEventType(ddl.Type)
   142  	if evenType == bf.NullEvent {
   143  		log.Warn("sql event filter unsupported ddl type, do nothing",
   144  			zap.String("type", ddl.Type.String()),
   145  			zap.String("query", ddl.Query))
   146  		return false, nil
   147  	}
   148  
   149  	rules := f.getRules(schema, table)
   150  	for _, rule := range rules {
   151  		action, err := rule.bf.Filter(
   152  			binlogFilterSchemaPlaceholder,
   153  			binlogFilterTablePlaceholder,
   154  			evenType, ddl.Query)
   155  		if err != nil {
   156  			return false, errors.Trace(err)
   157  		}
   158  		if action == bf.Ignore {
   159  			return true, nil
   160  		}
   161  
   162  		// If the ddl is alter table's subtype,
   163  		// we need try to filter it by bf.AlterTable.
   164  		if isAlterTable(ddl.Type) {
   165  			action, err = rule.bf.Filter(
   166  				binlogFilterSchemaPlaceholder,
   167  				binlogFilterTablePlaceholder,
   168  				bf.AlterTable, ddl.Query)
   169  			if err != nil {
   170  				return false, errors.Trace(err)
   171  			}
   172  			if action == bf.Ignore {
   173  				return true, nil
   174  			}
   175  		}
   176  	}
   177  	return false, nil
   178  }
   179  
   180  // shouldSkipDML skips dml event by its type.
   181  func (f *sqlEventFilter) shouldSkipDML(event *model.RowChangedEvent) (bool, error) {
   182  	if len(f.rules) == 0 {
   183  		return false, nil
   184  	}
   185  
   186  	var et bf.EventType
   187  	switch {
   188  	case event.IsInsert():
   189  		et = bf.InsertEvent
   190  	case event.IsUpdate():
   191  		et = bf.UpdateEvent
   192  	case event.IsDelete():
   193  		et = bf.DeleteEvent
   194  	default:
   195  		// It should never happen.
   196  		log.Warn("unknown row changed event type")
   197  		return false, nil
   198  	}
   199  	rules := f.getRules(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName())
   200  	for _, rule := range rules {
   201  		action, err := rule.bf.Filter(binlogFilterSchemaPlaceholder, binlogFilterTablePlaceholder, et, dmlQuery)
   202  		if err != nil {
   203  			return false, cerror.WrapError(cerror.ErrFailedToFilterDML, err, event)
   204  		}
   205  		if action == bf.Ignore {
   206  			return true, nil
   207  		}
   208  	}
   209  	return false, nil
   210  }