github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/online-ddl-tools/online_ddl.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 onlineddl 15 16 import ( 17 "encoding/json" 18 "fmt" 19 "regexp" 20 "sync" 21 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/tidb/pkg/parser" 24 "github.com/pingcap/tidb/pkg/parser/ast" 25 "github.com/pingcap/tidb/pkg/parser/model" 26 "github.com/pingcap/tidb/pkg/util/dbutil" 27 "github.com/pingcap/tidb/pkg/util/filter" 28 "github.com/pingcap/tiflow/dm/config" 29 "github.com/pingcap/tiflow/dm/config/dbconfig" 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 parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" 34 "github.com/pingcap/tiflow/dm/pkg/terror" 35 "github.com/pingcap/tiflow/dm/syncer/dbconn" 36 "github.com/pingcap/tiflow/dm/syncer/metrics" 37 "go.uber.org/zap" 38 ) 39 40 // refactor to reduce duplicate later. 41 var ( 42 maxCheckPointTimeout = "1m" 43 ) 44 45 // OnlinePlugin handles online ddl solutions like pt, gh-ost. 46 type OnlinePlugin interface { 47 // Apply does: 48 // * detect online ddl 49 // * record changes 50 // * apply online ddl on real table 51 // returns sqls, error 52 Apply(tctx *tcontext.Context, tables []*filter.Table, statement string, stmt ast.StmtNode, p *parser.Parser) ([]string, error) 53 // Finish would delete online ddl from memory and storage 54 Finish(tctx *tcontext.Context, table *filter.Table) error 55 // TableType returns ghost/real table 56 TableType(table string) TableType 57 // RealName returns real table name that removed ghost suffix and handled by table router 58 RealName(table string) string 59 // ResetConn reset db connection 60 ResetConn(tctx *tcontext.Context) error 61 // Clear clears all online information 62 // TODO: not used now, check if we could remove it later 63 Clear(tctx *tcontext.Context) error 64 // Close closes online ddl plugin 65 Close() 66 // CheckAndUpdate try to check and fix the schema/table case-sensitive issue 67 CheckAndUpdate(tctx *tcontext.Context, schemas map[string]string, tables map[string]map[string]string) error 68 // CheckRegex checks the regex of shadow/trash table rules and reports an error if a ddl event matches only either of the rules 69 CheckRegex(stmt ast.StmtNode, schema string, flavor conn.LowerCaseTableNamesFlavor) error 70 } 71 72 // TableType is type of table. 73 type TableType string 74 75 // below variables will be explained later. 76 const ( 77 RealTable TableType = "real table" 78 GhostTable TableType = "ghost table" 79 TrashTable TableType = "trash table" // means we should ignore these tables 80 ) 81 82 const ( 83 shadowTable int = iota 84 trashTable 85 allTable 86 ) 87 88 // GhostDDLInfo stores ghost information and ddls. 89 type GhostDDLInfo struct { 90 Schema string `json:"schema"` 91 Table string `json:"table"` 92 93 DDLs []string `json:"ddls"` 94 } 95 96 // Storage stores sharding group online ddls information. 97 type Storage struct { 98 sync.RWMutex 99 100 cfg *config.SubTaskConfig 101 metricProxies *metrics.Proxies 102 103 db *conn.BaseDB 104 dbConn *dbconn.DBConn 105 schema string // schema name, set through task config 106 tableName string // table name with schema, now it's task name 107 id string // the source ID of the upstream MySQL/MariaDB replica. 108 109 // map ghost schema => [ghost table => ghost ddl info, ...] 110 ddls map[string]map[string]*GhostDDLInfo 111 112 logCtx *tcontext.Context 113 } 114 115 // NewOnlineDDLStorage creates a new online ddl storager. 116 func NewOnlineDDLStorage( 117 logCtx *tcontext.Context, 118 cfg *config.SubTaskConfig, 119 metricProxies *metrics.Proxies, 120 ) *Storage { 121 s := &Storage{ 122 cfg: cfg, 123 metricProxies: metricProxies, 124 schema: dbutil.ColumnName(cfg.MetaSchema), 125 tableName: dbutil.TableName(cfg.MetaSchema, cputil.SyncerOnlineDDL(cfg.Name)), 126 id: cfg.SourceID, 127 ddls: make(map[string]map[string]*GhostDDLInfo), 128 logCtx: logCtx, 129 } 130 131 return s 132 } 133 134 // Init initials online handler. 135 func (s *Storage) Init(tctx *tcontext.Context) error { 136 onlineDB := s.cfg.To 137 onlineDB.RawDBCfg = dbconfig.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) 138 db, dbConns, err := dbconn.CreateConns(tctx, s.cfg, conn.DownstreamDBConfig(&onlineDB), 1, s.cfg.IOTotalBytes, s.cfg.UUID) 139 if err != nil { 140 return terror.WithScope(err, terror.ScopeDownstream) 141 } 142 s.db = db 143 s.dbConn = dbConns[0] 144 145 if err = s.prepare(tctx); err != nil { 146 s.Close() 147 return err 148 } 149 150 if err = s.Load(tctx); err != nil { 151 s.Close() 152 return err 153 } 154 155 return nil 156 } 157 158 // Load loads information from storage. 159 func (s *Storage) Load(tctx *tcontext.Context) error { 160 s.Lock() 161 defer s.Unlock() 162 163 query := fmt.Sprintf("SELECT `ghost_schema`, `ghost_table`, `ddls` FROM %s WHERE `id`= ?", s.tableName) 164 rows, err := s.dbConn.QuerySQL(tctx, s.metricProxies, query, s.id) 165 if err != nil { 166 return terror.WithScope(err, terror.ScopeDownstream) 167 } 168 defer rows.Close() 169 170 var ( 171 schema string 172 table string 173 ddls string 174 ) 175 for rows.Next() { 176 err := rows.Scan(&schema, &table, &ddls) 177 if err != nil { 178 return terror.DBErrorAdapt(err, s.dbConn.Scope(), terror.ErrDBDriverError) 179 } 180 181 mSchema, ok := s.ddls[schema] 182 if !ok { 183 mSchema = make(map[string]*GhostDDLInfo) 184 s.ddls[schema] = mSchema 185 } 186 187 mSchema[table] = &GhostDDLInfo{} 188 err = json.Unmarshal([]byte(ddls), mSchema[table]) 189 if err != nil { 190 return terror.ErrSyncerUnitOnlineDDLInvalidMeta.Delegate(err) 191 } 192 tctx.L().Info("loaded online ddl meta from checkpoint", 193 zap.String("db", schema), 194 zap.String("table", table)) 195 } 196 197 return terror.DBErrorAdapt(rows.Err(), s.dbConn.Scope(), terror.ErrDBDriverError) 198 } 199 200 // Get returns ddls by given schema/table. 201 func (s *Storage) Get(ghostSchema, ghostTable string) *GhostDDLInfo { 202 s.RLock() 203 defer s.RUnlock() 204 205 mSchema, ok := s.ddls[ghostSchema] 206 if !ok { 207 return nil 208 } 209 210 if mSchema == nil || mSchema[ghostTable] == nil { 211 return nil 212 } 213 214 clone := new(GhostDDLInfo) 215 *clone = *mSchema[ghostTable] 216 217 return clone 218 } 219 220 // Save saves online ddl information. 221 func (s *Storage) Save(tctx *tcontext.Context, ghostSchema, ghostTable, realSchema, realTable, ddl string) error { 222 s.Lock() 223 defer s.Unlock() 224 225 mSchema, ok := s.ddls[ghostSchema] 226 if !ok { 227 mSchema = make(map[string]*GhostDDLInfo) 228 s.ddls[ghostSchema] = mSchema 229 } 230 231 info, ok := mSchema[ghostTable] 232 if !ok { 233 info = &GhostDDLInfo{ 234 Schema: realSchema, 235 Table: realTable, 236 } 237 mSchema[ghostTable] = info 238 } 239 240 // maybe we meed more checks for it 241 242 if len(info.DDLs) != 0 && info.DDLs[len(info.DDLs)-1] == ddl { 243 tctx.L().Warn("online ddl may be saved before, just ignore it", zap.String("ddl", ddl)) 244 return nil 245 } 246 info.DDLs = append(info.DDLs, ddl) 247 err := s.saveToDB(tctx, ghostSchema, ghostTable, info) 248 return terror.WithScope(err, terror.ScopeDownstream) 249 } 250 251 func (s *Storage) saveToDB(tctx *tcontext.Context, ghostSchema, ghostTable string, ddl *GhostDDLInfo) error { 252 ddlsBytes, err := json.Marshal(ddl) 253 if err != nil { 254 return terror.ErrSyncerUnitOnlineDDLInvalidMeta.Delegate(err) 255 } 256 257 query := fmt.Sprintf("REPLACE INTO %s(`id`,`ghost_schema`, `ghost_table`, `ddls`) VALUES (?, ?, ?, ?)", s.tableName) 258 _, err = s.dbConn.ExecuteSQL(tctx, s.metricProxies, []string{query}, []interface{}{s.id, ghostSchema, ghostTable, string(ddlsBytes)}) 259 failpoint.Inject("ExitAfterSaveOnlineDDL", func() { 260 tctx.L().Info("failpoint ExitAfterSaveOnlineDDL") 261 panic("ExitAfterSaveOnlineDDL") 262 }) 263 return terror.WithScope(err, terror.ScopeDownstream) 264 } 265 266 // Delete deletes online ddl informations. 267 func (s *Storage) Delete(tctx *tcontext.Context, ghostSchema, ghostTable string) error { 268 s.Lock() 269 defer s.Unlock() 270 return s.delete(tctx, ghostSchema, ghostTable) 271 } 272 273 func (s *Storage) delete(tctx *tcontext.Context, ghostSchema, ghostTable string) error { 274 mSchema, ok := s.ddls[ghostSchema] 275 if !ok { 276 return nil 277 } 278 279 // delete all checkpoints 280 sql := fmt.Sprintf("DELETE FROM %s WHERE `id` = ? and `ghost_schema` = ? and `ghost_table` = ?", s.tableName) 281 _, err := s.dbConn.ExecuteSQL(tctx, s.metricProxies, []string{sql}, []interface{}{s.id, ghostSchema, ghostTable}) 282 if err != nil { 283 return terror.WithScope(err, terror.ScopeDownstream) 284 } 285 286 delete(mSchema, ghostTable) 287 return nil 288 } 289 290 // Clear clears online ddl information from storage. 291 func (s *Storage) Clear(tctx *tcontext.Context) error { 292 s.Lock() 293 defer s.Unlock() 294 295 // delete all checkpoints 296 sql := fmt.Sprintf("DELETE FROM %s WHERE `id` = ?", s.tableName) 297 _, err := s.dbConn.ExecuteSQL(tctx, s.metricProxies, []string{sql}, []interface{}{s.id}) 298 if err != nil { 299 return terror.WithScope(err, terror.ScopeDownstream) 300 } 301 302 s.ddls = make(map[string]map[string]*GhostDDLInfo) 303 return nil 304 } 305 306 // ResetConn implements OnlinePlugin.ResetConn. 307 func (s *Storage) ResetConn(tctx *tcontext.Context) error { 308 return s.dbConn.ResetConn(tctx) 309 } 310 311 // Close closes database connection. 312 func (s *Storage) Close() { 313 s.Lock() 314 defer s.Unlock() 315 316 dbconn.CloseBaseDB(s.logCtx, s.db) 317 } 318 319 func (s *Storage) prepare(tctx *tcontext.Context) error { 320 if err := s.createSchema(tctx); err != nil { 321 return err 322 } 323 324 return s.createTable(tctx) 325 } 326 327 func (s *Storage) createSchema(tctx *tcontext.Context) error { 328 sql := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", s.schema) 329 _, err := s.dbConn.ExecuteSQL(tctx, s.metricProxies, []string{sql}) 330 return terror.WithScope(err, terror.ScopeDownstream) 331 } 332 333 func (s *Storage) createTable(tctx *tcontext.Context) error { 334 sql := fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( 335 id VARCHAR(32) NOT NULL, 336 ghost_schema VARCHAR(128) NOT NULL, 337 ghost_table VARCHAR(128) NOT NULL, 338 ddls text, 339 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 340 UNIQUE KEY uk_id_schema_table (id, ghost_schema, ghost_table) 341 )`, s.tableName) 342 _, err := s.dbConn.ExecuteSQL(tctx, s.metricProxies, []string{sql}) 343 return terror.WithScope(err, terror.ScopeDownstream) 344 } 345 346 // CheckAndUpdate try to check and fix the schema/table case-sensitive issue. 347 func (s *Storage) CheckAndUpdate( 348 tctx *tcontext.Context, 349 schemaMap map[string]string, 350 tablesMap map[string]map[string]string, 351 realNameFn func(table string) string, 352 ) error { 353 s.Lock() 354 defer s.Unlock() 355 356 changedSchemas := make([]string, 0) 357 for schema, tblDDLInfos := range s.ddls { 358 realSchema, hasChange := schemaMap[schema] 359 if !hasChange { 360 realSchema = schema 361 } else { 362 changedSchemas = append(changedSchemas, schema) 363 } 364 tblMap := tablesMap[schema] 365 for tbl, ddlInfos := range tblDDLInfos { 366 realTbl, tableChange := tblMap[tbl] 367 if !tableChange { 368 realTbl = tbl 369 tableChange = hasChange 370 } 371 if tableChange { 372 targetTable := realNameFn(realTbl) 373 ddlInfos.Table = targetTable 374 err := s.saveToDB(tctx, realSchema, realTbl, ddlInfos) 375 if err != nil { 376 return err 377 } 378 err = s.delete(tctx, schema, tbl) 379 if err != nil { 380 return err 381 } 382 } 383 } 384 } 385 for _, schema := range changedSchemas { 386 ddl := s.ddls[schema] 387 s.ddls[schemaMap[schema]] = ddl 388 delete(s.ddls, schema) 389 } 390 return nil 391 } 392 393 // RealOnlinePlugin support ghost and pt 394 // Ghost's table format: 395 // _*_gho ghost table 396 // _*_ghc ghost changelog table 397 // _*_del ghost transh table. 398 // PT's table format: 399 // (_*).*_new ghost table 400 // (_*).*_old ghost trash table 401 // we don't support `--new-table-name` flag. 402 type RealOnlinePlugin struct { 403 storage *Storage 404 shadowRegs []*regexp.Regexp 405 trashRegs []*regexp.Regexp 406 } 407 408 // NewRealOnlinePlugin returns real online plugin. 409 func NewRealOnlinePlugin( 410 tctx *tcontext.Context, 411 cfg *config.SubTaskConfig, 412 metricProxies *metrics.Proxies, 413 ) (OnlinePlugin, error) { 414 shadowRegs := make([]*regexp.Regexp, 0, len(cfg.ShadowTableRules)) 415 trashRegs := make([]*regexp.Regexp, 0, len(cfg.TrashTableRules)) 416 for _, sg := range cfg.ShadowTableRules { 417 shadowReg, err := regexp.Compile(sg) 418 if err != nil { 419 return nil, terror.ErrConfigOnlineDDLInvalidRegex.Generate(config.ShadowTableRules, sg, "fail to compile: "+err.Error()) 420 } 421 shadowRegs = append(shadowRegs, shadowReg) 422 } 423 for _, tg := range cfg.TrashTableRules { 424 trashReg, err := regexp.Compile(tg) 425 if err != nil { 426 return nil, terror.ErrConfigOnlineDDLInvalidRegex.Generate(config.TrashTableRules, tg, "fail to compile: "+err.Error()) 427 } 428 trashRegs = append(trashRegs, trashReg) 429 } 430 r := &RealOnlinePlugin{ 431 storage: NewOnlineDDLStorage( 432 tcontext.Background().WithLogger(tctx.L().WithFields(zap.String("online ddl", ""))), 433 cfg, 434 metricProxies, 435 ), // create a context for logger 436 shadowRegs: shadowRegs, 437 trashRegs: trashRegs, 438 } 439 440 return r, r.storage.Init(tctx) 441 } 442 443 // Apply implements interface. 444 // returns ddls, error. 445 func (r *RealOnlinePlugin) Apply(tctx *tcontext.Context, tables []*filter.Table, statement string, stmt ast.StmtNode, p *parser.Parser) ([]string, error) { 446 if len(tables) < 1 { 447 return nil, terror.ErrSyncerUnitGhostApplyEmptyTable.Generate() 448 } 449 450 schema, table := tables[0].Schema, tables[0].Name 451 targetTable := r.RealName(table) 452 tp := r.TableType(table) 453 454 tctx.L().Debug("online ddl", zap.Any("table name", table), zap.Any("table type", tp)) 455 switch tp { 456 case RealTable: 457 if _, ok := stmt.(*ast.RenameTableStmt); ok { 458 if len(tables) != parserpkg.SingleRenameTableNameNum { 459 return nil, terror.ErrSyncerUnitGhostRenameTableNotValid.Generate() 460 } 461 462 tp1 := r.TableType(tables[1].Name) 463 if tp1 == TrashTable { 464 return nil, nil 465 } else if tp1 == GhostTable { 466 return nil, terror.ErrSyncerUnitGhostRenameToGhostTable.Generate(statement) 467 } 468 } 469 return []string{statement}, nil 470 case TrashTable: 471 // ignore TrashTable 472 if _, ok := stmt.(*ast.RenameTableStmt); ok { 473 if len(tables) != parserpkg.SingleRenameTableNameNum { 474 return nil, terror.ErrSyncerUnitGhostRenameTableNotValid.Generate() 475 } 476 477 tp1 := r.TableType(tables[1].Name) 478 if tp1 == GhostTable { 479 return nil, terror.ErrSyncerUnitGhostRenameGhostTblToOther.Generate(statement) 480 } 481 } 482 case GhostTable: 483 // record ghost table ddl changes 484 switch stmt.(type) { 485 case *ast.CreateTableStmt: 486 err := r.storage.Delete(tctx, schema, table) 487 if err != nil { 488 return nil, err 489 } 490 case *ast.DropTableStmt: 491 err := r.storage.Delete(tctx, schema, table) 492 if err != nil { 493 return nil, err 494 } 495 case *ast.RenameTableStmt: 496 if len(tables) != parserpkg.SingleRenameTableNameNum { 497 return nil, terror.ErrSyncerUnitGhostRenameTableNotValid.Generate() 498 } 499 500 tp1 := r.TableType(tables[1].Name) 501 if tp1 == RealTable { 502 ghostInfo := r.storage.Get(schema, table) 503 if ghostInfo != nil { 504 return renameOnlineDDLTable(p, tables[1], ghostInfo.DDLs) 505 } 506 return nil, terror.ErrSyncerUnitGhostOnlineDDLOnGhostTbl.Generate(schema, table) 507 } else if tp1 == GhostTable { 508 return nil, terror.ErrSyncerUnitGhostRenameGhostTblToOther.Generate(statement) 509 } 510 511 // rename ghost table to trash table 512 err := r.storage.Delete(tctx, schema, table) 513 if err != nil { 514 return nil, err 515 } 516 517 default: 518 err := r.storage.Save(tctx, schema, table, schema, targetTable, statement) 519 if err != nil { 520 return nil, err 521 } 522 } 523 } 524 return nil, nil 525 } 526 527 // Finish implements interface. 528 func (r *RealOnlinePlugin) Finish(tctx *tcontext.Context, table *filter.Table) error { 529 if r == nil { 530 return nil 531 } 532 533 return r.storage.Delete(tctx, table.Schema, table.Name) 534 } 535 536 // TableType implements interface. 537 func (r *RealOnlinePlugin) TableType(table string) TableType { 538 // 5 is _ _gho/ghc/del or _ _old/new 539 for _, shadowReg := range r.shadowRegs { 540 if shadowReg.MatchString(table) { 541 return GhostTable 542 } 543 } 544 545 for _, trashReg := range r.trashRegs { 546 if trashReg.MatchString(table) { 547 return TrashTable 548 } 549 } 550 return RealTable 551 } 552 553 // RealName implements interface. 554 func (r *RealOnlinePlugin) RealName(table string) string { 555 for _, shadowReg := range r.shadowRegs { 556 shadowRes := shadowReg.FindStringSubmatch(table) 557 if len(shadowRes) > 1 { 558 return shadowRes[1] 559 } 560 } 561 562 for _, trashReg := range r.trashRegs { 563 trashRes := trashReg.FindStringSubmatch(table) 564 if len(trashRes) > 1 { 565 return trashRes[1] 566 } 567 } 568 return table 569 } 570 571 // Clear clears online ddl information. 572 func (r *RealOnlinePlugin) Clear(tctx *tcontext.Context) error { 573 return r.storage.Clear(tctx) 574 } 575 576 // Close implements interface. 577 func (r *RealOnlinePlugin) Close() { 578 r.storage.Close() 579 } 580 581 // ResetConn implements interface. 582 func (r *RealOnlinePlugin) ResetConn(tctx *tcontext.Context) error { 583 return r.storage.ResetConn(tctx) 584 } 585 586 // CheckAndUpdate try to check and fix the schema/table case-sensitive issue. 587 func (r *RealOnlinePlugin) CheckAndUpdate(tctx *tcontext.Context, schemas map[string]string, tables map[string]map[string]string) error { 588 return r.storage.CheckAndUpdate(tctx, schemas, tables, r.RealName) 589 } 590 591 // CheckRegex checks the regex of shadow/trash table rules and reports an error if a ddl event matches only either of the rules. 592 func (r *RealOnlinePlugin) CheckRegex(stmt ast.StmtNode, schema string, flavor conn.LowerCaseTableNamesFlavor) error { 593 var ( 594 v *ast.RenameTableStmt 595 ok bool 596 ) 597 if v, ok = stmt.(*ast.RenameTableStmt); !ok { 598 return nil 599 } 600 t2ts := v.TableToTables 601 if len(t2ts) != 2 { 602 return nil 603 } 604 onlineDDLMatched := allTable 605 tableRecords := make([]*filter.Table, 2) 606 schemaName := model.NewCIStr(schema) // fill schema name 607 608 // Online DDL sql example: RENAME TABLE `test`.`t1` TO `test`.`_t1_old`, `test`.`_t1_new` TO `test`.`t1` 609 // We should parse two rename DDL from this DDL: 610 // tables[0] tables[1] 611 // DDL 0 real table ───► trash table 612 // DDL 1 shadow table ───► real table 613 // If we only have one of them, that means users may configure a wrong trash/shadow table regex 614 for i, t2t := range t2ts { 615 if t2t.OldTable.Schema.O == "" { 616 t2t.OldTable.Schema = schemaName 617 } 618 if t2t.NewTable.Schema.O == "" { 619 t2t.NewTable.Schema = schemaName 620 } 621 622 v.TableToTables = []*ast.TableToTable{t2t} 623 624 if i == 0 { 625 tableRecords[trashTable] = fetchTable(t2t.NewTable, flavor) 626 if r.TableType(t2t.OldTable.Name.String()) == RealTable && 627 r.TableType(t2t.NewTable.Name.String()) == TrashTable { 628 onlineDDLMatched = trashTable 629 } 630 } else { 631 tableRecords[shadowTable] = fetchTable(t2t.OldTable, flavor) 632 if r.TableType(t2t.OldTable.Name.String()) == GhostTable && 633 r.TableType(t2t.NewTable.Name.String()) == RealTable { 634 // if no trash table is not matched before, we should record that shadow table is matched here 635 // if shadow table is matched before, we just return all tables are matched and a nil error 636 if onlineDDLMatched != trashTable { 637 onlineDDLMatched = shadowTable 638 } else { 639 onlineDDLMatched = allTable 640 } 641 } 642 } 643 } 644 if onlineDDLMatched != allTable { 645 return terror.ErrConfigOnlineDDLMistakeRegex.Generate(stmt.Text(), tableRecords[onlineDDLMatched^1], unmatchedOnlineDDLRules(onlineDDLMatched)) 646 } 647 return nil 648 } 649 650 func unmatchedOnlineDDLRules(match int) string { 651 switch match { 652 case shadowTable: 653 return config.TrashTableRules 654 case trashTable: 655 return config.ShadowTableRules 656 default: 657 return "" 658 } 659 } 660 661 func fetchTable(t *ast.TableName, flavor conn.LowerCaseTableNamesFlavor) *filter.Table { 662 var tb *filter.Table 663 if flavor == conn.LCTableNamesSensitive { 664 tb = &filter.Table{Schema: t.Schema.O, Name: t.Name.O} 665 } else { 666 tb = &filter.Table{Schema: t.Schema.L, Name: t.Name.L} 667 } 668 return tb 669 } 670 671 // renameOnlineDDLTable renames the given ddl sqls by given targetTable. 672 func renameOnlineDDLTable(p *parser.Parser, targetTable *filter.Table, sqls []string) ([]string, error) { 673 renamedSQLs := make([]string, 0, len(sqls)) 674 targetTables := []*filter.Table{targetTable} 675 for _, sql := range sqls { 676 if len(sql) == 0 { 677 continue 678 } 679 stmt, err := p.ParseOneStmt(sql, "", "") 680 if err != nil { 681 return nil, terror.ErrSyncerUnitParseStmt.New(err.Error()) 682 } 683 684 sql, err = parserpkg.RenameDDLTable(stmt, targetTables) 685 if err != nil { 686 return nil, err 687 } 688 renamedSQLs = append(renamedSQLs, sql) 689 } 690 return renamedSQLs, nil 691 }