github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/ha/load_task.go (about) 1 // Copyright 2021 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 ha 15 16 import ( 17 "context" 18 "encoding/json" 19 "fmt" 20 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 ) 27 28 // LoadTask uses to watch load worker events. 29 type LoadTask struct { 30 Task string 31 Source string 32 Worker string 33 IsDelete bool 34 } 35 36 // GetLoadTask gets the worker which in load stage for the source of the subtask. 37 // k/v: (task, sourceID) -> worker-name. 38 func GetLoadTask(cli *clientv3.Client, task, sourceID string) (string, int64, error) { 39 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 40 defer cancel() 41 resp, err := cli.Get(ctx, common.LoadTaskKeyAdapter.Encode(task, sourceID)) 42 if err != nil { 43 return "", 0, terror.ErrHAFailTxnOperation.Delegate(err, fmt.Sprintf("fail to get load task, task: %s, sourceID: %s", task, sourceID)) 44 } 45 46 if resp.Count <= 0 { 47 return "", resp.Header.Revision, nil 48 } 49 50 var worker string 51 err = json.Unmarshal(resp.Kvs[0].Value, &worker) 52 53 return worker, resp.Header.Revision, terror.ErrHAInvalidItem.Delegate(err, "fail to unmarshal load task") 54 } 55 56 // GetAllLoadTask gets all the worker which in load stage. 57 // k/v: (task, sourceID) -> worker-name. 58 func GetAllLoadTask(cli *clientv3.Client) (map[string]map[string]string, int64, error) { 59 var ( 60 worker string 61 loadTaskMap = make(map[string]map[string]string) 62 ) 63 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 64 defer cancel() 65 resp, err := cli.Get(ctx, common.LoadTaskKeyAdapter.Path(), clientv3.WithPrefix()) 66 if err != nil { 67 return loadTaskMap, 0, terror.ErrHAFailTxnOperation.Delegate(err, "fail to get all load task") 68 } 69 70 for _, kv := range resp.Kvs { 71 keys, err2 := common.LoadTaskKeyAdapter.Decode(string(kv.Key)) 72 if err2 != nil { 73 return nil, 0, err2 74 } 75 76 task := keys[0] 77 source := keys[1] 78 79 err = json.Unmarshal(kv.Value, &worker) 80 if err != nil { 81 return loadTaskMap, 0, terror.ErrHAInvalidItem.Delegate(err, "fail to unmarshal load task") 82 } 83 84 if _, ok := loadTaskMap[task]; !ok { 85 loadTaskMap[task] = make(map[string]string) 86 } 87 loadTaskMap[task][source] = worker 88 } 89 90 return loadTaskMap, resp.Header.Revision, nil 91 } 92 93 // WatchLoadTask watches PUT & DELETE operations for worker in load stage. 94 // This function should often be called by DM-master. 95 func WatchLoadTask(ctx context.Context, cli *clientv3.Client, revision int64, 96 outCh chan<- LoadTask, errCh chan<- error, 97 ) { 98 wCtx, cancel := context.WithCancel(ctx) 99 defer cancel() 100 // NOTE: WithPrevKV used to get a valid `ev.PrevKv` for deletion. 101 ch := cli.Watch(wCtx, common.LoadTaskKeyAdapter.Path(), 102 clientv3.WithPrefix(), clientv3.WithRev(revision), clientv3.WithPrevKV()) 103 104 for { 105 select { 106 case <-ctx.Done(): 107 return 108 case resp, ok := <-ch: 109 if !ok { 110 return 111 } 112 if resp.Canceled { 113 select { 114 case errCh <- terror.ErrHAFailWatchEtcd.Delegate(resp.Err(), "watch load task canceled"): 115 case <-ctx.Done(): 116 } 117 return 118 } 119 120 for _, ev := range resp.Events { 121 var ( 122 loadTask LoadTask 123 err error 124 keys []string 125 ) 126 127 switch ev.Type { 128 case mvccpb.PUT, mvccpb.DELETE: 129 keys, err = common.LoadTaskKeyAdapter.Decode(string(ev.Kv.Key)) 130 if err == nil { 131 loadTask.Task = keys[0] 132 loadTask.Source = keys[1] 133 if ev.Type == mvccpb.PUT { 134 err = json.Unmarshal(ev.Kv.Value, &loadTask.Worker) 135 } else { 136 loadTask.IsDelete = true 137 } 138 } 139 default: 140 // this should not happen. 141 err = fmt.Errorf("unsupported ectd event type %v", ev.Type) 142 } 143 144 if err != nil { 145 select { 146 case errCh <- err: 147 case <-ctx.Done(): 148 return 149 } 150 } else { 151 select { 152 case outCh <- loadTask: 153 case <-ctx.Done(): 154 return 155 } 156 } 157 } 158 } 159 } 160 } 161 162 // PutLoadTask puts the worker which load stage for the source of the subtask. 163 // k/v: (task, sourceID) -> worker. 164 // This function should often be called by DM-worker. 165 func PutLoadTask(cli *clientv3.Client, task, sourceID, worker string) (int64, error) { 166 data, err := json.Marshal(worker) 167 if err != nil { 168 return 0, err 169 } 170 key := common.LoadTaskKeyAdapter.Encode(task, sourceID) 171 172 _, rev, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(clientv3.OpPut(key, string(data)))) 173 if err != nil { 174 return 0, err 175 } 176 return rev, nil 177 } 178 179 // DelLoadTask dels the worker in load stage for the source of the subtask. 180 // k/v: (task, sourceID) -> worker. 181 func DelLoadTask(cli *clientv3.Client, task, sourceID string) (int64, bool, error) { 182 key := common.LoadTaskKeyAdapter.Encode(task, sourceID) 183 184 resp, rev, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(clientv3.OpDelete(key))) 185 if err != nil { 186 return 0, false, err 187 } 188 return rev, resp.Succeeded, nil 189 } 190 191 // DelLoadTaskByTask del the worker in load stage for the source by task. 192 func DelLoadTaskByTask(cli *clientv3.Client, task string) (int64, bool, error) { 193 key := common.LoadTaskKeyAdapter.Encode(task) 194 195 resp, rev, err := etcdutil.DoTxnWithRepeatable(cli, etcdutil.ThenOpFunc(clientv3.OpDelete(key, clientv3.WithPrefix()))) 196 if err != nil { 197 return 0, false, err 198 } 199 return rev, resp.Succeeded, nil 200 }