github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dispatcher/event_router.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 dispatcher 15 16 import ( 17 "strings" 18 19 "github.com/pingcap/log" 20 filter "github.com/pingcap/tidb/pkg/util/table-filter" 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher/partition" 23 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher/topic" 24 "github.com/pingcap/tiflow/pkg/config" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/sink" 27 "go.uber.org/zap" 28 ) 29 30 // EventRouter is a router, it determines which topic and which partition 31 // an event should be dispatched to. 32 type EventRouter struct { 33 defaultTopic string 34 35 rules []struct { 36 partitionDispatcher partition.Dispatcher 37 topicDispatcher topic.Dispatcher 38 filter.Filter 39 } 40 } 41 42 // NewEventRouter creates a new EventRouter. 43 func NewEventRouter( 44 cfg *config.ReplicaConfig, protocol config.Protocol, defaultTopic, scheme string, 45 ) (*EventRouter, error) { 46 // If an event does not match any dispatching rules in the config file, 47 // it will be dispatched by the default partition dispatcher and 48 // static topic dispatcher because it matches *.* rule. 49 ruleConfigs := append(cfg.Sink.DispatchRules, &config.DispatchRule{ 50 Matcher: []string{"*.*"}, 51 PartitionRule: "default", 52 TopicRule: "", 53 }) 54 55 rules := make([]struct { 56 partitionDispatcher partition.Dispatcher 57 topicDispatcher topic.Dispatcher 58 filter.Filter 59 }, 0, len(ruleConfigs)) 60 61 for _, ruleConfig := range ruleConfigs { 62 f, err := filter.Parse(ruleConfig.Matcher) 63 if err != nil { 64 return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, ruleConfig.Matcher) 65 } 66 if !cfg.CaseSensitive { 67 f = filter.CaseInsensitive(f) 68 } 69 70 d := getPartitionDispatcher( 71 ruleConfig.PartitionRule, scheme, ruleConfig.IndexName, ruleConfig.Columns, 72 ) 73 t, err := getTopicDispatcher(ruleConfig.TopicRule, defaultTopic, protocol, scheme) 74 if err != nil { 75 return nil, err 76 } 77 rules = append(rules, struct { 78 partitionDispatcher partition.Dispatcher 79 topicDispatcher topic.Dispatcher 80 filter.Filter 81 }{partitionDispatcher: d, topicDispatcher: t, Filter: f}) 82 } 83 84 return &EventRouter{ 85 defaultTopic: defaultTopic, 86 rules: rules, 87 }, nil 88 } 89 90 // GetTopicForRowChange returns the target topic for row changes. 91 func (s *EventRouter) GetTopicForRowChange(row *model.RowChangedEvent) string { 92 topicDispatcher, _ := s.matchDispatcher(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) 93 return topicDispatcher.Substitute(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()) 94 } 95 96 // GetTopicForDDL returns the target topic for DDL. 97 func (s *EventRouter) GetTopicForDDL(ddl *model.DDLEvent) string { 98 var schema, table string 99 if ddl.PreTableInfo != nil { 100 if ddl.PreTableInfo.TableName.Table == "" { 101 return s.defaultTopic 102 } 103 schema = ddl.PreTableInfo.TableName.Schema 104 table = ddl.PreTableInfo.TableName.Table 105 } else { 106 if ddl.TableInfo.TableName.Table == "" { 107 return s.defaultTopic 108 } 109 schema = ddl.TableInfo.TableName.Schema 110 table = ddl.TableInfo.TableName.Table 111 } 112 113 topicDispatcher, _ := s.matchDispatcher(schema, table) 114 return topicDispatcher.Substitute(schema, table) 115 } 116 117 // GetPartitionForRowChange returns the target partition for row changes. 118 func (s *EventRouter) GetPartitionForRowChange( 119 row *model.RowChangedEvent, 120 partitionNum int32, 121 ) (int32, string, error) { 122 return s.GetPartitionDispatcher(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName()). 123 DispatchRowChangedEvent(row, partitionNum) 124 } 125 126 // GetPartitionDispatcher returns the partition dispatcher for a specific table. 127 func (s *EventRouter) GetPartitionDispatcher(schema, table string) partition.Dispatcher { 128 _, partitionDispatcher := s.matchDispatcher(schema, table) 129 return partitionDispatcher 130 } 131 132 // VerifyTables return error if any one table route rule is invalid. 133 func (s *EventRouter) VerifyTables(infos []*model.TableInfo) error { 134 for _, table := range infos { 135 _, partitionDispatcher := s.matchDispatcher(table.TableName.Schema, table.TableName.Table) 136 switch v := partitionDispatcher.(type) { 137 case *partition.IndexValueDispatcher: 138 if v.IndexName != "" { 139 index := table.GetIndex(v.IndexName) 140 if index == nil { 141 return cerror.ErrDispatcherFailed.GenWithStack( 142 "index not found when verify the table, table: %v, index: %s", table.TableName, v.IndexName) 143 } 144 // only allow the unique index to be set. 145 // For the non-unique index, if any column belongs to the index is updated, 146 // the event is not split, it may cause incorrect data consumption. 147 if !index.Unique { 148 return cerror.ErrDispatcherFailed.GenWithStack( 149 "index is not unique when verify the table, table: %v, index: %s", table.TableName, v.IndexName) 150 } 151 } 152 case *partition.ColumnsDispatcher: 153 _, ok := table.OffsetsByNames(v.Columns) 154 if !ok { 155 return cerror.ErrDispatcherFailed.GenWithStack( 156 "columns not found when verify the table, table: %v, columns: %v", table.TableName, v.Columns) 157 } 158 default: 159 } 160 } 161 return nil 162 } 163 164 // GetActiveTopics returns a list of the corresponding topics 165 // for the tables that are actively synchronized. 166 func (s *EventRouter) GetActiveTopics(activeTables []model.TableName) []string { 167 topics := make([]string, 0) 168 topicsMap := make(map[string]bool, len(activeTables)) 169 for _, table := range activeTables { 170 topicDispatcher, _ := s.matchDispatcher(table.Schema, table.Table) 171 topicName := topicDispatcher.Substitute(table.Schema, table.Table) 172 if topicName == s.defaultTopic { 173 log.Debug("topic name corresponding to the table is the same as the default topic name", 174 zap.String("table", table.String()), 175 zap.String("defaultTopic", s.defaultTopic), 176 zap.String("topicDispatcherExpression", topicDispatcher.String()), 177 ) 178 } 179 if !topicsMap[topicName] { 180 topicsMap[topicName] = true 181 topics = append(topics, topicName) 182 } 183 } 184 185 // We also need to add the default topic. 186 if !topicsMap[s.defaultTopic] { 187 topics = append(topics, s.defaultTopic) 188 } 189 190 return topics 191 } 192 193 // GetDefaultTopic returns the default topic name. 194 func (s *EventRouter) GetDefaultTopic() string { 195 return s.defaultTopic 196 } 197 198 // matchDispatcher returns the target topic dispatcher and partition dispatcher if a 199 // row changed event matches a specific table filter. 200 func (s *EventRouter) matchDispatcher( 201 schema, table string, 202 ) (topic.Dispatcher, partition.Dispatcher) { 203 for _, rule := range s.rules { 204 if !rule.MatchTable(schema, table) { 205 continue 206 } 207 return rule.topicDispatcher, rule.partitionDispatcher 208 } 209 log.Panic("the dispatch rule must cover all tables") 210 return nil, nil 211 } 212 213 // getPartitionDispatcher returns the partition dispatcher for a specific partition rule. 214 func getPartitionDispatcher( 215 rule string, scheme string, indexName string, columns []string, 216 ) partition.Dispatcher { 217 switch strings.ToLower(rule) { 218 case "default": 219 return partition.NewDefaultDispatcher() 220 case "ts": 221 return partition.NewTsDispatcher() 222 case "table": 223 return partition.NewTableDispatcher() 224 case "index-value": 225 return partition.NewIndexValueDispatcher(indexName) 226 case "rowid": 227 log.Warn("rowid is deprecated, index-value is used as the partition dispatcher.") 228 return partition.NewIndexValueDispatcher(indexName) 229 case "columns": 230 return partition.NewColumnsDispatcher(columns) 231 default: 232 } 233 234 if sink.IsPulsarScheme(scheme) { 235 return partition.NewKeyDispatcher(rule) 236 } 237 238 log.Warn("the partition dispatch rule is not default/ts/table/index-value/columns," + 239 " use the default rule instead.") 240 return partition.NewDefaultDispatcher() 241 } 242 243 // getTopicDispatcher returns the topic dispatcher for a specific topic rule (aka topic expression). 244 func getTopicDispatcher( 245 rule string, defaultTopic string, protocol config.Protocol, schema string, 246 ) (topic.Dispatcher, error) { 247 if rule == "" { 248 return topic.NewStaticTopicDispatcher(defaultTopic), nil 249 } 250 251 if topic.IsHardCode(rule) { 252 return topic.NewStaticTopicDispatcher(rule), nil 253 } 254 255 // check if this rule is a valid topic expression 256 topicExpr := topic.Expression(rule) 257 err := validateTopicExpression(topicExpr, schema, protocol) 258 if err != nil { 259 return nil, err 260 } 261 return topic.NewDynamicTopicDispatcher(topicExpr), nil 262 } 263 264 func validateTopicExpression(expr topic.Expression, scheme string, protocol config.Protocol) error { 265 if sink.IsPulsarScheme(scheme) { 266 return expr.PulsarValidate() 267 } 268 269 switch protocol { 270 case config.ProtocolAvro: 271 return expr.ValidateForAvro() 272 default: 273 } 274 275 return expr.Validate() 276 }