github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/v1dbschema/schema.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 v1dbschema 15 16 import ( 17 "fmt" 18 "strconv" 19 20 gmysql "github.com/go-mysql-org/go-mysql/mysql" 21 "github.com/go-mysql-org/go-mysql/replication" 22 "github.com/go-sql-driver/mysql" 23 "github.com/pingcap/errors" 24 "github.com/pingcap/failpoint" 25 "github.com/pingcap/tidb/pkg/errno" 26 "github.com/pingcap/tidb/pkg/util/dbutil" 27 "github.com/pingcap/tiflow/dm/config" 28 "github.com/pingcap/tiflow/dm/pkg/binlog" 29 "github.com/pingcap/tiflow/dm/pkg/binlog/reader" 30 "github.com/pingcap/tiflow/dm/pkg/conn" 31 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 32 "github.com/pingcap/tiflow/dm/pkg/cputil" 33 "github.com/pingcap/tiflow/dm/pkg/gtid" 34 "github.com/pingcap/tiflow/dm/pkg/log" 35 "github.com/pingcap/tiflow/dm/pkg/terror" 36 "github.com/pingcap/tiflow/dm/pkg/utils" 37 "go.uber.org/zap" 38 ) 39 40 // UpdateSchema updates the DB schema from v1.0.x to v2.0.x, including: 41 // - update checkpoint. 42 // - update online DDL meta. 43 func UpdateSchema(tctx *tcontext.Context, db *conn.BaseDB, cfg *config.SubTaskConfig) error { 44 // get db connection. 45 dbConn, err := db.GetBaseConn(tctx.Context()) 46 if err != nil { 47 return terror.ErrFailUpdateV1DBSchema.Delegate(err) 48 } 49 defer func() { 50 err2 := db.ForceCloseConn(dbConn) 51 if err2 != nil { 52 tctx.L().Warn("fail to close connection", zap.Error(err2)) 53 } 54 }() 55 56 // setup a TCP binlog reader (because no relay can be used when upgrading). 57 syncCfg := replication.BinlogSyncerConfig{ 58 ServerID: cfg.ServerID, 59 Flavor: cfg.Flavor, 60 Host: cfg.From.Host, 61 Port: uint16(cfg.From.Port), 62 User: cfg.From.User, 63 Password: cfg.From.Password, // plaintext. 64 UseDecimal: false, 65 VerifyChecksum: true, 66 } 67 tcpReader := reader.NewTCPReader(syncCfg) 68 69 // update checkpoint. 70 err = updateSyncerCheckpoint(tctx, dbConn, cfg.Name, dbutil.TableName(cfg.MetaSchema, cputil.SyncerCheckpoint(cfg.Name)), cfg.SourceID, cfg.EnableGTID, tcpReader) 71 if err != nil { 72 return terror.ErrFailUpdateV1DBSchema.Delegate(err) 73 } 74 75 // update online DDL meta. 76 err = updateSyncerOnlineDDLMeta(tctx, dbConn, cfg.Name, dbutil.TableName(cfg.MetaSchema, cputil.SyncerOnlineDDL(cfg.Name)), cfg.SourceID, cfg.ServerID) 77 return terror.ErrFailUpdateV1DBSchema.Delegate(err) 78 } 79 80 // updateSyncerCheckpoint updates the checkpoint table of sync unit, including: 81 // - update table schema: 82 // - add column `binlog_gtid VARCHAR(256)`. 83 // - add column `table_info JSON NOT NULL`. 84 // 85 // - update column value: 86 // - fill `binlog_gtid` based on `binlog_name` and `binlog_pos` if GTID mode enable. 87 // 88 // NOTE: no need to update the value of `table_info` because DM can get schema automatically from downstream when replicating DML. 89 func updateSyncerCheckpoint(tctx *tcontext.Context, dbConn *conn.BaseConn, taskName, tableName, sourceID string, fillGTIDs bool, tcpReader reader.Reader) error { 90 logger := log.L().WithFields(zap.String("task", taskName), zap.String("source", sourceID)) 91 logger.Info("updating syncer checkpoint", zap.Bool("fill GTID", fillGTIDs)) 92 var gs gmysql.GTIDSet 93 if fillGTIDs { 94 // NOTE: get GTID sets for all (global & tables) binlog position has many problems, at least including: 95 // - it is a heavy work because it should read binlog events once for each position 96 // - some binlog file for the position may have already been purge 97 // so we only get GTID sets for the global position now, 98 // and this should only have side effects for in-syncing shard tables, but we can mention and warn this case in the user docs. 99 pos, err := getGlobalPos(tctx, dbConn, tableName, sourceID) 100 logger.Info("got global checkpoint position", zap.Stringer("position", pos)) 101 if err != nil { 102 return terror.Annotatef(err, "get global checkpoint position for source %s", sourceID) 103 } 104 if pos.Name != "" { 105 gs, err = getGTIDsForPos(tctx, pos, tcpReader) 106 if err != nil { 107 return terror.Annotatef(err, "get GTID sets for position %s", pos) 108 } 109 gs, err = conn.AddGSetWithPurged(tctx.Context(), gs, dbConn) 110 if err != nil { 111 return terror.Annotatef(err, "get GTID sets for position %s", pos) 112 } 113 logger.Info("got global checkpoint GTID sets", log.WrapStringerField("GTID sets", gs)) 114 } 115 } 116 117 // try to add columns. 118 // NOTE: ignore already exists error to continue the process. 119 queries := []string{ 120 fmt.Sprintf(`ALTER TABLE %s ADD COLUMN binlog_gtid TEXT AFTER binlog_pos`, tableName), 121 fmt.Sprintf(`ALTER TABLE %s ADD COLUMN table_info JSON NOT NULL AFTER binlog_gtid`, tableName), 122 } 123 updateTaskName := "importFromV10x" 124 _, err := dbConn.ExecuteSQLWithIgnoreError(tctx, nil, updateTaskName, utils.IgnoreErrorCheckpoint, queries) 125 if err != nil { 126 return terror.Annotatef(err, "add columns for checkpoint table") 127 } 128 129 if fillGTIDs && gs != nil { 130 // set binlog_gtid, `gs` should valid here. 131 err = setGlobalGTIDs(tctx, dbConn, taskName, tableName, sourceID, gs.String()) 132 if err != nil { 133 return terror.Annotatef(err, "set GTID sets %s for checkpoint table", gs.String()) 134 } 135 logger.Info("filled global checkpoint GTID sets", zap.Stringer("GTID sets", gs)) 136 } 137 return nil 138 } 139 140 // updateSyncerOnlineDDLMeta updates the online DDL meta data, including: 141 // - update the value of `id` from `server-id` to `source-id`. 142 // NOTE: online DDL may not exist if not enabled. 143 func updateSyncerOnlineDDLMeta(tctx *tcontext.Context, dbConn *conn.BaseConn, taskName, tableName, sourceID string, serverID uint32) error { 144 logger := log.L().WithFields(zap.String("task", taskName), zap.String("source", sourceID)) 145 logger.Info("updating syncer online DDL meta") 146 queries := []string{ 147 fmt.Sprintf(`UPDATE %s SET id=? WHERE id=?`, tableName), // for multiple columns. 148 } 149 args := []interface{}{sourceID, strconv.FormatUint(uint64(serverID), 10)} 150 _, err := dbConn.ExecuteSQLWithIgnoreError(tctx, nil, taskName, ignoreErrorOnlineDDL, queries, args) 151 return terror.Annotatef(err, "update id column for online DDL meta table") 152 } 153 154 // getGlobalPos tries to get the global checkpoint position. 155 func getGlobalPos(tctx *tcontext.Context, dbConn *conn.BaseConn, tableName, sourceID string) (gmysql.Position, error) { 156 query := fmt.Sprintf(`SELECT binlog_name, binlog_pos FROM %s WHERE id=? AND is_global=? LIMIT 1`, tableName) 157 args := []interface{}{sourceID, true} 158 rows, err := dbConn.QuerySQL(tctx, query, args...) 159 if err != nil { 160 return gmysql.Position{}, err 161 } 162 defer rows.Close() 163 if !rows.Next() { 164 return gmysql.Position{}, nil // no global checkpoint position exists. 165 } 166 167 var ( 168 name string 169 pos uint32 170 ) 171 err = rows.Scan(&name, &pos) 172 if err != nil { 173 return gmysql.Position{}, err 174 } 175 176 return gmysql.Position{ 177 Name: name, 178 Pos: pos, 179 }, rows.Err() 180 } 181 182 // getGTIDsForPos gets the GTID sets for the position. 183 func getGTIDsForPos(tctx *tcontext.Context, pos gmysql.Position, tcpReader reader.Reader) (gs gmysql.GTIDSet, err error) { 184 // NOTE: because we have multiple unit test cases updating/clearing binlog in the upstream, 185 // we may encounter errors when reading binlog event but cleared by another test case. 186 failpoint.Inject("MockGetGTIDsForPos", func(val failpoint.Value) { 187 str := val.(string) 188 gs, _ = gtid.ParserGTID(gmysql.MySQLFlavor, str) 189 tctx.L().Info("set gs for position", zap.String("failpoint", "MockGetGTIDsForPos"), zap.Stringer("pos", pos)) 190 failpoint.Return(gs, nil) 191 }) 192 193 var realPos gmysql.Position 194 realPos, err = binlog.RealMySQLPos(pos) 195 if err != nil { 196 return nil, err 197 } 198 gs, err = reader.GetGTIDsForPos(tctx.Ctx, tcpReader, realPos) 199 if err != nil { 200 return nil, err 201 } 202 return gs, nil 203 } 204 205 // setGlobalGTIDs tries to set `binlog_gtid` for the global checkpoint. 206 func setGlobalGTIDs(tctx *tcontext.Context, dbConn *conn.BaseConn, taskName, tableName, sourceID, gs string) error { 207 queries := []string{ 208 fmt.Sprintf(`UPDATE %s SET binlog_gtid=? WHERE id=? AND is_global=? LIMIT 1`, tableName), 209 } 210 args := []interface{}{gs, sourceID, true} 211 _, err := dbConn.ExecuteSQL(tctx, nil, taskName, queries, args) 212 return err 213 } 214 215 func ignoreErrorOnlineDDL(err error) bool { 216 err = errors.Cause(err) // check the original error 217 mysqlErr, ok := err.(*mysql.MySQLError) 218 if !ok { 219 return false 220 } 221 222 switch mysqlErr.Number { 223 case errno.ErrNoSuchTable: 224 return true 225 default: 226 return false 227 } 228 }