github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/binlog/pos_finder.go (about) 1 // Copyright 2021 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 binlog 15 16 import ( 17 "path" 18 19 "github.com/go-mysql-org/go-mysql/mysql" 20 "github.com/go-mysql-org/go-mysql/replication" 21 "github.com/pingcap/errors" 22 "github.com/pingcap/tiflow/dm/pkg/binlog/common" 23 "github.com/pingcap/tiflow/dm/pkg/binlog/event" 24 "github.com/pingcap/tiflow/dm/pkg/binlog/reader" 25 "github.com/pingcap/tiflow/dm/pkg/conn" 26 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 27 "github.com/pingcap/tiflow/dm/pkg/gtid" 28 "github.com/pingcap/tiflow/dm/pkg/utils" 29 "go.uber.org/zap" 30 ) 31 32 // FakeBinlogName is used to bypass the checking of meta in task config when start-task with --start-time. 33 const FakeBinlogName = "start-task with --start-time" 34 35 type binlogPosFinder struct { 36 remote bool 37 tctx *tcontext.Context 38 enableGTID bool 39 parser *replication.BinlogParser 40 flavor string 41 42 // fields used for remote mode 43 db *conn.BaseDB 44 syncCfg replication.BinlogSyncerConfig 45 46 // fields used for local relay 47 relayDir string // should be a directory with current UUID 48 49 // fields used inside FindByTimestamp 50 targetBinlog binlogSize // target binlog file the timestamp may reside 51 tsBeforeFirstBinlog bool // whether the timestamp is before the first binlog 52 lastBinlogFile bool // whether targetBinlog is the last binlog file 53 54 // one binlog file can either be GTID enabled or not, cannot be mixed up 55 // we mark it using this field to avoid parsing events. 56 everMetGTIDEvent bool 57 inTransaction bool // whether in transaction 58 } 59 60 type PosType int 61 62 func (b PosType) String() string { 63 switch b { 64 case BelowLowerBoundBinlogPos: 65 return "BelowLowerBound" 66 case InRangeBinlogPos: 67 return "InRange" 68 case AboveUpperBoundBinlogPos: 69 return "AboveUpperBound" 70 } 71 return "Invalid" 72 } 73 74 const ( 75 InvalidBinlogPos PosType = iota 76 BelowLowerBoundBinlogPos 77 InRangeBinlogPos 78 AboveUpperBoundBinlogPos 79 ) 80 81 func NewLocalBinlogPosFinder(tctx *tcontext.Context, enableGTID bool, flavor string, relayDir string) *binlogPosFinder { 82 parser := replication.NewBinlogParser() 83 parser.SetFlavor(flavor) 84 parser.SetVerifyChecksum(true) 85 86 return &binlogPosFinder{ 87 remote: false, 88 tctx: tctx, 89 enableGTID: enableGTID, 90 parser: parser, 91 flavor: flavor, 92 93 relayDir: relayDir, 94 } 95 } 96 97 func NewRemoteBinlogPosFinder(tctx *tcontext.Context, db *conn.BaseDB, syncCfg replication.BinlogSyncerConfig, enableGTID bool) *binlogPosFinder { 98 // make sure raw mode enabled, and MaxReconnectAttempts set 99 syncCfg.RawModeEnabled = true 100 if syncCfg.MaxReconnectAttempts == 0 { 101 syncCfg.MaxReconnectAttempts = common.MaxBinlogSyncerReconnect 102 } 103 104 parser := replication.NewBinlogParser() 105 parser.SetFlavor(syncCfg.Flavor) 106 parser.SetVerifyChecksum(true) 107 108 return &binlogPosFinder{ 109 remote: true, 110 tctx: tctx, 111 enableGTID: enableGTID, 112 parser: parser, 113 flavor: syncCfg.Flavor, 114 115 db: db, 116 syncCfg: syncCfg, 117 } 118 } 119 120 func (r *binlogPosFinder) getBinlogFiles() (FileSizes, error) { 121 if r.remote { 122 return GetBinaryLogs(r.tctx, r.db) 123 } 124 return GetLocalBinaryLogs(r.relayDir) 125 } 126 127 func (r *binlogPosFinder) startSync(position mysql.Position) (reader.Reader, error) { 128 if r.remote { 129 binlogReader := reader.NewTCPReader(r.syncCfg) 130 return binlogReader, binlogReader.StartSyncByPos(position) 131 } 132 binlogReader := reader.NewFileReader(&reader.FileReaderConfig{EnableRawMode: true}) 133 position.Name = path.Join(r.relayDir, position.Name) 134 return binlogReader, binlogReader.StartSyncByPos(position) 135 } 136 137 func (r *binlogPosFinder) findMinTimestampOfBinlog(currBinlog binlogSize) (uint32, error) { 138 var minTS uint32 139 binlogReader, err := r.startSync(mysql.Position{Name: currBinlog.name, Pos: FileHeaderLen}) 140 if err != nil { 141 return 0, err 142 } 143 for { 144 ev, err := binlogReader.GetEvent(r.tctx.Ctx) 145 if err != nil { 146 binlogReader.Close() 147 return 0, err 148 } 149 // break on first non-fake event(must be a format description event) 150 if !utils.IsFakeRotateEvent(ev.Header) { 151 minTS = ev.Header.Timestamp 152 break 153 } 154 } 155 binlogReader.Close() 156 157 return minTS, nil 158 } 159 160 func (r *binlogPosFinder) initTargetBinlogFile(ts int64) error { 161 targetTS := uint32(ts) 162 var lastTS, minTS uint32 163 var lastMid int 164 binaryLogs, err := r.getBinlogFiles() 165 if err != nil { 166 return err 167 } 168 if len(binaryLogs) == 0 { 169 // should not happen on a master with binlog enabled 170 return errors.New("cannot find binlog files") 171 } 172 173 begin, end := 0, len(binaryLogs)-1 174 for begin <= end { 175 mid := (begin + end) / 2 176 currBinlog := binaryLogs[mid] 177 178 minTS, err = r.findMinTimestampOfBinlog(currBinlog) 179 if err != nil { 180 return err 181 } 182 183 r.tctx.L().Debug("min timestamp in binlog file", zap.Reflect("file", currBinlog), zap.Uint32("ts", minTS)) 184 185 lastTS = minTS 186 lastMid = mid 187 188 if minTS >= targetTS { 189 end = mid - 1 190 } else { 191 // current binlog maybe the target binlog file, we'll backtrace to it later. 192 begin = mid + 1 193 } 194 } 195 if lastTS >= targetTS { 196 if lastMid == 0 { 197 // timestamp of first binlog event in first binlog file >= targetTS 198 r.targetBinlog = binaryLogs[lastMid] 199 r.tsBeforeFirstBinlog = true 200 } else { 201 // timestamp of first event in lastMid >= targetTS, need to search from previous binlog file 202 r.targetBinlog = binaryLogs[lastMid-1] 203 } 204 } else { 205 r.targetBinlog = binaryLogs[lastMid] 206 } 207 r.lastBinlogFile = r.targetBinlog.name == binaryLogs[len(binaryLogs)-1].name 208 209 r.tctx.L().Info("target binlog file", zap.Reflect("file", r.targetBinlog), 210 zap.Bool("before first binlog", r.tsBeforeFirstBinlog), 211 zap.Bool("last binlog", r.lastBinlogFile)) 212 213 return nil 214 } 215 216 func (r *binlogPosFinder) processGTIDRelatedEvent(ev *replication.BinlogEvent, prevSet mysql.GTIDSet) (mysql.GTIDSet, error) { 217 ev, err := r.parser.Parse(ev.RawData) 218 if err != nil { 219 return nil, err 220 } 221 switch ev.Header.EventType { 222 case replication.PREVIOUS_GTIDS_EVENT: 223 newSet, err := event.GTIDsFromPreviousGTIDsEvent(ev) 224 if err != nil { 225 return nil, err 226 } 227 return newSet, nil 228 case replication.MARIADB_GTID_LIST_EVENT: 229 newSet, err := event.GTIDsFromMariaDBGTIDListEvent(ev) 230 if err != nil { 231 return nil, err 232 } 233 return newSet, nil 234 case replication.MARIADB_GTID_EVENT, replication.GTID_EVENT: 235 gtidStr, _ := event.GetGTIDStr(ev) 236 if err := prevSet.Update(gtidStr); err != nil { 237 return nil, err 238 } 239 } 240 return prevSet, nil 241 } 242 243 func (r *binlogPosFinder) checkTransactionBeginEvent(ev *replication.BinlogEvent) (bool, error) { 244 // we find the timestamp at transaction boundary 245 // When there are GTID events in this binlog file, we use GTID event as the start event, else: 246 // for DML 247 // take a 'BEGIN' query event as the start event 248 // XID event or a 'COMMIT' query event as the end event 249 // for DDL 250 // one single query event acts as both the start and end event 251 var transactionBeginEvent bool 252 switch ev.Header.EventType { 253 case replication.FORMAT_DESCRIPTION_EVENT: 254 _, err := r.parser.Parse(ev.RawData) 255 if err != nil { 256 return false, err 257 } 258 case replication.GTID_EVENT, replication.ANONYMOUS_GTID_EVENT, replication.MARIADB_GTID_EVENT: 259 // since 5.7, when GTID not enabled, mysql add a anonymous gtid event. we use this to avoid parsing query event 260 r.everMetGTIDEvent = true 261 transactionBeginEvent = true 262 case replication.QUERY_EVENT: 263 if !r.everMetGTIDEvent { 264 // user may change session level binlog-format=statement, but it's an unusual operation, so we parse it every time 265 // In MySQL 5.6.x without GTID, the timestamp of BEGIN is the timestamp of the first statement in the transaction, 266 // not the commit timestamp of the transaction. 267 // To simplify implementation, we use timestamp of BEGIN as the transaction timestamp, 268 // but this may cause some transaction with timestamp >= target timestamp be skipped. 269 // TODO maybe add backtrace to support this case later 270 ev2, err := r.parser.Parse(ev.RawData) 271 if err != nil { 272 return false, err 273 } 274 e := ev2.Event.(*replication.QueryEvent) 275 switch string(e.Query) { 276 case "BEGIN": 277 transactionBeginEvent = true 278 r.inTransaction = true 279 case "COMMIT": // MyISAM use COMMIT to end transaction 280 r.inTransaction = false 281 default: 282 if !r.inTransaction { 283 // DDL 284 transactionBeginEvent = true 285 } 286 } 287 } 288 case replication.XID_EVENT: 289 r.inTransaction = false 290 } 291 return transactionBeginEvent, nil 292 } 293 294 // FindByTimestamp get binlog location of first event or transaction with timestamp >= ts 295 // go-mysql has BinlogStreamer.GetEventWithStartTime, but it doesn't fit our need. And we need to support relay log. 296 // if posType != AboveUpperBoundBinlogPos, then location is the target location we want. 297 // if posType == BelowLowerBoundBinlogPos, master binlog may have purged. 298 func (r *binlogPosFinder) FindByTimestamp(ts int64) (*Location, PosType, error) { 299 r.tctx.L().Info("target timestamp", zap.Int64("ts", ts)) 300 301 if err := r.initTargetBinlogFile(ts); err != nil { 302 return nil, InvalidBinlogPos, err 303 } 304 305 targetTS := uint32(ts) 306 position := mysql.Position{Name: r.targetBinlog.name, Pos: FileHeaderLen} 307 gtidSet, err := gtid.ZeroGTIDSet(r.flavor) 308 if err != nil { 309 return nil, InvalidBinlogPos, err 310 } 311 312 binlogReader, err := r.startSync(position) 313 if err != nil { 314 return nil, InvalidBinlogPos, err 315 } 316 defer binlogReader.Close() 317 for { 318 ev, err := binlogReader.GetEvent(r.tctx.Ctx) 319 // let outer layer retry 320 if err != nil { 321 return nil, InvalidBinlogPos, err 322 } 323 if utils.IsFakeRotateEvent(ev.Header) { 324 continue 325 } 326 327 transactionBeginEvent, err := r.checkTransactionBeginEvent(ev) 328 if err != nil { 329 return nil, InvalidBinlogPos, err 330 } 331 332 if transactionBeginEvent && ev.Header.Timestamp >= targetTS { 333 break 334 } 335 position.Pos = ev.Header.LogPos 336 337 if r.enableGTID { 338 eventType := ev.Header.EventType 339 if eventType == replication.PREVIOUS_GTIDS_EVENT || 340 eventType == replication.MARIADB_GTID_LIST_EVENT || 341 eventType == replication.GTID_EVENT || 342 eventType == replication.MARIADB_GTID_EVENT { 343 gtidSet, err = r.processGTIDRelatedEvent(ev, gtidSet) 344 if err != nil { 345 return nil, InvalidBinlogPos, err 346 } 347 // we meet PREVIOUS_GTIDS_EVENT or MARIADB_GTID_LIST_EVENT first, so break after get previous GTIDs 348 if r.tsBeforeFirstBinlog { 349 break 350 } 351 } 352 } 353 354 // still not found the timestamp after reached the end of this binlog file 355 if int64(position.Pos) >= r.targetBinlog.size { 356 // if it's the last binlog file, then this timestamp is out of range, 357 // else the end of this binlog file is the position we want, 358 // since the timestamp of the first event in next binlog >= target timestamp 359 if r.lastBinlogFile { 360 return nil, AboveUpperBoundBinlogPos, nil 361 } 362 break 363 } 364 } 365 if r.tsBeforeFirstBinlog { 366 // always return the position of the first event in target binlog 367 loc := NewLocation(mysql.Position{Name: r.targetBinlog.name, Pos: FileHeaderLen}, gtidSet) 368 return &loc, BelowLowerBoundBinlogPos, nil 369 } 370 loc := NewLocation(position, gtidSet) 371 return &loc, InRangeBinlogPos, nil 372 }