github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/transformer/columnselector/column_selector.go (about) 1 // Copyright 2023 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 columnselector 15 16 import ( 17 filter "github.com/pingcap/tidb/pkg/util/table-filter" 18 "github.com/pingcap/tiflow/cdc/model" 19 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher" 20 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher/partition" 21 "github.com/pingcap/tiflow/pkg/config" 22 "github.com/pingcap/tiflow/pkg/errors" 23 ) 24 25 type selector struct { 26 tableF filter.Filter 27 columnM filter.ColumnFilter 28 } 29 30 func newSelector( 31 rule *config.ColumnSelector, caseSensitive bool, 32 ) (*selector, error) { 33 tableM, err := filter.Parse(rule.Matcher) 34 if err != nil { 35 return nil, errors.WrapError(errors.ErrFilterRuleInvalid, err, rule.Matcher) 36 } 37 if !caseSensitive { 38 tableM = filter.CaseInsensitive(tableM) 39 } 40 columnM, err := filter.ParseColumnFilter(rule.Columns) 41 if err != nil { 42 return nil, errors.WrapError(errors.ErrFilterRuleInvalid, err, rule.Columns) 43 } 44 45 return &selector{ 46 tableF: tableM, 47 columnM: columnM, 48 }, nil 49 } 50 51 // Match implements Transformer interface 52 func (s *selector) Match(schema, table string) bool { 53 return s.tableF.MatchTable(schema, table) 54 } 55 56 // Apply implements Transformer interface 57 // return error if the given event cannot match the selector, or the column cannot be filtered out. 58 // the caller's should make sure the given event match the selector first before apply it. 59 func (s *selector) Apply(event *model.RowChangedEvent) error { 60 // defensive check, this should not happen. 61 if !s.Match(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) { 62 return errors.ErrColumnSelectorFailed.GenWithStack( 63 "the given event does not match the column selector, table: %v", event.TableInfo.TableName) 64 } 65 66 retainedColumns := make(map[string]struct{}, len(event.Columns)) 67 if len(event.Columns) != 0 { 68 for idx, column := range event.Columns { 69 colName := event.TableInfo.ForceGetColumnName(column.ColumnID) 70 if s.columnM.MatchColumn(colName) { 71 retainedColumns[colName] = struct{}{} 72 continue 73 } 74 event.Columns[idx] = nil 75 } 76 77 if !verifyIndices(event.TableInfo, retainedColumns) { 78 return errors.ErrColumnSelectorFailed.GenWithStack( 79 "no primary key columns or unique key columns obtained after filter out, "+ 80 "table: %+v", event.TableInfo.TableName) 81 } 82 } 83 84 if len(event.PreColumns) != 0 { 85 clear(retainedColumns) 86 for idx, column := range event.PreColumns { 87 colName := event.TableInfo.ForceGetColumnName(column.ColumnID) 88 if s.columnM.MatchColumn(colName) { 89 retainedColumns[colName] = struct{}{} 90 continue 91 } 92 event.PreColumns[idx] = nil 93 } 94 if !verifyIndices(event.TableInfo, retainedColumns) { 95 return errors.ErrColumnSelectorFailed.GenWithStack( 96 "no primary key columns or unique key columns obtained after filter out, "+ 97 "table: %+v", event.TableInfo.TableName) 98 } 99 } 100 101 return nil 102 } 103 104 // ColumnSelector manages an array of selectors, the first selector match the given 105 // event is used to select out columns. 106 type ColumnSelector struct { 107 selectors []*selector 108 } 109 110 // New return a column selector 111 func New(cfg *config.ReplicaConfig) (*ColumnSelector, error) { 112 selectors := make([]*selector, 0, len(cfg.Sink.ColumnSelectors)) 113 for _, r := range cfg.Sink.ColumnSelectors { 114 selector, err := newSelector(r, cfg.CaseSensitive) 115 if err != nil { 116 return nil, err 117 } 118 selectors = append(selectors, selector) 119 } 120 121 return &ColumnSelector{ 122 selectors: selectors, 123 }, nil 124 } 125 126 // Apply the column selector to the given event. 127 func (c *ColumnSelector) Apply(event *model.RowChangedEvent) error { 128 for _, s := range c.selectors { 129 if s.Match(event.TableInfo.GetSchemaName(), event.TableInfo.GetTableName()) { 130 return s.Apply(event) 131 } 132 } 133 return nil 134 } 135 136 // VerifyTables return the error if any given table cannot satisfy the column selector constraints. 137 // 1. if the column is filter out, it must not be a part of handle key or the unique key. 138 // 2. if the filtered out column is used in the column dispatcher, return error. 139 func (c *ColumnSelector) VerifyTables( 140 infos []*model.TableInfo, eventRouter *dispatcher.EventRouter, 141 ) error { 142 if len(c.selectors) == 0 { 143 return nil 144 } 145 146 for _, table := range infos { 147 for _, s := range c.selectors { 148 if !s.Match(table.TableName.Schema, table.TableName.Table) { 149 continue 150 } 151 152 retainedColumns := make(map[string]struct{}) 153 for columnID := range table.ColumnsFlag { 154 columnInfo, ok := table.GetColumnInfo(columnID) 155 if !ok { 156 return errors.ErrColumnSelectorFailed.GenWithStack( 157 "column not found when verify the table for the column selector, table: %v, column: %s", 158 table.TableName, columnInfo.Name) 159 } 160 columnName := columnInfo.Name.O 161 if s.columnM.MatchColumn(columnName) { 162 retainedColumns[columnName] = struct{}{} 163 continue 164 } 165 166 partitionDispatcher := eventRouter.GetPartitionDispatcher(table.TableName.Schema, table.TableName.Table) 167 switch v := partitionDispatcher.(type) { 168 case *partition.ColumnsDispatcher: 169 for _, col := range v.Columns { 170 if col == columnInfo.Name.O { 171 return errors.ErrColumnSelectorFailed.GenWithStack( 172 "the filtered out column is used in the column dispatcher, "+ 173 "table: %v, column: %s", table.TableName, columnInfo.Name) 174 } 175 } 176 default: 177 } 178 } 179 180 if !verifyIndices(table, retainedColumns) { 181 return errors.ErrColumnSelectorFailed.GenWithStack( 182 "no primary key columns or unique key columns obtained after filter out, table: %+v", table.TableName) 183 } 184 } 185 } 186 return nil 187 } 188 189 // verifyIndices return true if the primary key retained, 190 // else at least there are one unique key columns in the retained columns. 191 func verifyIndices(table *model.TableInfo, retainedColumns map[string]struct{}) bool { 192 primaryKeyColumns := table.GetPrimaryKeyColumnNames() 193 194 retained := true 195 for _, name := range primaryKeyColumns { 196 if _, ok := retainedColumns[name]; !ok { 197 retained = false 198 break 199 } 200 } 201 // primary key columns are retained, return true. 202 if retained { 203 return true 204 } 205 206 // at least one unique key columns are retained, return true. 207 for _, index := range table.Indices { 208 if !index.Unique { 209 continue 210 } 211 212 retained = true 213 for _, col := range index.Columns { 214 if _, ok := retainedColumns[col.Name.O]; !ok { 215 retained = false 216 break 217 } 218 } 219 if retained { 220 return true 221 } 222 } 223 return false 224 } 225 226 // VerifyColumn return true if the given `schema.table` column is matched. 227 func (c *ColumnSelector) VerifyColumn(schema, table, column string) bool { 228 for _, s := range c.selectors { 229 if !s.Match(schema, table) { 230 continue 231 } 232 return s.columnM.MatchColumn(column) 233 } 234 return true 235 }