github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/error.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 "database/sql" 18 "fmt" 19 "strings" 20 "time" 21 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/infoschema" 27 "github.com/pingcap/tidb/pkg/parser" 28 "github.com/pingcap/tidb/pkg/parser/ast" 29 "github.com/pingcap/tidb/pkg/parser/model" 30 tmysql "github.com/pingcap/tidb/pkg/parser/mysql" 31 "github.com/pingcap/tidb/pkg/util/dbterror" 32 "github.com/pingcap/tidb/pkg/util/dbutil" 33 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 34 "github.com/pingcap/tiflow/dm/pkg/log" 35 "github.com/pingcap/tiflow/dm/pkg/terror" 36 "github.com/pingcap/tiflow/dm/syncer/dbconn" 37 "github.com/pingcap/tiflow/pkg/errorutil" 38 "go.uber.org/zap" 39 ) 40 41 // ignoreTrackerDDLError is also same with ignoreDDLError, but in order to keep tracker's table structure same as 42 // upstream's, we can't ignore "already exists" errors because already exists doesn't mean same. 43 func ignoreTrackerDDLError(err error) bool { 44 switch { 45 case infoschema.ErrDatabaseExists.Equal(err), infoschema.ErrDatabaseDropExists.Equal(err), 46 infoschema.ErrTableDropExists.Equal(err), 47 dbterror.ErrCantDropFieldOrKey.Equal(err): 48 return true 49 default: 50 return false 51 } 52 } 53 54 func isDropColumnWithIndexError(err error) bool { 55 mysqlErr, ok := errors.Cause(err).(*mysql.MySQLError) 56 if !ok { 57 return false 58 } 59 // different version of TiDB has different error message, try to cover most versions 60 return (mysqlErr.Number == errno.ErrUnsupportedDDLOperation || mysqlErr.Number == tmysql.ErrUnknown) && 61 strings.Contains(mysqlErr.Message, "drop column") && 62 (strings.Contains(mysqlErr.Message, "with index") || 63 strings.Contains(mysqlErr.Message, "with composite index") || 64 strings.Contains(mysqlErr.Message, "with tidb_enable_change_multi_schema is disable")) 65 } 66 67 // handleSpecialDDLError handles special errors for DDL execution 68 // if createTime equals to -1, skip the handle procedure for waitAsyncDDL. 69 func (s *Syncer) handleSpecialDDLError(tctx *tcontext.Context, err error, ddls []string, index int, conn *dbconn.DBConn, createTime int64) error { 70 // We use default parser because ddls are came from *Syncer.genDDLInfo, which is StringSingleQuotes, KeyWordUppercase and NameBackQuotes 71 parser2 := parser.New() 72 73 // it only ignore `invalid connection` error (timeout or other causes) for `ADD INDEX`. 74 // `invalid connection` means some data already sent to the server, 75 // and we assume that the whole SQL statement has already sent to the server for this error. 76 // if we have other methods to judge the DDL dispatched but timeout for executing, we can update this method. 77 // NOTE: we must ensure other PK/UK exists for correctness. 78 // NOTE: when we are refactoring the shard DDL algorithm, we also need to consider supporting non-blocking `ADD INDEX`. 79 ignoreAddIndexTimeout := func(tctx *tcontext.Context, err error, ddls []string, index int, conn *dbconn.DBConn, createTime int64) error { 80 // must ensure only the last statement executed failed with the `invalid connection` error 81 if len(ddls) == 0 || index != len(ddls)-1 || errors.Cause(err) != mysql.ErrInvalidConn { 82 return err // return the original error 83 } 84 85 ddl2 := ddls[index] 86 stmt, err2 := parser2.ParseOneStmt(ddl2, "", "") 87 if err2 != nil { 88 return err // return the original error 89 } 90 91 handle := func() { 92 tctx.L().Warn("ignore special error for DDL", zap.String("DDL", ddl2), log.ShortError(err)) 93 err2 := conn.ResetConn(tctx) // also reset the `invalid connection` for later use. 94 if err2 != nil { 95 tctx.L().Warn("reset connection failed", log.ShortError(err2)) 96 } 97 } 98 99 switch v := stmt.(type) { 100 case *ast.AlterTableStmt: 101 // ddls should be split with only one spec 102 if len(v.Specs) > 1 { 103 return err 104 } else if v.Specs[0].Tp == ast.AlterTableAddConstraint { 105 // only take effect on `ADD INDEX`, no UNIQUE KEY and FOREIGN KEY 106 // UNIQUE KEY may affect correctness, FOREIGN KEY should be filtered. 107 // ref https://github.com/pingcap/tidb/blob/3cdea0dfdf28197ee65545debce8c99e6d2945e3/ddl/ddl_api.go#L1929-L1948. 108 switch v.Specs[0].Constraint.Tp { 109 case ast.ConstraintKey, ast.ConstraintIndex: 110 handle() 111 return nil // ignore the error 112 } 113 } 114 case *ast.CreateIndexStmt: 115 handle() 116 return nil // ignore the error 117 } 118 return err 119 } 120 121 // for DROP COLUMN with its single-column index, try drop index first then drop column 122 // TiDB will support DROP COLUMN with index soon. After its support, executing that SQL will not have an error, 123 // so this function will not trigger and cause some trouble 124 dropColumnF := func(tctx *tcontext.Context, originErr error, ddls []string, index int, conn *dbconn.DBConn, createTime int64) error { 125 if !isDropColumnWithIndexError(originErr) { 126 return originErr 127 } 128 ddl2 := ddls[index] 129 stmt, err2 := parser2.ParseOneStmt(ddl2, "", "") 130 if err2 != nil { 131 return originErr // return the original error 132 } 133 134 var ( 135 schema string 136 table string 137 col string 138 ) 139 n, ok := stmt.(*ast.AlterTableStmt) 140 switch { 141 case !ok: 142 return originErr 143 case len(n.Specs) != 1: 144 return originErr 145 case n.Specs[0].Tp != ast.AlterTableDropColumn: 146 return originErr 147 default: 148 schema = n.Table.Schema.O 149 table = n.Table.Name.O 150 col = n.Specs[0].OldColumnName.Name.O 151 } 152 tctx.L().Warn("try to fix drop column error", zap.String("DDL", ddl2), log.ShortError(originErr)) 153 154 // check if dependent index is single-column index on this column 155 sql2 := "SELECT INDEX_NAME FROM information_schema.statistics WHERE TABLE_SCHEMA = ? and TABLE_NAME = ? and COLUMN_NAME = ?" 156 var rows *sql.Rows 157 rows, err2 = conn.QuerySQL(tctx, s.metricsProxies, sql2, schema, table, col) 158 if err2 != nil { 159 return originErr 160 } 161 var ( 162 idx string 163 idx2Check []string 164 idx2Drop []string 165 count int 166 ) 167 for rows.Next() { 168 if err2 = rows.Scan(&idx); err2 != nil { 169 // nolint:sqlclosecheck 170 rows.Close() 171 return originErr 172 } 173 idx2Check = append(idx2Check, idx) 174 } 175 if rows.Err() != nil { 176 return rows.Err() 177 } 178 // Close is idempotent, we could close in advance to reuse conn 179 rows.Close() 180 181 sql2 = "SELECT count(*) FROM information_schema.statistics WHERE TABLE_SCHEMA = ? and TABLE_NAME = ? and INDEX_NAME = ?" 182 for _, idx := range idx2Check { 183 rows, err2 = conn.QuerySQL(tctx, s.metricsProxies, sql2, schema, table, idx) 184 if err2 != nil || !rows.Next() || rows.Scan(&count) != nil || count != 1 { 185 tctx.L().Warn("can't auto drop index", zap.String("index", idx)) 186 if rows != nil { 187 // nolint: sqlclosecheck 188 rows.Close() 189 } 190 return originErr 191 } 192 if rows.Err() != nil { 193 return rows.Err() 194 } 195 idx2Drop = append(idx2Drop, idx) 196 rows.Close() 197 } 198 199 sqls := make([]string, len(idx2Drop)) 200 for i, idx := range idx2Drop { 201 sqls[i] = fmt.Sprintf("ALTER TABLE %s DROP INDEX %s", dbutil.TableName(schema, table), dbutil.ColumnName(idx)) 202 } 203 if _, err2 = conn.ExecuteSQL(tctx, s.metricsProxies, sqls); err2 != nil { 204 tctx.L().Warn("auto drop index failed", log.ShortError(err2)) 205 return originErr 206 } 207 208 tctx.L().Info("drop index success, now try to drop column", zap.Strings("index", idx2Drop)) 209 if _, err2 = conn.ExecuteSQLWithIgnore(tctx, s.metricsProxies, errorutil.IsIgnorableMySQLDDLError, ddls[index:]); err2 != nil { 210 return err2 211 } 212 213 tctx.L().Info("execute drop column SQL success", zap.String("DDL", ddl2)) 214 return nil 215 } 216 // TODO: add support for downstream alter pk without schema 217 218 // it handles the operations for DDL when encountering `invalid connection` by waiting the asynchronous ddl to synchronize 219 waitAsyncDDL := func(tctx *tcontext.Context, err error, ddls []string, index int, conn *dbconn.DBConn, createTime int64) error { 220 if len(ddls) == 0 || index > len(ddls)-1 || errors.Cause(err) != mysql.ErrInvalidConn || createTime == -1 { 221 return err // return the original error 222 } 223 224 duration := 30 225 failpoint.Inject("ChangeDuration", func() { 226 duration = 1 227 }) 228 ticker := time.NewTicker(time.Duration(duration) * time.Second) 229 defer ticker.Stop() 230 231 for { 232 status, err2 := getDDLStatusFromTiDB(tctx, conn, ddls[index], createTime) 233 if err2 != nil { 234 s.tctx.L().Warn("error when getting DDL status from TiDB", zap.Error(err2)) 235 } 236 failpoint.Inject("TestStatus", func(val failpoint.Value) { 237 status = val.(string) 238 s.tctx.L().Info("injected test status:", zap.String("TestStatus", status)) 239 }) 240 switch status { 241 case model.JobStateDone.String(), model.JobStateSynced.String(): 242 return nil 243 case model.JobStateCancelled.String(), model.JobStateRollingback.String(), model.JobStateRollbackDone.String(), model.JobStateCancelling.String(): 244 return terror.ErrSyncerCancelledDDL.Generate(ddls[index]) 245 case model.JobStateRunning.String(), model.JobStateQueueing.String(), model.JobStateNone.String(): 246 default: 247 tctx.L().Warn("Unexpected DDL status", zap.String("DDL status", status)) 248 return err 249 } 250 select { 251 case <-tctx.Ctx.Done(): 252 return err 253 case <-ticker.C: 254 } 255 } 256 } 257 258 retErr := err 259 toHandle := []func(*tcontext.Context, error, []string, int, *dbconn.DBConn, int64) error{ 260 dropColumnF, 261 ignoreAddIndexTimeout, 262 waitAsyncDDL, 263 } 264 for _, f := range toHandle { 265 retErr = f(tctx, retErr, ddls, index, conn, createTime) 266 if retErr == nil { 267 break 268 } 269 } 270 return retErr 271 } 272 273 func isDuplicateServerIDError(err error) bool { 274 if err == nil { 275 return false 276 } 277 278 return strings.Contains(err.Error(), "A slave with the same server_uuid/server_id as this slave has connected to the master") 279 } 280 281 func isConnectionRefusedError(err error) bool { 282 if err == nil { 283 return false 284 } 285 286 return strings.Contains(err.Error(), "connect: connection refused") 287 }