github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/shardddl/optimism/operation.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 optimism 15 16 import ( 17 "context" 18 "encoding/json" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/tiflow/dm/common" 22 "github.com/pingcap/tiflow/dm/pkg/etcdutil" 23 "github.com/pingcap/tiflow/dm/pkg/terror" 24 "go.etcd.io/etcd/api/v3/mvccpb" 25 clientv3 "go.etcd.io/etcd/client/v3" 26 "go.etcd.io/etcd/client/v3/clientv3util" 27 ) 28 29 // ConflictStage represents the current shard DDL conflict stage in the optimistic mode. 30 type ConflictStage string 31 32 const ( 33 // ConflictNone indicates no conflict exists, 34 // DM-worker can execute DDL/DML to the downstream normally. 35 ConflictNone ConflictStage = "none" 36 // ConflictDetected indicates a conflict will exist after applied the shard DDL. 37 // in this stage, DM-worker should not execute/skip DDL/DML, 38 // but it should still try to find the DDL which can resolve the conflict in the binlog stream. 39 ConflictDetected ConflictStage = "detected" 40 // ConflictResolved indicates a conflict DDL be resolved. 41 // in this stage, DM-worker should redirect to the conflict DDL. 42 ConflictResolved ConflictStage = "resolved" 43 // ConflictUnlocked indicates a conflict will be unlocked after applied the shard DDL. 44 // in this stage, DM-worker should directly execute/skip DDLs. 45 ConflictUnlocked ConflictStage = "unlocked" 46 // ConflictSkipWaitRedirect indicates a conflict happened and will be skipped and redirected until all tables has no conflict 47 // in this stage, DM-worker should skip all DML and DDL for the conflict table until redirect. 48 ConflictSkipWaitRedirect ConflictStage = "skip and wait for redirect" // #nosec 49 // ConflictError indicates an error happened when we try to sync the DDLs 50 // in this stage, DM-worker should retry and can skip ddls for this error. 51 ConflictError ConflictStage = "error" 52 ) 53 54 // Operation represents a shard DDL coordinate operation. 55 // This information should be persistent in etcd so can be retrieved after the DM-master leader restarted or changed. 56 // NOTE: `Task`, `Source`, `UpSchema` and `UpTable` are redundant in the etcd key path for convenient. 57 // Operation is putted when coordinating a shard DDL operation for DM-worker by DM-master, 58 // and is updated (with `done`) after DM-worker has done the operation by DM-worker, 59 // and is deleted when removing the lock by DM-master. 60 // because we need the newest stage in Operation to recover the lock when restarting DM-master. 61 type Operation struct { 62 ID string `json:"id"` // the corresponding DDL lock ID 63 Task string `json:"task"` // data migration task name 64 Source string `json:"source"` // upstream source ID 65 UpSchema string `json:"up-schema"` // upstream/source schema name, different sources can have the same schema name 66 UpTable string `json:"up-table"` // upstream/source table name, different sources can have the same table name 67 DDLs []string `json:"ddls"` // DDL statements need to apply to the downstream. 68 ConflictStage ConflictStage `json:"conflict-stage"` // current conflict stage. 69 ConflictMsg string `json:"conflict-message"` // current conflict message 70 Done bool `json:"done"` // whether the operation has done 71 Cols []string `json:"cols"` // drop columns' name 72 73 // only set it when get from etcd 74 // use for sort infos in recovering locks 75 Revision int64 `json:"-"` 76 } 77 78 // NewOperation creates a new Operation instance. 79 func NewOperation(id, task, source, upSchema, upTable string, 80 ddls []string, conflictStage ConflictStage, conflictMsg string, done bool, cols []string, 81 ) Operation { 82 return Operation{ 83 ID: id, 84 Task: task, 85 Source: source, 86 UpSchema: upSchema, 87 UpTable: upTable, 88 DDLs: ddls, 89 ConflictStage: conflictStage, 90 ConflictMsg: conflictMsg, 91 Done: done, 92 Cols: cols, 93 } 94 } 95 96 // String implements Stringer interface. 97 func (o Operation) String() string { 98 s, _ := o.toJSON() 99 return s 100 } 101 102 // toJSON returns the string of JSON represent. 103 func (o Operation) toJSON() (string, error) { 104 data, err := json.Marshal(o) 105 if err != nil { 106 return "", err 107 } 108 return string(data), nil 109 } 110 111 // operationFromJSON constructs Operation from its JSON represent. 112 func operationFromJSON(s string) (o Operation, err error) { 113 err = json.Unmarshal([]byte(s), &o) 114 return 115 } 116 117 // PutOperation puts the shard DDL operation into etcd. 118 func PutOperation(cli *clientv3.Client, skipDone bool, op Operation, infoModRev int64) (rev int64, putted bool, err error) { 119 value, err := op.toJSON() 120 if err != nil { 121 return 0, false, err 122 } 123 key := common.ShardDDLOptimismOperationKeyAdapter.Encode(op.Task, op.Source, op.UpSchema, op.UpTable) 124 opPut := clientv3.OpPut(key, value) 125 126 cmpsNotExist := make([]clientv3.Cmp, 0, 1) 127 cmpsNotDone := make([]clientv3.Cmp, 0, 1) 128 cmpsLessRev := make([]clientv3.Cmp, 0, 1) 129 if skipDone { 130 opDone := op 131 opDone.Done = true // set `done` to `true`. 132 valueDone, err2 := opDone.toJSON() 133 if err2 != nil { 134 return 0, false, err2 135 } 136 cmpsNotExist = append(cmpsNotExist, clientv3util.KeyMissing(key)) 137 cmpsNotDone = append(cmpsNotDone, clientv3.Compare(clientv3.Value(key), "!=", valueDone)) 138 cmpsLessRev = append(cmpsLessRev, clientv3.Compare(clientv3.ModRevision(key), "<", infoModRev)) 139 } 140 141 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 142 defer cancel() 143 144 // txn 1: try to PUT if the key "not exist". 145 resp, err := cli.Txn(ctx).If(cmpsNotExist...).Then(opPut).Commit() 146 if err != nil { 147 return 0, false, terror.ErrHAFailTxnOperation.Delegate(err, "fail to put operation at not exist") 148 } else if resp.Succeeded { 149 return resp.Header.Revision, resp.Succeeded, nil 150 } 151 152 // txn 2: try to PUT if the key "the `done`" field is not `true`. 153 resp, err = cli.Txn(ctx).If(cmpsNotDone...).Then(opPut).Commit() 154 if err != nil { 155 return 0, false, terror.ErrHAFailTxnOperation.Delegate(err, "fail to put operation at not done") 156 } else if resp.Succeeded { 157 return resp.Header.Revision, resp.Succeeded, nil 158 } 159 160 // txn 3: try to PUT if the key has less mod revision than info's mod revision, which means this operation is an old one 161 // without this, failed case time series: 162 // 1. dm-master received an old done DDL operation from dm-worker 163 // 2. dm-worker putted a new DDL info into dm-master 164 // 3. dm-master quited before dm-master putted the DDL operation to dm-worker 165 // 4. dm-master restarted and tried to put DDL operation, but found a done one and failed to put 166 // 5. dm-worker didn't receive a DDL operation, will get blocked forever 167 resp, err = cli.Txn(ctx).If(cmpsLessRev...).Then(opPut).Commit() 168 if err != nil { 169 return 0, false, terror.ErrHAFailTxnOperation.Delegate(err, "fail to put operation at less rev") 170 } 171 return resp.Header.Revision, resp.Succeeded, nil 172 } 173 174 // GetAllOperations gets all shard DDL operation in etcd currently. 175 // This function should often be called by DM-master. 176 // k/k/k/k/v: task-name -> source-ID -> upstream-schema-name -> upstream-table-name -> shard DDL operation. 177 func GetAllOperations(cli *clientv3.Client) (map[string]map[string]map[string]map[string]Operation, int64, error) { 178 respTxn, _, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(clientv3.OpGet(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()))) 179 if err != nil { 180 return nil, 0, err 181 } 182 resp := respTxn.Responses[0].GetResponseRange() 183 184 opm := make(map[string]map[string]map[string]map[string]Operation) 185 for _, kv := range resp.Kvs { 186 op, err2 := operationFromJSON(string(kv.Value)) 187 if err2 != nil { 188 return nil, 0, err2 189 } 190 op.Revision = kv.ModRevision 191 192 if _, ok := opm[op.Task]; !ok { 193 opm[op.Task] = make(map[string]map[string]map[string]Operation) 194 } 195 if _, ok := opm[op.Task][op.Source]; !ok { 196 opm[op.Task][op.Source] = make(map[string]map[string]Operation) 197 } 198 if _, ok := opm[op.Task][op.Source][op.UpSchema]; !ok { 199 opm[op.Task][op.Source][op.UpSchema] = make(map[string]Operation) 200 } 201 opm[op.Task][op.Source][op.UpSchema][op.UpTable] = op 202 } 203 204 return opm, resp.Header.Revision, nil 205 } 206 207 // GetOperation gets shard DDL operation in etcd currently. 208 // This function should often be called by DM-worker. 209 // (task-name, source-ID, upstream-schema-name, upstream-table-name) -> shard DDL operation. 210 func GetOperation(cli *clientv3.Client, task, source, upSchema, upTable string) (Operation, int64, error) { 211 respTxn, _, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(clientv3.OpGet(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source, upSchema, upTable)))) 212 if err != nil { 213 return Operation{}, 0, err 214 } 215 resp := respTxn.Responses[0].GetResponseRange() 216 217 switch { 218 case resp.Count == 0: 219 return Operation{}, resp.Header.Revision, nil 220 case resp.Count > 1: 221 return Operation{}, 0, errors.Errorf("too many operations for %s/%s/%s/%s", task, source, upSchema, upTable) 222 default: 223 op, err2 := operationFromJSON(string(resp.Kvs[0].Value)) 224 if err2 != nil { 225 return Operation{}, 0, err2 226 } 227 op.Revision = resp.Kvs[0].ModRevision 228 return op, op.Revision, nil 229 } 230 } 231 232 // GetInfosOperationsByTask gets all shard DDL info and operation in etcd currently. 233 // This function should often be called by DM-master. 234 func GetInfosOperationsByTask(cli *clientv3.Client, task string) ([]Info, []Operation, int64, error) { 235 respTxn, _, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc( 236 clientv3.OpGet(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix()), 237 clientv3.OpGet(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix()))) 238 if err != nil { 239 return nil, nil, 0, err 240 } 241 infoResp := respTxn.Responses[0].GetResponseRange() 242 opsResp := respTxn.Responses[1].GetResponseRange() 243 var ( 244 infos = make([]Info, 0, len(infoResp.Kvs)) 245 ops = make([]Operation, 0, len(opsResp.Kvs)) 246 ) 247 for _, kv := range infoResp.Kvs { 248 info, err2 := infoFromJSON(string(kv.Value)) 249 if err2 != nil { 250 return nil, nil, 0, err2 251 } 252 infos = append(infos, info) 253 } 254 for _, kv := range opsResp.Kvs { 255 op, err2 := operationFromJSON(string(kv.Value)) 256 if err2 != nil { 257 return nil, nil, 0, err2 258 } 259 op.Revision = kv.ModRevision 260 ops = append(ops, op) 261 } 262 return infos, ops, respTxn.Header.Revision, nil 263 } 264 265 // WatchOperationPut watches PUT operations for DDL lock operation. 266 // If want to watch all operations matching, pass empty string for `task`, `source`, `upSchema` and `upTable`. 267 // This function can be called by DM-worker and DM-master. 268 func WatchOperationPut(ctx context.Context, cli *clientv3.Client, 269 task, source, upSchema, upTable string, revision int64, 270 outCh chan<- Operation, errCh chan<- error, 271 ) { 272 var ch clientv3.WatchChan 273 wCtx, cancel := context.WithCancel(ctx) 274 defer cancel() 275 // caller may use empty keys to expect a prefix watch 276 if upTable == "" { 277 ch = cli.Watch(wCtx, common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix(), 278 clientv3.WithRev(revision)) 279 } else { 280 ch = cli.Watch(wCtx, common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source, upSchema, upTable), 281 clientv3.WithRev(revision)) 282 } 283 284 for { 285 select { 286 case <-ctx.Done(): 287 return 288 case resp, ok := <-ch: 289 if !ok { 290 return 291 } 292 if resp.Canceled { 293 select { 294 case errCh <- resp.Err(): 295 case <-ctx.Done(): 296 } 297 return 298 } 299 300 for _, ev := range resp.Events { 301 if ev.Type != mvccpb.PUT { 302 continue 303 } 304 305 op, err := operationFromJSON(string(ev.Kv.Value)) 306 if err != nil { 307 select { 308 case errCh <- err: 309 case <-ctx.Done(): 310 return 311 } 312 } else { 313 op.Revision = ev.Kv.ModRevision 314 select { 315 case outCh <- op: 316 case <-ctx.Done(): 317 return 318 } 319 } 320 } 321 } 322 } 323 } 324 325 // deleteOperationOp returns a DELETE etcd operation for Operation. 326 func deleteOperationOp(op Operation) clientv3.Op { 327 return clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(op.Task, op.Source, op.UpSchema, op.UpTable)) 328 } 329 330 // CheckOperations try to check and fix all the schema and table names for operation infos. 331 func CheckOperations(cli *clientv3.Client, source string, schemaMap map[string]string, tablesMap map[string]map[string]string) error { 332 allOperations, rev, err := GetAllOperations(cli) 333 if err != nil { 334 return err 335 } 336 337 for _, taskTableOps := range allOperations { 338 sourceOps, ok := taskTableOps[source] 339 if !ok { 340 continue 341 } 342 for schema, tblOps := range sourceOps { 343 realSchema, hasChange := schemaMap[schema] 344 if !hasChange { 345 realSchema = schema 346 } 347 348 tblMap := tablesMap[schema] 349 for tbl, info := range tblOps { 350 realTable, tblChange := tblMap[tbl] 351 if !tblChange { 352 realTable = tbl 353 tblChange = hasChange 354 } 355 if tblChange { 356 newOperation := info 357 newOperation.UpSchema = realSchema 358 newOperation.UpTable = realTable 359 _, _, err = PutOperation(cli, false, newOperation, rev) 360 if err != nil { 361 return err 362 } 363 deleteOp := deleteOperationOp(info) 364 _, _, err = etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(deleteOp)) 365 if err != nil { 366 return err 367 } 368 } 369 } 370 } 371 } 372 return err 373 }