github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/shardddl/pessimism/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 pessimism 15 16 import ( 17 "context" 18 "encoding/json" 19 20 "github.com/pingcap/tiflow/dm/common" 21 "github.com/pingcap/tiflow/dm/pkg/etcdutil" 22 "go.etcd.io/etcd/api/v3/mvccpb" 23 clientv3 "go.etcd.io/etcd/client/v3" 24 "go.etcd.io/etcd/client/v3/clientv3util" 25 ) 26 27 // Operation represents a shard DDL coordinate operation. 28 // This information should be persistent in etcd so can be retrieved after the DM-master leader restarted or changed. 29 // NOTE: `Task` and `Source` are redundant in the etcd key path for convenient. 30 type Operation struct { 31 ID string `json:"id"` // the corresponding DDL lock ID 32 Task string `json:"task"` // data migration task name 33 Source string `json:"source"` // upstream source ID 34 DDLs []string `json:"ddls"` // DDL statements 35 Exec bool `json:"exec"` // execute or skip the DDL statements 36 Done bool `json:"done"` // whether the `Exec` operation has done 37 38 // only used to report to the caller of the watcher, do not marsh it. 39 // if it's true, it means the Operation has been deleted in etcd. 40 IsDeleted bool `json:"-"` 41 } 42 43 // NewOperation creates a new Operation instance. 44 func NewOperation(id, task, source string, ddls []string, exec, done bool) Operation { 45 return Operation{ 46 ID: id, 47 Task: task, 48 Source: source, 49 DDLs: ddls, 50 Exec: exec, 51 Done: done, 52 } 53 } 54 55 // String implements Stringer interface. 56 func (o Operation) String() string { 57 s, _ := o.toJSON() 58 return s 59 } 60 61 // toJSON returns the string of JSON represent. 62 func (o Operation) toJSON() (string, error) { 63 data, err := json.Marshal(o) 64 if err != nil { 65 return "", err 66 } 67 return string(data), nil 68 } 69 70 // operationFromJSON constructs Operation from its JSON represent. 71 func operationFromJSON(s string) (o Operation, err error) { 72 err = json.Unmarshal([]byte(s), &o) 73 return 74 } 75 76 // PutOperations puts the shard DDL operations into etcd. 77 // if `skipDone` is `true`: 78 // - PUT: all of kvs ("not exist" or "the `done` field is not `true`") 79 // - skip PUT: any of kvs ("exist" and "the `done` field is `true`") 80 // 81 // NOTE: 82 // 83 // `clientv3.Value` has a strange behavior for *not-exist* kv, 84 // see https://github.com/etcd-io/etcd/issues/10566. 85 // In addition, etcd compare has no `OR` operator now, 86 // see https://github.com/etcd-io/etcd/issues/10571. 87 // So, it's hard to do our `skipDone` logic in one txn. 88 // We break the logic into two txn, but this may lead to problem when PUT operations concurrently. 89 // 90 // This function should often be called by DM-master. 91 func PutOperations(cli *clientv3.Client, skipDone bool, ops ...Operation) (rev int64, putted bool, err error) { 92 cmpsNotExist := make([]clientv3.Cmp, 0, len(ops)) 93 cmpsNotDone := make([]clientv3.Cmp, 0, len(ops)) 94 opsPut := make([]clientv3.Op, 0, len(ops)) 95 for _, op := range ops { 96 value, err2 := op.toJSON() 97 if err2 != nil { 98 return 0, false, err2 99 } 100 101 key := common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source) 102 opsPut = append(opsPut, clientv3.OpPut(key, value)) 103 104 if skipDone { 105 opDone := op 106 opDone.Done = true // set `done` to `true`. 107 valueDone, err3 := opDone.toJSON() 108 if err3 != nil { 109 return 0, false, err3 110 } 111 cmpsNotExist = append(cmpsNotExist, clientv3util.KeyMissing(key)) 112 cmpsNotDone = append(cmpsNotDone, clientv3.Compare(clientv3.Value(key), "!=", valueDone)) 113 } 114 } 115 116 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 117 defer cancel() 118 119 // txn 1: try to PUT if all of kvs "not exist". 120 resp, err := cli.Txn(ctx).If(cmpsNotExist...).Then(opsPut...).Commit() 121 if err != nil { 122 return 0, false, err 123 } else if resp.Succeeded { 124 return resp.Header.Revision, resp.Succeeded, nil 125 } 126 127 // txn 2: try to PUT if all of kvs "the `done` field is not `true`. 128 // FIXME: if any "not `done`" kv putted after txn 1, this txn 2 will fail, but this is not what we want. 129 resp, err = cli.Txn(ctx).If(cmpsNotDone...).Then(opsPut...).Commit() 130 if err != nil { 131 return 0, false, err 132 } 133 return resp.Header.Revision, resp.Succeeded, nil 134 } 135 136 // DeleteOperations deletes the shard DDL operations in etcd. 137 // This function should often be called by DM-master. 138 func DeleteOperations(cli *clientv3.Client, ops ...Operation) (int64, error) { 139 opsDel := make([]clientv3.Op, 0, len(ops)) 140 for _, op := range ops { 141 opsDel = append(opsDel, deleteOperationOp(op)) 142 } 143 _, rev, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(opsDel...)) 144 return rev, err 145 } 146 147 // GetAllOperations gets all DDL lock operation in etcd currently. 148 // k/k/v: task-name -> source-ID -> lock operation. 149 func GetAllOperations(cli *clientv3.Client) (map[string]map[string]Operation, int64, error) { 150 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 151 defer cancel() 152 153 resp, err := cli.Get(ctx, common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) 154 if err != nil { 155 return nil, 0, err 156 } 157 158 opm := make(map[string]map[string]Operation) 159 for _, kv := range resp.Kvs { 160 op, err2 := operationFromJSON(string(kv.Value)) 161 if err2 != nil { 162 return nil, 0, err2 163 } 164 165 if _, ok := opm[op.Task]; !ok { 166 opm[op.Task] = make(map[string]Operation) 167 } 168 opm[op.Task][op.Source] = op 169 } 170 171 return opm, resp.Header.Revision, nil 172 } 173 174 // GetInfosOperationsByTask gets all DDL lock infos and operations in etcd currently. 175 func GetInfosOperationsByTask(cli *clientv3.Client, task string) ([]Info, []Operation, int64, error) { 176 respTxn, _, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc( 177 clientv3.OpGet(common.ShardDDLPessimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix()), 178 clientv3.OpGet(common.ShardDDLPessimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix()))) 179 if err != nil { 180 return nil, nil, 0, err 181 } 182 infoResp := respTxn.Responses[0].GetResponseRange() 183 opsResp := respTxn.Responses[1].GetResponseRange() 184 var ( 185 infos = make([]Info, 0, len(infoResp.Kvs)) 186 ops = make([]Operation, 0, len(opsResp.Kvs)) 187 ) 188 for _, kv := range infoResp.Kvs { 189 info, err2 := infoFromJSON(string(kv.Value)) 190 if err2 != nil { 191 return nil, nil, 0, err2 192 } 193 infos = append(infos, info) 194 } 195 for _, kv := range opsResp.Kvs { 196 op, err2 := operationFromJSON(string(kv.Value)) 197 if err2 != nil { 198 return nil, nil, 0, err2 199 } 200 ops = append(ops, op) 201 } 202 return infos, ops, respTxn.Header.Revision, nil 203 } 204 205 // WatchOperationPut watches PUT operations for DDL lock operation. 206 // If want to watch all operations, pass empty string for `task` and `source`. 207 // This function can be called by DM-worker and DM-master. 208 // TODO(csuzhangxc): report error and do some retry. 209 func WatchOperationPut(ctx context.Context, cli *clientv3.Client, task, source string, revision int64, outCh chan<- Operation, errCh chan<- error) { 210 watchOperation(ctx, cli, mvccpb.PUT, task, source, revision, outCh, errCh) 211 } 212 213 // WatchOperationDelete watches DELETE operations for DDL lock operation. 214 // If want to watch all operations, pass empty string for `task` and `source`. 215 // This function is often called by DM-worker. 216 func WatchOperationDelete(ctx context.Context, cli *clientv3.Client, task, source string, revision int64, outCh chan<- Operation, errCh chan<- error) { 217 watchOperation(ctx, cli, mvccpb.DELETE, task, source, revision, outCh, errCh) 218 } 219 220 // watchOperation watches PUT or DELETE operations for DDL lock operation. 221 func watchOperation(ctx context.Context, cli *clientv3.Client, watchType mvccpb.Event_EventType, 222 task, source string, revision int64, 223 outCh chan<- Operation, errCh chan<- error, 224 ) { 225 wCtx, cancel := context.WithCancel(ctx) 226 defer cancel() 227 var ch clientv3.WatchChan 228 // caller may use empty keys to expect a prefix watch 229 if source == "" { 230 ch = cli.Watch(wCtx, common.ShardDDLPessimismOperationKeyAdapter.Path(), clientv3.WithPrefix(), 231 clientv3.WithRev(revision), clientv3.WithPrevKV()) 232 } else { 233 ch = cli.Watch(wCtx, common.ShardDDLPessimismOperationKeyAdapter.Encode(task, source), 234 clientv3.WithRev(revision), clientv3.WithPrevKV()) 235 } 236 237 for { 238 select { 239 case <-ctx.Done(): 240 return 241 case resp, ok := <-ch: 242 if !ok { 243 return 244 } 245 if resp.Canceled { 246 select { 247 case errCh <- resp.Err(): 248 case <-ctx.Done(): 249 } 250 return 251 } 252 253 for _, ev := range resp.Events { 254 var ( 255 op Operation 256 err error 257 ) 258 259 if ev.Type == mvccpb.PUT && watchType == mvccpb.PUT { 260 op, err = operationFromJSON(string(ev.Kv.Value)) 261 } else if ev.Type == mvccpb.DELETE && watchType == mvccpb.DELETE { 262 op, err = operationFromJSON(string(ev.PrevKv.Value)) 263 op.IsDeleted = true 264 } 265 266 if err != nil { 267 select { 268 case errCh <- err: 269 case <-ctx.Done(): 270 return 271 } 272 } else if op.Task != "" { // valid operation got. 273 select { 274 case outCh <- op: 275 case <-ctx.Done(): 276 return 277 } 278 } 279 } 280 } 281 } 282 } 283 284 // putOperationOp returns a PUT etcd operation for Operation. 285 // This operation should often be sent by DM-worker. 286 func putOperationOp(o Operation) (clientv3.Op, error) { 287 value, err := o.toJSON() 288 if err != nil { 289 return clientv3.Op{}, err 290 } 291 key := common.ShardDDLPessimismOperationKeyAdapter.Encode(o.Task, o.Source) 292 293 return clientv3.OpPut(key, value), nil 294 } 295 296 // deleteOperationOp returns a DELETE etcd operation for Operation. 297 func deleteOperationOp(op Operation) clientv3.Op { 298 return clientv3.OpDelete(common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source)) 299 } 300 301 func getOperationOp(op Operation) clientv3.Op { 302 return clientv3.OpGet(common.ShardDDLPessimismOperationKeyAdapter.Encode(op.Task, op.Source)) 303 }