github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/sharding_group.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 /* 17 * sharding DDL sync for Syncer (pessimism) 18 * 19 * assumption: 20 * all tables in a sharding group execute same DDLs in the same order 21 * when the first staring of the task, same DDL for the whole sharding group should not in partial executed 22 * the startup point can not be in the middle of the first-pos and last-pos of a sharding group's DDL 23 * do not support to modify router-rules online (when unresolved) 24 * do not support to rename table or database in a sharding group, another solution for it 25 * do not support to sync using GTID mode (GTID is supported by relay unit now) 26 * do not support same <schema-name, table-name> pair for upstream and downstream when merging sharding group 27 * ignore all drop schema/table and truncate table ddls 28 * 29 * checkpoint mechanism (ref: checkpoint.go): 30 * save checkpoint for every upstream table, and also global checkpoint 31 * global checkpoint can be used to re-sync after restarted 32 * per-table's checkpoint can be used to check whether the binlog has synced before 33 * 34 * normal work flow 35 * 1. use the global streamer to sync regular binlog events as before 36 * update per-table's and global checkpoint 37 * 2. the first sharding DDL encountered for a table in a sharding group 38 * save this DDL's binlog pos as first pos 39 * save this table's name 40 * 3. continue the syncing with global streamer 41 * ignore binlog events for the table in step.2 42 * stop the updating for global checkpoint and this table's checkpoint 43 * 4. more sharding DDLs encountered for tables in some sharding groups 44 * save these tables' name 45 * 5. continue the syncing with global streamer 46 * ignore binlog events for tables which encountered sharding DDLs (step.2 and step.4) 47 * 6. the last sharding DDL encountered for table in a sharding group 48 * save this DDL's next binlog pos as last pos for the sharding group 49 * execute this DDL 50 * reset the sharding group 51 * 7. redirect global streamer to the first DDL's binlog pos in step.2 52 * 8. continue the syncing with the global streamer 53 * ignore binlog events which not belong to the sharding group 54 * ignore binlog events have synced (obsolete) for the sharding group 55 * synced ignored binlog events in the sharding group from step.3 to step.5 56 * update per-table's and global checkpoint 57 * 9. last pos in step.6 arrived 58 * 10. redirect global streamer to the active DDL in sequence sharding if needed 59 * 11. use the global streamer to continue the syncing 60 * 61 * all binlogs executed at least once: 62 * no sharding group and no restart: syncing as previous 63 * sharding group: binlog events ignored between first-pos and last-pos will be re-sync using a special streamer 64 * restart: global checkpoint never upper than any ignored binlogs 65 * 66 * all binlogs executed at most once: 67 * NO guarantee for this even though every table records checkpoint dependently 68 * because execution of binlog and update of checkpoint are in different goroutines concurrently 69 * so when re-starting the process or recovering from errors, safe-mode must be enabled 70 * 71 */ 72 73 import ( 74 "fmt" 75 "sync" 76 77 "github.com/pingcap/tidb/pkg/util/dbutil" 78 "github.com/pingcap/tidb/pkg/util/filter" 79 "github.com/pingcap/tiflow/dm/config" 80 "github.com/pingcap/tiflow/dm/config/dbconfig" 81 "github.com/pingcap/tiflow/dm/pb" 82 "github.com/pingcap/tiflow/dm/pkg/binlog" 83 "github.com/pingcap/tiflow/dm/pkg/conn" 84 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 85 "github.com/pingcap/tiflow/dm/pkg/cputil" 86 fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" 87 "github.com/pingcap/tiflow/dm/pkg/terror" 88 "github.com/pingcap/tiflow/dm/pkg/utils" 89 "github.com/pingcap/tiflow/dm/syncer/dbconn" 90 "github.com/pingcap/tiflow/dm/syncer/metrics" 91 shardmeta "github.com/pingcap/tiflow/dm/syncer/sharding-meta" 92 "go.uber.org/zap" 93 ) 94 95 // ShardingGroup represents a sharding DDL sync group. 96 type ShardingGroup struct { 97 sync.RWMutex 98 // remain count waiting for syncing 99 // == len(sources): DDL syncing not started or resolved 100 // == 0: all DDLs synced, will be reset to len(sources) after resolved combining with other dm-workers 101 // (0, len(sources)): waiting for syncing 102 // NOTE: we can make remain to be configurable if needed 103 remain int 104 sources map[string]bool // source table ID -> whether source table's DDL synced 105 IsSchemaOnly bool // whether is a schema (database) only DDL TODO: zxc add schema-level syncing support later 106 107 sourceID string // associate dm-worker source ID 108 meta *shardmeta.ShardingMeta // sharding sequence meta storage 109 110 firstLocation *binlog.Location // first DDL's binlog pos and gtid, used to restrain the global checkpoint when un-resolved 111 firstEndLocation *binlog.Location // first DDL's binlog End_log_pos and gtid, used to re-direct binlog streamer after synced 112 ddls []string // DDL which current in syncing 113 114 flavor string 115 enableGTID bool 116 } 117 118 // NewShardingGroup creates a new ShardingGroup. 119 func NewShardingGroup(sourceID, shardMetaSchema, shardMetaTable string, sources []string, meta *shardmeta.ShardingMeta, isSchemaOnly bool, flavor string, enableGTID bool) *ShardingGroup { 120 sg := &ShardingGroup{ 121 remain: len(sources), 122 sources: make(map[string]bool, len(sources)), 123 IsSchemaOnly: isSchemaOnly, 124 sourceID: sourceID, 125 firstLocation: nil, 126 firstEndLocation: nil, 127 flavor: flavor, 128 enableGTID: enableGTID, 129 } 130 if meta != nil { 131 sg.meta = meta 132 } else { 133 sg.meta = shardmeta.NewShardingMeta(shardMetaSchema, shardMetaTable, enableGTID) 134 } 135 for _, source := range sources { 136 sg.sources[source] = false 137 } 138 return sg 139 } 140 141 // Merge merges new sources to exists 142 // used cases 143 // - add a new table to exists sharding group 144 // - add new table(s) to parent database's sharding group 145 // if group is in sequence sharding, return error directly 146 // otherwise add it in source, set it false and increment remain 147 func (sg *ShardingGroup) Merge(sources []string) (bool, bool, int, error) { 148 sg.Lock() 149 defer sg.Unlock() 150 151 // NOTE: we don't support add shard table when in sequence sharding 152 if sg.meta.InSequenceSharding() { 153 return true, sg.remain <= 0, sg.remain, terror.ErrSyncUnitAddTableInSharding.Generate(sg.meta.GetGlobalActiveDDL(), sg.meta.GetGlobalItems()) 154 } 155 156 for _, source := range sources { 157 _, exist := sg.sources[source] 158 if !exist { 159 sg.remain++ 160 sg.sources[source] = false 161 } 162 } 163 164 return false, sg.remain <= 0, sg.remain, nil 165 } 166 167 // Leave leaves from sharding group 168 // it, doesn't affect in syncing process 169 // used cases 170 // - drop a database 171 // - drop table 172 func (sg *ShardingGroup) Leave(sources []string) error { 173 sg.Lock() 174 defer sg.Unlock() 175 176 // NOTE: if group is in sequence sharding, we can't do drop (DROP DATABASE / TABLE) 177 if sg.meta.InSequenceSharding() { 178 return terror.ErrSyncUnitDropSchemaTableInSharding.Generate(sources, sg.meta.GetGlobalActiveDDL(), sg.meta.GetGlobalItems()) 179 } 180 181 for _, source := range sources { 182 if synced, ok := sg.sources[source]; ok && !synced { 183 sg.remain-- 184 } 185 delete(sg.sources, source) 186 } 187 188 return nil 189 } 190 191 // Reset resets all sources to un-synced state 192 // when the previous sharding DDL synced and resolved, we need reset it. 193 func (sg *ShardingGroup) Reset() { 194 sg.Lock() 195 defer sg.Unlock() 196 197 sg.remain = len(sg.sources) 198 for source := range sg.sources { 199 sg.sources[source] = false 200 } 201 sg.firstLocation = nil 202 sg.firstEndLocation = nil 203 sg.ddls = nil 204 } 205 206 // TrySync tries to sync the sharding group 207 // returns 208 // 209 // synced: whether the source table's sharding group synced 210 // active: whether the DDL will be processed in this round 211 // remain: remain un-synced source table's count 212 func (sg *ShardingGroup) TrySync(source string, location, endLocation binlog.Location, ddls []string) (bool, bool, int, error) { 213 sg.Lock() 214 defer sg.Unlock() 215 216 ddlItem := shardmeta.NewDDLItem(location, ddls, source) 217 active, err := sg.meta.AddItem(ddlItem) 218 if err != nil { 219 return sg.remain <= 0, active, sg.remain, err 220 } 221 if active && !sg.sources[source] { 222 sg.sources[source] = true 223 sg.remain-- 224 } 225 226 if sg.firstLocation == nil { 227 sg.firstLocation = &location // save first DDL's pos 228 sg.firstEndLocation = &endLocation 229 sg.ddls = ddls 230 } 231 return sg.remain <= 0, active, sg.remain, nil 232 } 233 234 // CheckSyncing checks the source table syncing status 235 // returns 236 // 237 // beforeActiveDDL: whether the position is before active DDL 238 func (sg *ShardingGroup) CheckSyncing(source string, location binlog.Location) (beforeActiveDDL bool) { 239 sg.RLock() 240 defer sg.RUnlock() 241 activeDDLItem := sg.meta.GetActiveDDLItem(source) 242 if activeDDLItem == nil { 243 return true 244 } 245 // this function only affects dml 246 // activeDDLItem.FirstLocation is ddl's startLocation 247 // location is dml's endLocation 248 // dml should be synced when the comparation is equal 249 return binlog.CompareLocation(activeDDLItem.FirstLocation, location, sg.enableGTID) >= 0 250 } 251 252 // UnresolvedGroupInfo returns pb.ShardingGroup if is unresolved, else returns nil. 253 func (sg *ShardingGroup) UnresolvedGroupInfo() *pb.ShardingGroup { 254 sg.RLock() 255 defer sg.RUnlock() 256 257 if sg.remain == len(sg.sources) { 258 return nil 259 } 260 261 group := &pb.ShardingGroup{ 262 DDLs: sg.ddls, 263 FirstLocation: sg.firstLocation.String(), 264 Synced: make([]string, 0, len(sg.sources)-sg.remain), 265 Unsynced: make([]string, 0, sg.remain), 266 } 267 for source, synced := range sg.sources { 268 if synced { 269 group.Synced = append(group.Synced, source) 270 } else { 271 group.Unsynced = append(group.Unsynced, source) 272 } 273 } 274 return group 275 } 276 277 // Sources returns all sources (and whether synced). 278 func (sg *ShardingGroup) Sources() map[string]bool { 279 sg.RLock() 280 defer sg.RUnlock() 281 ret := make(map[string]bool, len(sg.sources)) 282 for k, v := range sg.sources { 283 ret[k] = v 284 } 285 return ret 286 } 287 288 // Tables returns all source tables' <schema, table> pair. 289 func (sg *ShardingGroup) Tables() []*filter.Table { 290 sources := sg.Sources() 291 tables := make([]*filter.Table, 0, len(sources)) 292 for id := range sources { 293 tables = append(tables, utils.UnpackTableID(id)) 294 } 295 return tables 296 } 297 298 // UnresolvedTables returns all source tables' <schema, table> pair if is unresolved, else returns nil. 299 func (sg *ShardingGroup) UnresolvedTables() []*filter.Table { 300 sg.RLock() 301 defer sg.RUnlock() 302 303 // TODO: if we have sharding ddl sequence, and partial ddls synced, we treat 304 // all the of the tables as unresolved 305 if sg.remain == len(sg.sources) { 306 return nil 307 } 308 309 tables := make([]*filter.Table, 0, len(sg.sources)) 310 for id := range sg.sources { 311 tables = append(tables, utils.UnpackTableID(id)) 312 } 313 return tables 314 } 315 316 // FirstLocationUnresolved returns the first DDL pos if un-resolved, else nil. 317 func (sg *ShardingGroup) FirstLocationUnresolved() *binlog.Location { 318 sg.RLock() 319 defer sg.RUnlock() 320 if sg.remain < len(sg.sources) && sg.firstLocation != nil { 321 // create a new location to return 322 location := sg.firstLocation.CloneWithFlavor(sg.flavor) 323 return &location 324 } 325 item := sg.meta.GetGlobalActiveDDL() 326 if item != nil { 327 // make a new copy 328 location := item.FirstLocation.CloneWithFlavor(sg.flavor) 329 return &location 330 } 331 return nil 332 } 333 334 // FirstEndPosUnresolved returns the first DDL End_log_pos if un-resolved, else nil. 335 func (sg *ShardingGroup) FirstEndPosUnresolved() *binlog.Location { 336 sg.RLock() 337 defer sg.RUnlock() 338 if sg.remain < len(sg.sources) && sg.firstEndLocation != nil { 339 // create a new location to return 340 location := sg.firstEndLocation.CloneWithFlavor(sg.flavor) 341 return &location 342 } 343 return nil 344 } 345 346 // String implements Stringer.String. 347 func (sg *ShardingGroup) String() string { 348 return fmt.Sprintf("IsSchemaOnly:%v remain:%d, sources:%+v", sg.IsSchemaOnly, sg.remain, sg.sources) 349 } 350 351 // ResolveShardingDDL resolves sharding DDL in sharding group. 352 func (sg *ShardingGroup) ResolveShardingDDL() bool { 353 sg.Lock() 354 defer sg.Unlock() 355 reset := sg.meta.ResolveShardingDDL() 356 // reset sharding group after DDL is executed 357 return reset 358 } 359 360 // ActiveDDLFirstLocation returns the first binlog position of active DDL. 361 func (sg *ShardingGroup) ActiveDDLFirstLocation() (binlog.Location, error) { 362 sg.RLock() 363 defer sg.RUnlock() 364 location, err := sg.meta.ActiveDDLFirstLocation() 365 return location, err 366 } 367 368 // FlushData returns sharding meta flush SQLs and args. 369 func (sg *ShardingGroup) FlushData(targetTableID string) ([]string, [][]interface{}) { 370 sg.RLock() 371 defer sg.RUnlock() 372 return sg.meta.FlushData(sg.sourceID, targetTableID) 373 } 374 375 // ShardingGroupKeeper used to keep ShardingGroup. 376 type ShardingGroupKeeper struct { 377 sync.RWMutex 378 groups map[string]*ShardingGroup // target table ID -> ShardingGroup 379 cfg *config.SubTaskConfig 380 metricProxies *metrics.Proxies 381 382 shardMetaSchema string 383 shardMetaTable string 384 shardMetaTableName string 385 386 db *conn.BaseDB 387 dbConn *dbconn.DBConn 388 389 tctx *tcontext.Context 390 } 391 392 // NewShardingGroupKeeper creates a new ShardingGroupKeeper. 393 func NewShardingGroupKeeper( 394 tctx *tcontext.Context, 395 cfg *config.SubTaskConfig, 396 metricProxies *metrics.Proxies, 397 ) *ShardingGroupKeeper { 398 k := &ShardingGroupKeeper{ 399 groups: make(map[string]*ShardingGroup), 400 cfg: cfg, 401 metricProxies: metricProxies, 402 tctx: tctx.WithLogger(tctx.L().WithFields(zap.String("component", "shard group keeper"))), 403 } 404 k.shardMetaSchema = cfg.MetaSchema 405 k.shardMetaTable = cputil.SyncerShardMeta(cfg.Name) 406 k.shardMetaTableName = dbutil.TableName(k.shardMetaSchema, k.shardMetaTable) 407 return k 408 } 409 410 // AddGroup adds new group(s) according to target schema, table and source IDs. 411 func (k *ShardingGroupKeeper) AddGroup(targetTable *filter.Table, sourceIDs []string, meta *shardmeta.ShardingMeta, merge bool) (needShardingHandle bool, group *ShardingGroup, synced bool, remain int, err error) { 412 // if need to support target table-level sharding DDL 413 // we also need to support target schema-level sharding DDL 414 targetSchemaID := utils.GenSchemaID(targetTable) 415 targetTableID := utils.GenTableID(targetTable) 416 417 k.Lock() 418 defer k.Unlock() 419 420 if schemaGroup, ok := k.groups[targetSchemaID]; !ok { 421 k.groups[targetSchemaID] = NewShardingGroup(k.cfg.SourceID, k.shardMetaSchema, k.shardMetaTable, sourceIDs, meta, true, k.cfg.Flavor, k.cfg.EnableGTID) 422 } else { 423 _, _, _, err = schemaGroup.Merge(sourceIDs) 424 if err != nil { 425 return 426 } 427 } 428 429 group, ok := k.groups[targetTableID] 430 switch { 431 case !ok: 432 group = NewShardingGroup(k.cfg.SourceID, k.shardMetaSchema, k.shardMetaTable, sourceIDs, meta, false, k.cfg.Flavor, k.cfg.EnableGTID) 433 k.groups[targetTableID] = group 434 case merge: 435 needShardingHandle, synced, remain, err = k.groups[targetTableID].Merge(sourceIDs) 436 default: 437 err = terror.ErrSyncUnitDupTableGroup.Generate(targetTableID) 438 } 439 440 return 441 } 442 443 // Init does initialization staff. 444 func (k *ShardingGroupKeeper) Init() (err error) { 445 var db *conn.BaseDB 446 var dbConns []*dbconn.DBConn 447 448 rollbackHolder := fr.NewRollbackHolder("syncer") 449 defer func() { 450 if err != nil { 451 rollbackHolder.RollbackReverseOrder() 452 } 453 }() 454 455 k.clear() 456 sgkDB := k.cfg.To 457 sgkDB.RawDBCfg = dbconfig.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) 458 db, dbConns, err = dbconn.CreateConns(k.tctx, k.cfg, conn.DownstreamDBConfig(&sgkDB), 1, k.cfg.IOTotalBytes, k.cfg.UUID) 459 if err != nil { 460 return 461 } 462 k.db = db 463 k.dbConn = dbConns[0] 464 rollbackHolder.Add(fr.FuncRollback{Name: "CloseShardingGroupKeeper", Fn: k.Close}) 465 466 err = k.prepare() 467 468 return 469 } 470 471 // clear clears all sharding groups. 472 func (k *ShardingGroupKeeper) clear() { 473 k.Lock() 474 defer k.Unlock() 475 k.groups = make(map[string]*ShardingGroup) 476 } 477 478 // ResetGroups resets group's sync status. 479 func (k *ShardingGroupKeeper) ResetGroups() { 480 k.RLock() 481 defer k.RUnlock() 482 for _, group := range k.groups { 483 group.Reset() 484 // reset ShardingMeta when start or resume task 485 // it will be reconstructed by consuming binlog event 486 group.meta.Reinitialize() 487 } 488 } 489 490 // LeaveGroup leaves group according to target schema, table and source IDs 491 // LeaveGroup doesn't affect in syncing process. 492 func (k *ShardingGroupKeeper) LeaveGroup(targetTable *filter.Table, sources []string) error { 493 targetSchemaID := utils.GenSchemaID(targetTable) 494 targetTableID := utils.GenTableID(targetTable) 495 k.Lock() 496 defer k.Unlock() 497 if group, ok := k.groups[targetTableID]; ok { 498 if err := group.Leave(sources); err != nil { 499 return err 500 } 501 if len(group.sources) == 0 { 502 delete(k.groups, targetTableID) 503 } 504 } 505 if schemaGroup, ok := k.groups[targetSchemaID]; ok { 506 if err := schemaGroup.Leave(sources); err != nil { 507 return err 508 } 509 if len(schemaGroup.sources) == 0 { 510 delete(k.groups, targetSchemaID) 511 } 512 } 513 return nil 514 } 515 516 // TrySync tries to sync the sharding group 517 // returns 518 // 519 // isSharding: whether the source table is in a sharding group 520 // group: the sharding group 521 // synced: whether the source table's sharding group synced 522 // active: whether is active DDL in sequence sharding DDL 523 // remain: remain un-synced source table's count 524 func (k *ShardingGroupKeeper) TrySync( 525 sourceTable, targetTable *filter.Table, location, endLocation binlog.Location, ddls []string) ( 526 needShardingHandle bool, group *ShardingGroup, synced, active bool, remain int, err error, 527 ) { 528 targetTableID, schemaOnly := utils.GenTableIDAndCheckSchemaOnly(targetTable) 529 sourceTableID := utils.GenTableID(sourceTable) 530 if schemaOnly { 531 // NOTE: now we don't support syncing for schema only sharding DDL 532 return false, nil, true, false, 0, nil 533 } 534 535 k.Lock() 536 defer k.Unlock() 537 538 group, ok := k.groups[targetTableID] 539 if !ok { 540 return false, group, true, false, 0, nil 541 } 542 synced, active, remain, err = group.TrySync(sourceTableID, location, endLocation, ddls) 543 return true, group, synced, active, remain, err 544 } 545 546 // InSyncing checks whether the source is in sharding syncing, that is to say not before active DDL. 547 func (k *ShardingGroupKeeper) InSyncing(sourceTable, targetTable *filter.Table, location binlog.Location) bool { 548 group := k.Group(targetTable) 549 if group == nil { 550 return false 551 } 552 return !group.CheckSyncing(utils.GenTableID(sourceTable), location) 553 } 554 555 // UnresolvedTables returns 556 // 557 // all `target-schema.target-table` that has unresolved sharding DDL, 558 // all source tables which with DDLs are un-resolved 559 // 560 // NOTE: this func only ensure the returned tables are current un-resolved 561 // if passing the returned tables to other func (like checkpoint), 562 // must ensure their sync state not changed in this progress. 563 func (k *ShardingGroupKeeper) UnresolvedTables() (map[string]bool, []*filter.Table) { 564 ids := make(map[string]bool) 565 tables := make([]*filter.Table, 0, 10) 566 k.RLock() 567 defer k.RUnlock() 568 for id, group := range k.groups { 569 unresolved := group.UnresolvedTables() 570 if len(unresolved) > 0 { 571 // TODO: no need to return bool which indicates it has unresolved tables, because nowhere need it 572 ids[id] = true 573 tables = append(tables, unresolved...) 574 } 575 } 576 return ids, tables 577 } 578 579 // Group returns target table's group, nil if not exist. 580 func (k *ShardingGroupKeeper) Group(targetTable *filter.Table) *ShardingGroup { 581 k.RLock() 582 defer k.RUnlock() 583 return k.groups[utils.GenTableID(targetTable)] 584 } 585 586 // lowestFirstLocationInGroups returns the lowest pos in all groups which are unresolved. 587 func (k *ShardingGroupKeeper) lowestFirstLocationInGroups() *binlog.Location { 588 k.RLock() 589 defer k.RUnlock() 590 var lowest *binlog.Location 591 for _, group := range k.groups { 592 location := group.FirstLocationUnresolved() 593 if location == nil { 594 continue 595 } 596 if lowest == nil { 597 lowest = location 598 } else if binlog.CompareLocation(*lowest, *location, k.cfg.EnableGTID) > 0 { 599 lowest = location 600 } 601 } 602 return lowest 603 } 604 605 // AdjustGlobalLocation adjusts globalLocation with sharding groups' lowest first point. 606 func (k *ShardingGroupKeeper) AdjustGlobalLocation(globalLocation binlog.Location) binlog.Location { 607 lowestFirstLocation := k.lowestFirstLocationInGroups() 608 if lowestFirstLocation != nil && binlog.CompareLocation(*lowestFirstLocation, globalLocation, k.cfg.EnableGTID) < 0 { 609 return *lowestFirstLocation 610 } 611 return globalLocation 612 } 613 614 // Groups returns all sharding groups, often used for debug 615 // caution: do not modify the returned groups directly 616 func (k *ShardingGroupKeeper) Groups() map[string]*ShardingGroup { 617 k.RLock() 618 defer k.RUnlock() 619 620 // do a copy 621 groups := make(map[string]*ShardingGroup, len(k.groups)) 622 for key, value := range k.groups { 623 groups[key] = value 624 } 625 return groups 626 } 627 628 // UnresolvedGroups returns sharding groups which are un-resolved 629 // caution: do not modify the returned groups directly 630 func (k *ShardingGroupKeeper) UnresolvedGroups() []*pb.ShardingGroup { 631 groups := make([]*pb.ShardingGroup, 0) 632 k.RLock() 633 defer k.RUnlock() 634 for target, group := range k.groups { 635 gi := group.UnresolvedGroupInfo() 636 if gi != nil { 637 gi.Target = target // set target 638 groups = append(groups, gi) 639 } 640 } 641 return groups 642 } 643 644 // ResolveShardingDDL resolves one sharding DDL in specific group. 645 func (k *ShardingGroupKeeper) ResolveShardingDDL(targetTable *filter.Table) (bool, error) { 646 if group := k.Group(targetTable); group != nil { 647 return group.ResolveShardingDDL(), nil 648 } 649 return false, terror.ErrSyncUnitShardingGroupNotFound.Generate(targetTable) 650 } 651 652 // ActiveDDLFirstLocation returns the binlog position of active DDL. 653 func (k *ShardingGroupKeeper) ActiveDDLFirstLocation(targetTable *filter.Table) (binlog.Location, error) { 654 group := k.Group(targetTable) 655 k.Lock() 656 defer k.Unlock() 657 if group != nil { 658 location, err := group.ActiveDDLFirstLocation() 659 return location, err 660 } 661 return binlog.Location{}, terror.ErrSyncUnitShardingGroupNotFound.Generate(targetTable) 662 } 663 664 // PrepareFlushSQLs returns all sharding meta flushed SQLs except for given table IDs. 665 func (k *ShardingGroupKeeper) PrepareFlushSQLs(exceptTableIDs map[string]bool) ([]string, [][]interface{}) { 666 k.RLock() 667 defer k.RUnlock() 668 var ( 669 sqls = make([]string, 0, len(k.groups)) 670 args = make([][]interface{}, 0, len(k.groups)) 671 ) 672 for id, group := range k.groups { 673 if group.IsSchemaOnly { 674 continue 675 } 676 _, ok := exceptTableIDs[id] 677 if ok { 678 continue 679 } 680 sqls2, args2 := group.FlushData(id) 681 sqls = append(sqls, sqls2...) 682 args = append(args, args2...) 683 } 684 return sqls, args 685 } 686 687 // Prepare inits sharding meta schema and tables if not exists. 688 func (k *ShardingGroupKeeper) prepare() error { 689 if err := k.createSchema(); err != nil { 690 return err 691 } 692 693 return k.createTable() 694 } 695 696 // Close closes sharding group keeper. 697 func (k *ShardingGroupKeeper) Close() { 698 dbconn.CloseBaseDB(k.tctx, k.db) 699 } 700 701 func (k *ShardingGroupKeeper) createSchema() error { 702 stmt := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS `%s`", k.shardMetaSchema) 703 _, err := k.dbConn.ExecuteSQL(k.tctx, k.metricProxies, []string{stmt}) 704 k.tctx.L().Info("execute sql", zap.String("statement", stmt)) 705 return terror.WithScope(err, terror.ScopeDownstream) 706 } 707 708 func (k *ShardingGroupKeeper) createTable() error { 709 stmt := fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( 710 source_id VARCHAR(32) NOT NULL COMMENT 'replica source id, defined in task.yaml', 711 target_table_id VARCHAR(144) NOT NULL, 712 source_table_id VARCHAR(144) NOT NULL, 713 active_index INT, 714 is_global BOOLEAN, 715 data JSON, 716 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 717 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 718 UNIQUE KEY uk_source_id_table_id_source (source_id, target_table_id, source_table_id) 719 )`, k.shardMetaTableName) 720 _, err := k.dbConn.ExecuteSQL(k.tctx, k.metricProxies, []string{stmt}) 721 k.tctx.L().Info("execute sql", zap.String("statement", stmt)) 722 return terror.WithScope(err, terror.ScopeDownstream) 723 } 724 725 // LoadShardMeta implements CheckPoint.LoadShardMeta. 726 func (k *ShardingGroupKeeper) LoadShardMeta(flavor string, enableGTID bool) (map[string]*shardmeta.ShardingMeta, error) { 727 query := fmt.Sprintf("SELECT `target_table_id`, `source_table_id`, `active_index`, `is_global`, `data` FROM %s WHERE `source_id`='%s'", k.shardMetaTableName, k.cfg.SourceID) 728 rows, err := k.dbConn.QuerySQL(k.tctx, k.metricProxies, query) 729 if err != nil { 730 return nil, terror.WithScope(err, terror.ScopeDownstream) 731 } 732 defer rows.Close() 733 734 var ( 735 targetTableID string 736 sourceTableID string 737 activeIndex int 738 isGlobal bool 739 data string 740 meta = make(map[string]*shardmeta.ShardingMeta) 741 ) 742 for rows.Next() { 743 err := rows.Scan(&targetTableID, &sourceTableID, &activeIndex, &isGlobal, &data) 744 if err != nil { 745 return nil, terror.DBErrorAdapt(err, k.dbConn.Scope(), terror.ErrDBDriverError) 746 } 747 if _, ok := meta[targetTableID]; !ok { 748 meta[targetTableID] = shardmeta.NewShardingMeta(k.shardMetaSchema, k.shardMetaTable, enableGTID) 749 } 750 err = meta[targetTableID].RestoreFromData(sourceTableID, activeIndex, isGlobal, []byte(data), flavor) 751 if err != nil { 752 return nil, err 753 } 754 } 755 return meta, terror.DBErrorAdapt(rows.Err(), k.dbConn.Scope(), terror.ErrDBDriverError) 756 } 757 758 // CheckAndFix try to check and fix the schema/table case-sensitive issue. 759 // 760 // NOTE: CheckAndFix is called before sharding groups are inited. 761 func (k *ShardingGroupKeeper) CheckAndFix(metas map[string]*shardmeta.ShardingMeta, schemaMap map[string]string, tablesMap map[string]map[string]string) error { 762 k.Lock() 763 defer k.Unlock() 764 for targetID, meta := range metas { 765 sqls, args, err := meta.CheckAndUpdate(k.tctx.L(), targetID, schemaMap, tablesMap) 766 if err != nil { 767 return err 768 } 769 _, err = k.dbConn.ExecuteSQL(k.tctx, k.metricProxies, sqls, args...) 770 if err != nil { 771 return err 772 } 773 } 774 775 return nil 776 } 777 778 // ShardingReSync represents re-sync info for a sharding DDL group. 779 type ShardingReSync struct { 780 currLocation binlog.Location // current DDL's binlog location, initialize to first DDL's location 781 latestLocation binlog.Location // latest DDL's binlog location 782 targetTable *filter.Table 783 allResolved bool 784 } 785 786 // String implements stringer.String. 787 func (s *ShardingReSync) String() string { 788 return fmt.Sprintf("{table: %v, current location: %v, latest location: %v, all resolved: %v}", s.targetTable, s.currLocation, s.latestLocation, s.allResolved) 789 }