github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/optimist.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 syncer 15 16 import ( 17 "context" 18 19 "github.com/pingcap/tidb/pkg/util/filter" 20 "github.com/pingcap/tiflow/dm/pkg/utils" 21 "go.uber.org/zap" 22 ) 23 24 // initOptimisticShardDDL initializes the shard DDL support in the optimistic mode. 25 func (s *Syncer) initOptimisticShardDDL(ctx context.Context) error { 26 // fetch tables from source and filter them 27 sourceTables, err := s.fromDB.FetchAllDoTables(ctx, s.baList) 28 if err != nil { 29 return err 30 } 31 32 // convert according to router rules. 33 // downstream-schema -> downstream-table -> upstream-schema -> upstream-table. 34 // TODO: refine to downstream-ID -> upstream-ID 35 mapper := make(map[string]map[string]map[string]map[string]struct{}) 36 for upSchema, UpTables := range sourceTables { 37 for _, upTable := range UpTables { 38 up := &filter.Table{Schema: upSchema, Name: upTable} 39 down := s.route(up) 40 downSchema, downTable := down.Schema, down.Name 41 if _, ok := mapper[downSchema]; !ok { 42 mapper[downSchema] = make(map[string]map[string]map[string]struct{}) 43 } 44 if _, ok := mapper[downSchema][downTable]; !ok { 45 mapper[downSchema][downTable] = make(map[string]map[string]struct{}) 46 } 47 if _, ok := mapper[downSchema][downTable][upSchema]; !ok { 48 mapper[downSchema][downTable][upSchema] = make(map[string]struct{}) 49 } 50 mapper[downSchema][downTable][upSchema][upTable] = struct{}{} 51 } 52 } 53 54 return s.optimist.Init(mapper) 55 } 56 57 func (s *Syncer) resolveOptimisticDDL(ec *eventContext, sourceTable, targetTable *filter.Table) bool { 58 if sourceTable != nil && targetTable != nil { 59 if s.osgk.inConflictStage(sourceTable, targetTable) { 60 // in the following two situations we should resolve this ddl lock at now 61 // 1. after this worker's ddl, the ddl lock is resolved 62 // 2. other worker has resolved this ddl lock, receives resolve command from master 63 // TODO: maybe we don't need to resolve ddl lock in situation 1, because when situation 1 happens we 64 // should always receive a resolve operation like situation 2. 65 group, redirectLocation := s.osgk.resolveGroup(targetTable) 66 if len(group) > 0 { 67 s.optimist.DoneRedirectOperation(utils.GenTableID(targetTable)) 68 resync := &ShardingReSync{ 69 currLocation: redirectLocation, 70 latestLocation: ec.endLocation, 71 targetTable: targetTable, 72 allResolved: true, 73 } 74 s.osgk.tctx.L().Info("sending resync operation in optimistic shard mode", 75 zap.Stringer("shardingResync", resync)) 76 *ec.shardingReSyncCh <- resync 77 s.osgk.addShardingReSync(resync) 78 return true 79 } 80 } 81 } else { 82 s.osgk.tctx.L().Warn("invalid resolveOptimistic deploy without sourceTable/targetTable in optimistic shard mode", 83 zap.Bool("emptySourceTable", sourceTable == nil), 84 zap.Bool("emptyTargetTable", targetTable == nil)) 85 } 86 return false 87 }