github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/filter.go (about)

     1  // Copyright 2019 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 syncer
    15  
    16  import (
    17  	"github.com/go-mysql-org/go-mysql/replication"
    18  	"github.com/pingcap/tidb/pkg/util/filter"
    19  	"github.com/pingcap/tiflow/dm/pkg/terror"
    20  	"github.com/pingcap/tiflow/dm/pkg/utils"
    21  	onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools"
    22  	bf "github.com/pingcap/tiflow/pkg/binlog-filter"
    23  	"go.uber.org/zap"
    24  )
    25  
    26  // skipQueryEvent if skip by binlog-filter:
    27  // * track the ddlInfo;
    28  // * changes ddlInfo.originDDL to empty string.
    29  func (s *Syncer) skipQueryEvent(qec *queryEventContext, ddlInfo *ddlInfo) (bool, error) {
    30  	if utils.IsBuildInSkipDDL(qec.originSQL) {
    31  		return true, nil
    32  	}
    33  	et := bf.AstToDDLEvent(ddlInfo.stmtCache)
    34  	// get real tables before apply block-allow list
    35  	realTables := make([]*filter.Table, 0, len(ddlInfo.sourceTables))
    36  	for _, table := range ddlInfo.sourceTables {
    37  		realTableName := table.Name
    38  		if s.onlineDDL != nil {
    39  			realTableName = s.onlineDDL.RealName(table.Name)
    40  		}
    41  		realTables = append(realTables, &filter.Table{
    42  			Schema: table.Schema,
    43  			Name:   realTableName,
    44  		})
    45  	}
    46  	for _, table := range realTables {
    47  		s.tctx.L().Debug("query event info", zap.String("event", "query"), zap.String("origin sql", qec.originSQL), zap.Stringer("table", table), zap.Stringer("ddl info", ddlInfo))
    48  		if s.skipByTable(table) {
    49  			s.tctx.L().Debug("skip event by balist")
    50  			return true, nil
    51  		}
    52  		needSkip, err := s.skipByFilter(table, et, qec.originSQL)
    53  		if err != nil {
    54  			return needSkip, err
    55  		}
    56  
    57  		if needSkip {
    58  			s.tctx.L().Debug("skip event by binlog filter")
    59  			// In the case of online-ddl, if the generated table is skipped, track ddl will failed.
    60  			err := s.trackDDL(qec.ddlSchema, ddlInfo, qec.eventContext)
    61  			if err != nil {
    62  				s.tctx.L().Warn("track ddl failed", zap.Stringer("ddl info", ddlInfo))
    63  			}
    64  			s.saveTablePoint(table, qec.lastLocation)
    65  			s.tctx.L().Warn("track skipped ddl and return empty string", zap.String("origin sql", qec.originSQL), zap.Stringer("ddl info", ddlInfo))
    66  			ddlInfo.originDDL = ""
    67  			return true, nil
    68  		}
    69  	}
    70  	return false, nil
    71  }
    72  
    73  func (s *Syncer) skipRowsEvent(table *filter.Table, eventType replication.EventType) (bool, error) {
    74  	// skip un-realTable
    75  	if s.onlineDDL != nil && s.onlineDDL.TableType(table.Name) != onlineddl.RealTable {
    76  		return true, nil
    77  	}
    78  	if s.skipByTable(table) {
    79  		return true, nil
    80  	}
    81  	var et bf.EventType
    82  	switch eventType {
    83  	case replication.WRITE_ROWS_EVENTv0, replication.WRITE_ROWS_EVENTv1, replication.WRITE_ROWS_EVENTv2:
    84  		et = bf.InsertEvent
    85  	case replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2:
    86  		et = bf.UpdateEvent
    87  	case replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2:
    88  		et = bf.DeleteEvent
    89  	default:
    90  		return false, terror.ErrSyncerUnitInvalidReplicaEvent.Generate(eventType)
    91  	}
    92  	return s.skipByFilter(table, et, "")
    93  }
    94  
    95  // skipSQLByPattern skip unsupported sql in tidb and global sql-patterns in binlog-filter config file.
    96  func skipSQLByPattern(binlogFilter *bf.BinlogEvent, sql string) (bool, error) {
    97  	if utils.IsBuildInSkipDDL(sql) {
    98  		return true, nil
    99  	}
   100  	action, err := binlogFilter.Filter("", "", bf.NullEvent, sql)
   101  	if err != nil {
   102  		return false, terror.Annotatef(terror.ErrSyncerUnitBinlogEventFilter.New(err.Error()), "skip query %s", sql)
   103  	}
   104  	return action == bf.Ignore, nil
   105  }
   106  
   107  func (s *Syncer) skipByFilter(table *filter.Table, et bf.EventType, sql string) (bool, error) {
   108  	return skipByFilter(s.binlogFilter, table, et, sql)
   109  }
   110  
   111  // skipByFilter returns true when
   112  // * type of SQL doesn't pass binlog-filter.
   113  // * pattern of SQL doesn't pass binlog-filter.
   114  func skipByFilter(binlogFilter *bf.BinlogEvent, table *filter.Table, et bf.EventType, sql string) (bool, error) {
   115  	if binlogFilter == nil {
   116  		return false, nil
   117  	}
   118  	action, err := binlogFilter.Filter(table.Schema, table.Name, et, sql)
   119  	if err != nil {
   120  		return false, terror.Annotatef(terror.ErrSyncerUnitBinlogEventFilter.New(err.Error()), "skip event %s on %v", et, table)
   121  	}
   122  	switch action {
   123  	case bf.Ignore:
   124  		return true, nil
   125  	case bf.Error:
   126  		return false, terror.ErrSyncerUnitBinlogEventFilter.Generatef("event %s on %v", et, table)
   127  	}
   128  	return false, nil
   129  }
   130  
   131  func (s *Syncer) skipByTable(table *filter.Table) bool {
   132  	return skipByTable(s.baList, table)
   133  }
   134  
   135  // skipByTable returns true when
   136  // * any schema of table names is system schema.
   137  // * any table name doesn't pass block-allow list.
   138  func skipByTable(baList *filter.Filter, table *filter.Table) bool {
   139  	if filter.IsSystemSchema(table.Schema) {
   140  		return true
   141  	}
   142  	tables := baList.Apply([]*filter.Table{table})
   143  	return len(tables) == 0
   144  }