github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/causality.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 "math" 18 "time" 19 20 "github.com/pingcap/tidb/pkg/sessionctx" 21 "github.com/pingcap/tiflow/dm/pkg/log" 22 "github.com/pingcap/tiflow/dm/syncer/metrics" 23 "go.uber.org/zap" 24 ) 25 26 // causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. 27 // causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. 28 // if some conflicts exist in more than one groups, causality generate a conflict job and reset. 29 // this mechanism meets quiescent consistency to ensure correctness. 30 // causality relation is consisted of groups of keys separated by flush job, and such design helps removed flushed dml job keys. 31 type causality struct { 32 relation *causalityRelation 33 outCh chan *job 34 inCh chan *job 35 logger log.Logger 36 sessCtx sessionctx.Context 37 workerCount int 38 39 // for MetricsProxies 40 task string 41 source string 42 metricProxies *metrics.Proxies 43 } 44 45 // causalityWrap creates and runs a causality instance. 46 func causalityWrap(inCh chan *job, syncer *Syncer) chan *job { 47 causality := &causality{ 48 relation: newCausalityRelation(), 49 task: syncer.cfg.Name, 50 source: syncer.cfg.SourceID, 51 metricProxies: syncer.metricsProxies, 52 logger: syncer.tctx.Logger.WithFields(zap.String("component", "causality")), 53 inCh: inCh, 54 outCh: make(chan *job, syncer.cfg.QueueSize), 55 sessCtx: syncer.sessCtx, 56 workerCount: syncer.cfg.WorkerCount, 57 } 58 59 go func() { 60 causality.run() 61 causality.close() 62 }() 63 64 return causality.outCh 65 } 66 67 // run receives dml jobs and send causality jobs by adding causality key. 68 // When meet conflict, sends a conflict job. 69 func (c *causality) run() { 70 for j := range c.inCh { 71 c.metricProxies.QueueSizeGauge.WithLabelValues(c.task, "causality_input", c.source).Set(float64(len(c.inCh))) 72 73 startTime := time.Now() 74 75 switch j.tp { 76 case flush, asyncFlush: 77 c.relation.rotate(j.flushSeq) 78 case gc: 79 // gc is only used on inner-causality logic 80 c.relation.gc(j.flushSeq) 81 continue 82 default: 83 keys := j.dml.CausalityKeys() 84 85 // detectConflict before add 86 if c.detectConflict(keys) { 87 c.logger.Debug("meet causality key, will generate a conflict job to flush all sqls", zap.Strings("keys", keys)) 88 c.outCh <- newConflictJob(c.workerCount) 89 c.relation.clear() 90 } 91 j.dmlQueueKey = c.add(keys) 92 c.logger.Debug("key for keys", zap.String("key", j.dmlQueueKey), zap.Strings("keys", keys)) 93 } 94 c.metricProxies.Metrics.ConflictDetectDurationHistogram.Observe(time.Since(startTime).Seconds()) 95 96 c.outCh <- j 97 } 98 } 99 100 // close closes outer channel. 101 func (c *causality) close() { 102 close(c.outCh) 103 } 104 105 // add adds keys relation and return the relation. The keys must `detectConflict` first to ensure correctness. 106 func (c *causality) add(keys []string) string { 107 if len(keys) == 0 { 108 return "" 109 } 110 111 // find causal key 112 selectedRelation := keys[0] 113 var nonExistKeys []string 114 for _, key := range keys { 115 if val, ok := c.relation.get(key); ok { 116 selectedRelation = val 117 } else { 118 nonExistKeys = append(nonExistKeys, key) 119 } 120 } 121 // set causal relations for those non-exist keys 122 for _, key := range nonExistKeys { 123 c.relation.set(key, selectedRelation) 124 } 125 126 return selectedRelation 127 } 128 129 // detectConflict detects whether there is a conflict. 130 func (c *causality) detectConflict(keys []string) bool { 131 if len(keys) == 0 { 132 return false 133 } 134 135 var existedRelation string 136 for _, key := range keys { 137 if val, ok := c.relation.get(key); ok { 138 if existedRelation != "" && val != existedRelation { 139 return true 140 } 141 existedRelation = val 142 } 143 } 144 145 return false 146 } 147 148 // dmlJobKeyRelationGroup stores a group of dml job key relations as data, and a flush job seq representing last flush job before adding any job keys. 149 type dmlJobKeyRelationGroup struct { 150 data map[string]string 151 prevFlushJobSeq int64 152 } 153 154 // causalityRelation stores causality keys by group, where each group created on each flush and it helps to remove stale causality keys. 155 type causalityRelation struct { 156 groups []*dmlJobKeyRelationGroup 157 } 158 159 func newCausalityRelation() *causalityRelation { 160 m := &causalityRelation{} 161 m.rotate(-1) 162 return m 163 } 164 165 func (m *causalityRelation) get(key string) (string, bool) { 166 for i := len(m.groups) - 1; i >= 0; i-- { 167 if v, ok := m.groups[i].data[key]; ok { 168 return v, true 169 } 170 } 171 return "", false 172 } 173 174 func (m *causalityRelation) set(key string, val string) { 175 m.groups[len(m.groups)-1].data[key] = val 176 } 177 178 func (m *causalityRelation) len() int { 179 cnt := 0 180 for _, d := range m.groups { 181 cnt += len(d.data) 182 } 183 return cnt 184 } 185 186 func (m *causalityRelation) rotate(flushJobSeq int64) { 187 m.groups = append(m.groups, &dmlJobKeyRelationGroup{ 188 data: make(map[string]string), 189 prevFlushJobSeq: flushJobSeq, 190 }) 191 } 192 193 func (m *causalityRelation) clear() { 194 m.gc(math.MaxInt64) 195 } 196 197 // remove group of keys where its group's prevFlushJobSeq is smaller than or equal with the given flushJobSeq. 198 func (m *causalityRelation) gc(flushJobSeq int64) { 199 if flushJobSeq == math.MaxInt64 { 200 m.groups = m.groups[:0] 201 m.rotate(-1) 202 return 203 } 204 205 idx := 0 206 for i, d := range m.groups { 207 if d.prevFlushJobSeq <= flushJobSeq { 208 idx = i 209 } else { 210 break 211 } 212 } 213 214 m.groups = m.groups[idx:] 215 }