github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/job.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 "fmt" 18 "sync" 19 "time" 20 21 "github.com/go-mysql-org/go-mysql/replication" 22 "github.com/pingcap/tidb/pkg/util/filter" 23 "github.com/pingcap/tiflow/dm/pkg/binlog" 24 "github.com/pingcap/tiflow/pkg/sqlmodel" 25 ) 26 27 type opType byte 28 29 const ( 30 null opType = iota 31 dml 32 ddl 33 xid 34 flush 35 asyncFlush 36 skip // used by Syncer.recordSkipSQLsLocation to record global location, but not execute SQL 37 rotate 38 conflict 39 compact 40 gc // used to clean up out dated causality keys 41 ) 42 43 func (t opType) String() string { 44 switch t { 45 case dml: 46 return "dml" 47 case ddl: 48 return "ddl" 49 case xid: 50 return "xid" 51 case flush: 52 return "flush" 53 case asyncFlush: 54 return "asyncFlush" 55 case skip: 56 return "skip" 57 case rotate: 58 return "rotate" 59 case conflict: 60 return "conflict" 61 case compact: 62 return "compact" 63 case gc: 64 return "gc" 65 } 66 67 return "" 68 } 69 70 // job is the least unit of work for DMLWorker/DDL worker. It's generated by syncer's inner logic or binlog events. Specifically, most 71 // binlog events will generate one job except for DML events which will generate multiple jobs, one job for each row 72 // change. 73 type job struct { 74 tp opType 75 // ddl in ShardOptimistic and ShardPessimistic will only affect one table at one time but for normal node 76 // we don't have this limit. So we should update multi tables in normal mode. 77 // sql example: drop table `s1`.`t1`, `s2`.`t2`. 78 sourceTbls map[string][]*filter.Table 79 targetTable *filter.Table 80 dml *sqlmodel.RowChange 81 dmlQueueKey string 82 safeMode bool 83 retry bool 84 location binlog.Location // location of last received (ROTATE / QUERY / XID) event, for global/table checkpoint 85 startLocation binlog.Location // start location of the sql in binlog, for handle_error 86 currentLocation binlog.Location // end location of the sql in binlog, for user to skip sql manually by changing checkpoint 87 ddls []string 88 originSQL string // show origin sql when error, only DDL now 89 90 eventHeader *replication.EventHeader 91 jobAddTime time.Time // job commit time 92 flushSeq int64 // sequence number for sync and async flush job 93 flushWg *sync.WaitGroup // wait group for sync, async and conflict job 94 timestamp uint32 95 timezone string 96 } 97 98 func (j *job) clone() *job { 99 newJob := &job{} 100 *newJob = *j 101 return newJob 102 } 103 104 func (j *job) String() string { 105 // only output some important information, maybe useful in execution. 106 var dmlStr string 107 if j.dml != nil { 108 dmlStr = j.dml.String() 109 } 110 return fmt.Sprintf("tp: %s, flushSeq: %d, dml: [%s], safemode: %v, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.flushSeq, dmlStr, j.safeMode, j.ddls, j.location, j.startLocation, j.currentLocation) 111 } 112 113 func newDMLJob(rowChange *sqlmodel.RowChange, ec *eventContext) *job { 114 sourceTable := rowChange.GetSourceTable() 115 targetTable := rowChange.GetTargetTable() 116 // TODO: remove sourceTbls and targetTable for dml Job 117 return &job{ 118 tp: dml, 119 sourceTbls: map[string][]*filter.Table{ 120 sourceTable.Schema: { 121 &filter.Table{Schema: sourceTable.Schema, Name: sourceTable.Table}, 122 }, 123 }, 124 targetTable: &filter.Table{Schema: targetTable.Schema, Name: targetTable.Table}, 125 dml: rowChange, 126 retry: true, 127 safeMode: ec.safeMode, 128 129 location: ec.lastLocation, 130 startLocation: ec.startLocation, 131 currentLocation: ec.endLocation, 132 eventHeader: ec.header, 133 jobAddTime: time.Now(), 134 } 135 } 136 137 // newDDL job is used to create a new ddl job 138 // when cfg.ShardMode == ShardOptimistic || ShardPessimistic, len(qec.sourceTbls) == 0. 139 // when cfg.ShardMode == "", len(sourceTbls) != 0, we use sourceTbls to record ddl affected tables. 140 func newDDLJob(qec *queryEventContext) *job { 141 j := &job{ 142 tp: ddl, 143 targetTable: &filter.Table{}, 144 ddls: qec.needHandleDDLs, 145 originSQL: qec.originSQL, 146 147 location: qec.lastLocation, 148 startLocation: qec.startLocation, 149 currentLocation: qec.endLocation, 150 eventHeader: qec.header, 151 jobAddTime: time.Now(), 152 } 153 154 ddlInfo := qec.shardingDDLInfo 155 if len(qec.sourceTbls) != 0 { 156 j.sourceTbls = make(map[string][]*filter.Table, len(qec.sourceTbls)) 157 for schema, tbMap := range qec.sourceTbls { 158 if len(tbMap) > 0 { 159 j.sourceTbls[schema] = make([]*filter.Table, 0, len(tbMap)) 160 } 161 for name := range tbMap { 162 j.sourceTbls[schema] = append(j.sourceTbls[schema], &filter.Table{Schema: schema, Name: name}) 163 } 164 } 165 } else if ddlInfo != nil && ddlInfo.sourceTables != nil && ddlInfo.targetTables != nil { 166 j.sourceTbls = map[string][]*filter.Table{ddlInfo.sourceTables[0].Schema: {ddlInfo.sourceTables[0]}} 167 j.targetTable = ddlInfo.targetTables[0] 168 } 169 170 j.timestamp = qec.timestamp 171 j.timezone = qec.timezone 172 173 return j 174 } 175 176 func newSkipJob(ec *eventContext) *job { 177 return &job{ 178 tp: skip, 179 location: ec.lastLocation, 180 eventHeader: ec.header, 181 jobAddTime: time.Now(), 182 } 183 } 184 185 func newXIDJob(location, startLocation, currentLocation binlog.Location) *job { 186 return &job{ 187 tp: xid, 188 location: location, 189 startLocation: startLocation, 190 currentLocation: currentLocation, 191 jobAddTime: time.Now(), 192 } 193 } 194 195 func newFlushJob(workerCount int, seq int64) *job { 196 wg := &sync.WaitGroup{} 197 wg.Add(workerCount) 198 199 return &job{ 200 tp: flush, 201 targetTable: &filter.Table{}, 202 jobAddTime: time.Now(), 203 flushWg: wg, 204 flushSeq: seq, 205 } 206 } 207 208 func newAsyncFlushJob(workerCount int, seq int64) *job { 209 wg := &sync.WaitGroup{} 210 wg.Add(workerCount) 211 212 return &job{ 213 tp: asyncFlush, 214 targetTable: &filter.Table{}, 215 jobAddTime: time.Now(), 216 flushWg: wg, 217 flushSeq: seq, 218 } 219 } 220 221 func newGCJob(flushJobSeq int64) *job { 222 return &job{ 223 tp: gc, 224 flushSeq: flushJobSeq, 225 } 226 } 227 228 func newConflictJob(workerCount int) *job { 229 wg := &sync.WaitGroup{} 230 wg.Add(workerCount) 231 232 return &job{ 233 tp: conflict, 234 targetTable: &filter.Table{}, 235 jobAddTime: time.Now(), 236 flushWg: wg, 237 } 238 } 239 240 // newCompactJob is only used for MetricsProxies. 241 func newCompactJob(targetTable *filter.Table) *job { 242 return &job{ 243 tp: compact, 244 targetTable: targetTable, 245 } 246 } 247 248 // put queues into bucket to monitor them. 249 func queueBucketName(queueID int) string { 250 return fmt.Sprintf("q_%d", queueID%defaultBucketCount) 251 } 252 253 func dmlWorkerJobIdx(queueID int) int { 254 return queueID + workerJobTSArrayInitSize 255 }