github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/dispatcher/interface.go (about) 1 // Copyright 2020 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 "github.com/pingcap/ticdc/cdc/model" 21 "github.com/pingcap/ticdc/pkg/config" 22 cerror "github.com/pingcap/ticdc/pkg/errors" 23 filter "github.com/pingcap/tidb-tools/pkg/table-filter" 24 "go.uber.org/zap" 25 ) 26 27 // Dispatcher is an abstraction for dispatching rows into different partitions 28 type Dispatcher interface { 29 // Dispatch returns a index of partitions according to RowChangedEvent 30 Dispatch(row *model.RowChangedEvent) int32 31 } 32 33 type dispatchRule int 34 35 const ( 36 dispatchRuleDefault dispatchRule = iota 37 dispatchRuleRowID 38 dispatchRuleTS 39 dispatchRuleTable 40 dispatchRuleIndexValue 41 ) 42 43 func (r *dispatchRule) fromString(rule string) { 44 switch strings.ToLower(rule) { 45 case "default": 46 *r = dispatchRuleDefault 47 case "rowid": 48 *r = dispatchRuleRowID 49 case "ts": 50 *r = dispatchRuleTS 51 case "table": 52 *r = dispatchRuleTable 53 case "index-value": 54 *r = dispatchRuleIndexValue 55 default: 56 *r = dispatchRuleDefault 57 log.Warn("can't support dispatch rule, using default rule", zap.String("rule", rule)) 58 } 59 } 60 61 type dispatcherSwitcher struct { 62 rules []struct { 63 Dispatcher 64 filter.Filter 65 } 66 } 67 68 func (s *dispatcherSwitcher) Dispatch(row *model.RowChangedEvent) int32 { 69 return s.matchDispatcher(row).Dispatch(row) 70 } 71 72 func (s *dispatcherSwitcher) matchDispatcher(row *model.RowChangedEvent) Dispatcher { 73 for _, rule := range s.rules { 74 if !rule.MatchTable(row.Table.Schema, row.Table.Table) { 75 continue 76 } 77 return rule.Dispatcher 78 } 79 log.Panic("the dispatch rule must cover all tables") 80 return nil 81 } 82 83 // NewDispatcher creates a new dispatcher 84 func NewDispatcher(cfg *config.ReplicaConfig, partitionNum int32) (Dispatcher, error) { 85 ruleConfigs := append(cfg.Sink.DispatchRules, &config.DispatchRule{ 86 Matcher: []string{"*.*"}, 87 Dispatcher: "default", 88 }) 89 rules := make([]struct { 90 Dispatcher 91 filter.Filter 92 }, 0, len(ruleConfigs)) 93 94 for _, ruleConfig := range ruleConfigs { 95 f, err := filter.Parse(ruleConfig.Matcher) 96 if err != nil { 97 return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err) 98 } 99 if !cfg.CaseSensitive { 100 f = filter.CaseInsensitive(f) 101 } 102 var d Dispatcher 103 var rule dispatchRule 104 rule.fromString(ruleConfig.Dispatcher) 105 switch rule { 106 case dispatchRuleRowID, dispatchRuleIndexValue: 107 if cfg.EnableOldValue { 108 log.Warn("This index-value distribution mode " + 109 "does not guarantee row-level orderliness when " + 110 "switching on the old value, so please use caution!") 111 } 112 d = newIndexValueDispatcher(partitionNum) 113 case dispatchRuleTS: 114 d = newTsDispatcher(partitionNum) 115 case dispatchRuleTable: 116 d = newTableDispatcher(partitionNum) 117 case dispatchRuleDefault: 118 d = newDefaultDispatcher(partitionNum, cfg.EnableOldValue) 119 } 120 rules = append(rules, struct { 121 Dispatcher 122 filter.Filter 123 }{Dispatcher: d, Filter: f}) 124 } 125 return &dispatcherSwitcher{ 126 rules: rules, 127 }, nil 128 }