github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/binlog/reader/util.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 reader 15 16 import ( 17 "context" 18 19 gmysql "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/failpoint" 23 "github.com/pingcap/tiflow/dm/pkg/binlog/event" 24 "github.com/pingcap/tiflow/dm/pkg/gtid" 25 "github.com/pingcap/tiflow/dm/pkg/log" 26 "github.com/pingcap/tiflow/dm/pkg/parser" 27 "github.com/pingcap/tiflow/dm/pkg/terror" 28 "github.com/pingcap/tiflow/dm/pkg/utils" 29 "go.uber.org/zap" 30 ) 31 32 // GetGTIDsForPosFromStreamer tries to get GTID sets for the specified binlog position (for the corresponding txn) from a Streamer. 33 func GetGTIDsForPosFromStreamer(ctx context.Context, r Streamer, endPos gmysql.Position) (gmysql.GTIDSet, error) { 34 var ( 35 latestPos uint32 36 latestGSet gmysql.GTIDSet 37 nextGTIDStr string // can be recorded if the coming transaction completed 38 err error 39 ) 40 for { 41 var e *replication.BinlogEvent 42 e, err = r.GetEvent(ctx) 43 if err != nil { 44 return nil, err 45 } 46 47 // NOTE: only update endPos/GTIDs for DDL/XID to get an complete transaction. 48 switch ev := e.Event.(type) { 49 case *replication.QueryEvent: 50 parser2, err2 := event.GetParserForStatusVars(ev.StatusVars) 51 if err2 != nil { 52 log.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err2)) 53 } 54 55 isDDL := parser.CheckIsDDL(string(ev.Query), parser2) 56 if isDDL { 57 if latestGSet == nil { 58 // GTID not enabled, can't get GTIDs for the position. 59 return nil, errors.Errorf("should have a GTIDEvent before the DDL QueryEvent %+v", e.Header) 60 } 61 err2 = latestGSet.Update(nextGTIDStr) 62 if err2 != nil { 63 return nil, terror.Annotatef(err2, "update GTID set %v with GTID %s", latestGSet, nextGTIDStr) 64 } 65 latestPos = e.Header.LogPos 66 } 67 case *replication.XIDEvent: 68 if latestGSet == nil { 69 // GTID not enabled, can't get GTIDs for the position. 70 return nil, errors.Errorf("should have a GTIDEvent before the XIDEvent %+v", e.Header) 71 } 72 err = latestGSet.Update(nextGTIDStr) 73 if err != nil { 74 return nil, terror.Annotatef(err, "update GTID set %v with GTID %s", latestGSet, nextGTIDStr) 75 } 76 latestPos = e.Header.LogPos 77 case *replication.GTIDEvent: 78 if latestGSet == nil { 79 return nil, errors.Errorf("should have a PreviousGTIDsEvent before the GTIDEvent %+v", e.Header) 80 } 81 nextGTIDStr, err = event.GetGTIDStr(e) 82 if err != nil { 83 return nil, err 84 } 85 case *replication.MariadbGTIDEvent: 86 if latestGSet == nil { 87 return nil, errors.Errorf("should have a MariadbGTIDListEvent before the MariadbGTIDEvent %+v", e.Header) 88 } 89 nextGTIDStr, err = event.GetGTIDStr(e) 90 if err != nil { 91 return nil, err 92 } 93 case *replication.PreviousGTIDsEvent: 94 // if GTID enabled, we can get a PreviousGTIDEvent after the FormatDescriptionEvent 95 // ref: https://github.com/mysql/mysql-server/blob/8cc757da3d87bf4a1f07dcfb2d3c96fed3806870/sql/binlog.cc#L4549 96 // ref: https://github.com/mysql/mysql-server/blob/8cc757da3d87bf4a1f07dcfb2d3c96fed3806870/sql/binlog.cc#L5161 97 latestGSet, err = gtid.ParserGTID(gmysql.MySQLFlavor, ev.GTIDSets) 98 if err != nil { 99 return nil, err 100 } 101 latestPos = e.Header.LogPos 102 case *replication.MariadbGTIDListEvent: 103 // a MariadbGTIDListEvent logged in every binlog to record the current replication state if GTID enabled 104 // ref: https://mariadb.com/kb/en/library/gtid_list_event/ 105 latestGSet, err = event.GTIDsFromMariaDBGTIDListEvent(e) 106 if err != nil { 107 return nil, terror.Annotatef(err, "get GTID set from MariadbGTIDListEvent %+v", e.Header) 108 } 109 latestPos = e.Header.LogPos 110 } 111 112 if latestPos == endPos.Pos { 113 // reach the end position, return the GTID sets. 114 if latestGSet == nil { 115 return nil, errors.Errorf("no GTIDs get for position %s", endPos) 116 } 117 return latestGSet, nil 118 } else if latestPos > endPos.Pos { 119 return nil, errors.Errorf("invalid position %s or GTID not enabled in upstream", endPos) 120 } 121 } 122 } 123 124 // GetGTIDsForPos tries to get GTID sets for the specified binlog position (for the corresponding txn). 125 // NOTE: this method is very similar with `relay/writer/file_util.go/getTxnPosGTIDs`, unify them if needed later. 126 // NOTE: this method is not well tested directly, but more tests have already been done for `relay/writer/file_util.go/getTxnPosGTIDs`. 127 func GetGTIDsForPos(ctx context.Context, r Reader, endPos gmysql.Position) (gmysql.GTIDSet, error) { 128 // start to get and parse binlog event from the beginning of the file. 129 startPos := gmysql.Position{ 130 Name: endPos.Name, 131 Pos: 0, 132 } 133 err := r.StartSyncByPos(startPos) 134 if err != nil { 135 return nil, err 136 } 137 defer r.Close() 138 139 return GetGTIDsForPosFromStreamer(ctx, r, endPos) 140 } 141 142 // GetPreviousGTIDFromGTIDSet tries to get previous GTID sets from Previous_GTID_EVENT GTID for the specified GITD Set. 143 // events should be [fake_rotate_event,format_description_event,previous_gtids_event/mariadb_gtid_list_event]. 144 func GetPreviousGTIDFromGTIDSet(ctx context.Context, r Reader, gset gmysql.GTIDSet) (gmysql.GTIDSet, error) { 145 failpoint.Inject("MockGetEmptyPreviousGTIDFromGTIDSet", func(_ failpoint.Value) { 146 gset, _ = gtid.ParserGTID("mysql", "") 147 failpoint.Return(gset, nil) 148 }) 149 150 err := r.StartSyncByGTID(gset) 151 if err != nil { 152 return nil, err 153 } 154 defer r.Close() 155 156 for { 157 var e *replication.BinlogEvent 158 e, err = r.GetEvent(ctx) 159 if err != nil { 160 return nil, err 161 } 162 163 switch e.Header.EventType { 164 case replication.ROTATE_EVENT: 165 if utils.IsFakeRotateEvent(e.Header) { 166 continue 167 } 168 return nil, terror.ErrPreviousGTIDNotExist.Generate(gset.String()) 169 case replication.FORMAT_DESCRIPTION_EVENT: 170 continue 171 case replication.PREVIOUS_GTIDS_EVENT: 172 previousGset, err := event.GTIDsFromPreviousGTIDsEvent(e) 173 return previousGset, err 174 case replication.MARIADB_GTID_LIST_EVENT: 175 previousGset, err := event.GTIDsFromMariaDBGTIDListEvent(e) 176 return previousGset, err 177 default: 178 return nil, terror.ErrPreviousGTIDNotExist.Generate(gset.String()) 179 } 180 } 181 }