github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/opt_sharding_group.go (about) 1 // Copyright 2022 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 "sync" 18 19 "github.com/pingcap/tidb/pkg/util/filter" 20 "github.com/pingcap/tiflow/dm/config" 21 "github.com/pingcap/tiflow/dm/pkg/binlog" 22 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 23 "github.com/pingcap/tiflow/dm/pkg/utils" 24 "go.uber.org/zap" 25 ) 26 27 // OptShardingGroup represents a optimistic sharding DDL sync group. 28 type OptShardingGroup struct { 29 sync.RWMutex 30 31 // the conflict tableIDs hash set to quickly check whether this table is in conflict stage 32 // sourceTableID -> each table's conflicted ddl's startLocation 33 conflictTables map[string]binlog.Location 34 35 firstConflictLocation binlog.Location 36 flavor string 37 enableGTID bool 38 } 39 40 func NewOptShardingGroup(firstConflictLocation binlog.Location, flavor string, enableGTID bool) *OptShardingGroup { 41 return &OptShardingGroup{ 42 firstConflictLocation: firstConflictLocation, 43 flavor: flavor, 44 enableGTID: enableGTID, 45 conflictTables: make(map[string]binlog.Location, 1), 46 } 47 } 48 49 func (s *OptShardingGroup) appendConflictTable(table *filter.Table, location binlog.Location) { 50 s.Lock() 51 defer s.Unlock() 52 s.conflictTables[table.String()] = location 53 } 54 55 func (s *OptShardingGroup) inConflictStage(table *filter.Table) bool { 56 s.RLock() 57 defer s.RUnlock() 58 _, ok := s.conflictTables[utils.GenTableID(table)] 59 return ok 60 } 61 62 func (s *OptShardingGroup) Remove(sourceTableIDs []string) { 63 s.Lock() 64 defer s.Unlock() 65 for _, sourceTbl := range sourceTableIDs { 66 delete(s.conflictTables, sourceTbl) 67 } 68 } 69 70 // OptShardingGroupKeeper used to keep OptShardingGroup. 71 // It's used to keep sharding group meta data to make sure optimistic sharding resync redirection works correctly. 72 // 73 // newer 74 // │ ───────────────────────► time 75 // │ 76 // │ tb1 conflict DDL1 │ ▲ │ 77 // │ │ │ │ 78 // │ ... │ │ │ 79 // │ │ │ │ 80 // │ tb1 conflict DDL2 │ │ │ ▲ │ 81 // │ │ │ │ │ │ 82 // │ ... │ │ │ │ │ 83 // │ │ │ │ │ │ 84 // │ tb2 conflict DDL1 ▼ │ │ │ │ 85 // │ │ │ │ 86 // │ ... redirect │ │ │ 87 // │ │ │ │ 88 // │ tb2 conflict DDL2 ▼ │ │ 89 // │ │ 90 // │ ... redirect │ 91 // │ │ 92 // │ other dml events ▼ 93 // │ 94 // │ continue 95 // ▼ replicating 96 // 97 // newer 98 // binlog 99 // One redirection example is listed as above. 100 type OptShardingGroupKeeper struct { 101 sync.RWMutex 102 groups map[string]*OptShardingGroup // target table ID -> ShardingGroup 103 cfg *config.SubTaskConfig 104 tctx *tcontext.Context 105 // shardingReSyncs is used to save the shardingResyncs' redirect locations that are resolved but not finished 106 shardingReSyncs map[string]binlog.Location 107 } 108 109 // NewOptShardingGroupKeeper creates a new OptShardingGroupKeeper. 110 func NewOptShardingGroupKeeper(tctx *tcontext.Context, cfg *config.SubTaskConfig) *OptShardingGroupKeeper { 111 return &OptShardingGroupKeeper{ 112 groups: make(map[string]*OptShardingGroup), 113 cfg: cfg, 114 tctx: tctx.WithLogger(tctx.L().WithFields(zap.String("component", "optimistic shard group keeper"))), 115 shardingReSyncs: make(map[string]binlog.Location), 116 } 117 } 118 119 func (k *OptShardingGroupKeeper) resolveGroup(targetTable *filter.Table) (map[string]binlog.Location, binlog.Location) { 120 targetTableID := utils.GenTableID(targetTable) 121 k.Lock() 122 defer k.Unlock() 123 group, ok := k.groups[targetTableID] 124 if !ok { 125 return nil, binlog.Location{} 126 } 127 delete(k.groups, targetTableID) 128 return group.conflictTables, group.firstConflictLocation 129 } 130 131 func (k *OptShardingGroupKeeper) inConflictStage(sourceTable, targetTable *filter.Table) bool { 132 targetTableID := utils.GenTableID(targetTable) 133 k.RLock() 134 group, ok := k.groups[targetTableID] 135 k.RUnlock() 136 if !ok { 137 return false 138 } 139 140 return group.inConflictStage(sourceTable) 141 } 142 143 func (k *OptShardingGroupKeeper) tableInConflict(targetTable *filter.Table) bool { 144 targetTableID := utils.GenTableID(targetTable) 145 k.RLock() 146 defer k.RUnlock() 147 _, ok := k.groups[targetTableID] 148 return ok 149 } 150 151 // appendConflictTable returns whether sourceTable is the first conflict table for targetTable. 152 func (k *OptShardingGroupKeeper) appendConflictTable(sourceTable, targetTable *filter.Table, 153 conflictLocation binlog.Location, flavor string, enableGTID bool, 154 ) bool { 155 targetTableID := utils.GenTableID(targetTable) 156 k.Lock() 157 group, ok := k.groups[targetTableID] 158 if !ok { 159 group = NewOptShardingGroup(conflictLocation, flavor, enableGTID) 160 k.groups[targetTableID] = group 161 } 162 k.Unlock() 163 group.appendConflictTable(sourceTable, conflictLocation) 164 return !ok 165 } 166 167 func (k *OptShardingGroupKeeper) addShardingReSync(shardingReSync *ShardingReSync) { 168 if shardingReSync != nil { 169 k.shardingReSyncs[shardingReSync.targetTable.String()] = shardingReSync.currLocation 170 } 171 } 172 173 func (k *OptShardingGroupKeeper) removeShardingReSync(shardingReSync *ShardingReSync) { 174 if shardingReSync != nil { 175 delete(k.shardingReSyncs, shardingReSync.targetTable.String()) 176 } 177 } 178 179 func (k *OptShardingGroupKeeper) getShardingResyncs() map[string]binlog.Location { 180 return k.shardingReSyncs 181 } 182 183 func (k *OptShardingGroupKeeper) lowestFirstLocationInGroups() *binlog.Location { 184 k.RLock() 185 defer k.RUnlock() 186 var lowest *binlog.Location 187 for _, group := range k.groups { 188 if lowest == nil || binlog.CompareLocation(*lowest, group.firstConflictLocation, k.cfg.EnableGTID) > 0 { 189 lowest = &group.firstConflictLocation 190 } 191 } 192 for _, currLocation := range k.shardingReSyncs { 193 if lowest == nil || binlog.CompareLocation(*lowest, currLocation, k.cfg.EnableGTID) > 0 { 194 loc := currLocation // make sure lowest can point to correct variable 195 lowest = &loc 196 } 197 } 198 if lowest == nil { 199 return nil 200 } 201 loc := lowest.Clone() 202 return &loc 203 } 204 205 // AdjustGlobalLocation adjusts globalLocation with sharding groups' lowest first point. 206 func (k *OptShardingGroupKeeper) AdjustGlobalLocation(globalLocation binlog.Location) binlog.Location { 207 lowestFirstLocation := k.lowestFirstLocationInGroups() 208 if lowestFirstLocation != nil && binlog.CompareLocation(*lowestFirstLocation, globalLocation, k.cfg.EnableGTID) < 0 { 209 return *lowestFirstLocation 210 } 211 return globalLocation 212 } 213 214 func (k *OptShardingGroupKeeper) RemoveGroup(targetTable *filter.Table, sourceTableIDs []string) { 215 targetTableID := utils.GenTableID(targetTable) 216 217 k.Lock() 218 defer k.Unlock() 219 if group, ok := k.groups[targetTableID]; ok { 220 group.Remove(sourceTableIDs) 221 if len(group.conflictTables) == 0 { 222 delete(k.groups, targetTableID) 223 } 224 } 225 } 226 227 func (k *OptShardingGroupKeeper) RemoveSchema(schema string) { 228 k.Lock() 229 defer k.Unlock() 230 for targetTableID := range k.groups { 231 if targetTable := utils.UnpackTableID(targetTableID); targetTable.Schema == schema { 232 delete(k.groups, targetTableID) 233 } 234 } 235 } 236 237 // Reset resets the keeper. 238 func (k *OptShardingGroupKeeper) Reset() { 239 k.Lock() 240 defer k.Unlock() 241 k.groups = make(map[string]*OptShardingGroup) 242 k.shardingReSyncs = make(map[string]binlog.Location) 243 }