github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/shardddl/pessimism/info.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 "github.com/pingcap/tiflow/dm/pkg/utils" 23 "go.etcd.io/etcd/api/v3/mvccpb" 24 clientv3 "go.etcd.io/etcd/client/v3" 25 "go.etcd.io/etcd/client/v3/clientv3util" 26 ) 27 28 // Info represents the shard DDL information. 29 // This information should be persistent in etcd so can be retrieved after the DM-master leader restarted or changed. 30 // NOTE: `Task` and `Source` are redundant in the etcd key path for convenient. 31 type Info struct { 32 Task string `json:"task"` // data migration task name 33 Source string `json:"source"` // upstream source ID 34 Schema string `json:"schema"` // schema name of the DDL 35 Table string `json:"table"` // table name of the DDL 36 DDLs []string `json:"ddls"` // DDL statements 37 } 38 39 // NewInfo creates a new Info instance. 40 func NewInfo(task, source, schema, table string, ddls []string) Info { 41 return Info{ 42 Task: task, 43 Source: source, 44 Schema: schema, 45 Table: table, 46 DDLs: ddls, 47 } 48 } 49 50 // String implements Stringer interface. 51 func (i Info) String() string { 52 s, _ := i.toJSON() 53 return s 54 } 55 56 // toJSON returns the string of JSON represent. 57 func (i Info) toJSON() (string, error) { 58 data, err := json.Marshal(i) 59 if err != nil { 60 return "", err 61 } 62 return string(data), nil 63 } 64 65 // infoFromJSON constructs Info from its JSON represent. 66 func infoFromJSON(s string) (i Info, err error) { 67 err = json.Unmarshal([]byte(s), &i) 68 return 69 } 70 71 // PutInfo puts the shard DDL info into etcd. 72 // NOTE: 73 // 74 // In some cases before the lock resolved, the same DDL info may be PUT multiple times: 75 // 1. start-task after stop-task. 76 // 2. resume-task after paused manually or automatically. 77 // 3. the task scheduled to another DM-worker instance (just like case-1). 78 // Then we need to ensure re-PUT is safe: 79 // 1. DM-master can construct the lock and do the coordination correctly. 80 // 2. DM-worker can re-PUT and comply with the coordination correctly. 81 // 82 // This function should often be called by DM-worker. 83 func PutInfo(cli *clientv3.Client, info Info) (int64, error) { 84 value, err := info.toJSON() 85 if err != nil { 86 return 0, err 87 } 88 key := common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source) 89 90 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 91 defer cancel() 92 93 resp, err := cli.Put(ctx, key, value) 94 if err != nil { 95 return 0, err 96 } 97 return resp.Header.Revision, nil 98 } 99 100 // PutInfoIfOpNotDone puts the shard DDL info into etcd if the operation not exists or not `done`. 101 func PutInfoIfOpNotDone(cli *clientv3.Client, info Info) (rev int64, putted bool, err error) { 102 infoValue, err := info.toJSON() 103 if err != nil { 104 return 0, false, err 105 } 106 infoKey := common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source) 107 opKey := common.ShardDDLPessimismOperationKeyAdapter.Encode(info.Task, info.Source) 108 infoPut := clientv3.OpPut(infoKey, infoValue) 109 opGet := clientv3.OpGet(opKey) 110 111 // try to PUT info if the operation not exist. 112 resp, rev, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.FullOpFunc([]clientv3.Cmp{clientv3util.KeyMissing(opKey)}, 113 []clientv3.Op{infoPut}, []clientv3.Op{opGet})) 114 if err != nil { 115 return 0, false, err 116 } else if resp.Succeeded { 117 return rev, resp.Succeeded, nil 118 } 119 120 opsResp := resp.Responses[0].GetResponseRange() 121 opBefore, err := operationFromJSON(string(opsResp.Kvs[0].Value)) 122 switch { 123 case err != nil: 124 return 0, false, err 125 case opBefore.Done: 126 // the operation with `done` exist before, abort the PUT. 127 return rev, false, nil 128 case utils.CompareShardingDDLs(opBefore.DDLs, info.DDLs): 129 // TODO: try to handle put the same `done` DDL later. 130 } 131 132 // NOTE: try to PUT info if the operation still not done. 133 opNotDone := clientv3.Compare(clientv3.Value(opKey), "=", string(opsResp.Kvs[0].Value)) 134 resp, rev, err = etcdutil.DoTxnWithRepeatable(cli, etcdutil.FullOpFunc([]clientv3.Cmp{opNotDone}, []clientv3.Op{infoPut}, []clientv3.Op{})) 135 if err != nil { 136 return 0, false, err 137 } 138 return rev, resp.Succeeded, nil 139 } 140 141 // GetAllInfo gets all shard DDL info in etcd currently. 142 // k/k/v: task-name -> source-ID -> DDL info. 143 // This function should often be called by DM-master. 144 func GetAllInfo(cli *clientv3.Client) (map[string]map[string]Info, int64, error) { 145 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 146 defer cancel() 147 148 resp, err := cli.Get(ctx, common.ShardDDLPessimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) 149 if err != nil { 150 return nil, 0, err 151 } 152 153 ifm := make(map[string]map[string]Info) 154 for _, kv := range resp.Kvs { 155 info, err2 := infoFromJSON(string(kv.Value)) 156 if err2 != nil { 157 return nil, 0, err2 158 } 159 160 if _, ok := ifm[info.Task]; !ok { 161 ifm[info.Task] = make(map[string]Info) 162 } 163 ifm[info.Task][info.Source] = info 164 } 165 166 return ifm, resp.Header.Revision, nil 167 } 168 169 // WatchInfoPut watches PUT operations for info. 170 // This function should often be called by DM-master. 171 func WatchInfoPut(ctx context.Context, cli *clientv3.Client, revision int64, outCh chan<- Info, errCh chan<- error) { 172 wCtx, cancel := context.WithCancel(ctx) 173 defer cancel() 174 ch := cli.Watch(wCtx, common.ShardDDLPessimismInfoKeyAdapter.Path(), 175 clientv3.WithPrefix(), clientv3.WithRev(revision)) 176 177 for { 178 select { 179 case <-ctx.Done(): 180 return 181 case resp, ok := <-ch: 182 if !ok { 183 return 184 } 185 if resp.Canceled { 186 select { 187 case errCh <- resp.Err(): 188 case <-ctx.Done(): 189 } 190 return 191 } 192 193 for _, ev := range resp.Events { 194 if ev.Type != mvccpb.PUT { 195 continue 196 } 197 198 info, err := infoFromJSON(string(ev.Kv.Value)) 199 if err != nil { 200 select { 201 case errCh <- err: 202 case <-ctx.Done(): 203 return 204 } 205 } else { 206 select { 207 case outCh <- info: 208 case <-ctx.Done(): 209 return 210 } 211 } 212 } 213 } 214 } 215 } 216 217 // deleteInfoOp returns a DELETE etcd operation for info. 218 // This operation should often be sent by DM-worker. 219 func deleteInfoOp(info Info) clientv3.Op { 220 return clientv3.OpDelete(common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source)) 221 } 222 223 // infoExistCmp returns a etcd Cmp which indicates the info exists. 224 func infoExistCmp(info Info) clientv3.Cmp { 225 return clientv3util.KeyExists(common.ShardDDLPessimismInfoKeyAdapter.Encode(info.Task, info.Source)) 226 }