github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/ha/keepalive.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 ha 15 16 import ( 17 "context" 18 "encoding/json" 19 "fmt" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/tiflow/dm/common" 24 "github.com/pingcap/tiflow/dm/pkg/etcdutil" 25 "github.com/pingcap/tiflow/dm/pkg/log" 26 "github.com/pingcap/tiflow/dm/pkg/terror" 27 "go.etcd.io/etcd/api/v3/mvccpb" 28 clientv3 "go.etcd.io/etcd/client/v3" 29 "go.uber.org/zap" 30 ) 31 32 var ( 33 // currentKeepAliveTTL may be assigned to KeepAliveTTL or RelayKeepAliveTTL. 34 currentKeepAliveTTL int64 35 // KeepAliveUpdateCh is used to notify keepalive TTL changing, in order to let watcher not see a DELETE of old key. 36 KeepAliveUpdateCh = make(chan int64, 10) 37 ) 38 39 // WorkerEvent represents the PUT/DELETE keepalive event of DM-worker. 40 type WorkerEvent struct { 41 WorkerName string `json:"worker-name"` // the worker name of the worker. 42 JoinTime time.Time `json:"join-time"` // the time when worker start to keepalive with etcd 43 44 // only used to report to the caller of the watcher, do not marsh it. 45 // if it's true, it means the worker has been deleted in etcd. 46 IsDeleted bool `json:"-"` 47 } 48 49 // String implements Stringer interface. 50 func (w WorkerEvent) String() string { 51 str, _ := w.toJSON() 52 return str 53 } 54 55 // toJSON returns the string of JSON represent. 56 func (w WorkerEvent) toJSON() (string, error) { 57 data, err := json.Marshal(w) 58 if err != nil { 59 return "", terror.ErrHAInvalidItem.Delegate(err, fmt.Sprintf("failed to marshal worker event %+v", w)) 60 } 61 return string(data), nil 62 } 63 64 // workerEventFromJSON constructs WorkerEvent from its JSON represent. 65 func workerEventFromJSON(s string) (w WorkerEvent, err error) { 66 if err = json.Unmarshal([]byte(s), &w); err != nil { 67 err = terror.ErrHAInvalidItem.Delegate(err, fmt.Sprintf("failed to unmarshal worker event %s", s)) 68 } 69 return 70 } 71 72 func workerEventFromKey(key string) (WorkerEvent, error) { 73 var w WorkerEvent 74 ks, err := common.WorkerKeepAliveKeyAdapter.Decode(key) 75 if err != nil { 76 return w, err 77 } 78 w.WorkerName = ks[0] 79 return w, nil 80 } 81 82 // KeepAlive puts the join time of the workerName into etcd. 83 // this key will be kept in etcd until the worker is blocked or failed 84 // k/v: workerName -> join time. 85 func KeepAlive(ctx context.Context, cli *clientv3.Client, workerName string, keepAliveTTL int64) error { 86 // TTL in KeepAliveUpdateCh has higher priority 87 for len(KeepAliveUpdateCh) > 0 { 88 keepAliveTTL = <-KeepAliveUpdateCh 89 } 90 // a test concurrently call KeepAlive though in normal running we don't do that 91 atomic.StoreInt64(¤tKeepAliveTTL, keepAliveTTL) 92 93 k := common.WorkerKeepAliveKeyAdapter.Encode(workerName) 94 workerEventJSON, err := WorkerEvent{ 95 WorkerName: workerName, 96 JoinTime: time.Now(), 97 }.toJSON() 98 if err != nil { 99 return err 100 } 101 102 grantAndPutKV := func(k, v string, ttl int64) (clientv3.LeaseID, error) { 103 cliCtx, cancel := context.WithTimeout(ctx, etcdutil.DefaultRequestTimeout) 104 defer cancel() 105 lease, err2 := cli.Grant(cliCtx, ttl) 106 if err2 != nil { 107 return 0, terror.ErrHAFailLeaseOperation.Delegate(err2, "failed to grant lease for worker keepalive") 108 } 109 _, err = cli.Put(cliCtx, k, v, clientv3.WithLease(lease.ID)) 110 if err != nil { 111 return 0, terror.ErrHAFailTxnOperation.Delegate(err, "failed to put worker keepalive with lease") 112 } 113 return lease.ID, nil 114 } 115 116 leaseID, err := grantAndPutKV(k, workerEventJSON, keepAliveTTL) 117 if err != nil { 118 return err 119 } 120 121 // once we put the key successfully, we should revoke lease before we quit keepalive normally 122 defer func() { 123 _, err2 := revokeLease(cli, leaseID) 124 if err2 != nil { 125 log.L().Warn("fail to revoke lease", zap.Error(err)) 126 } 127 }() 128 129 keepAliveCtx, keepAliveCancel := context.WithCancel(ctx) 130 defer keepAliveCancel() 131 132 ch, err := cli.KeepAlive(keepAliveCtx, leaseID) 133 if err != nil { 134 return terror.ErrHAFailKeepalive.Delegate(err, "failed to keepalive") 135 } 136 for { 137 select { 138 case _, ok := <-ch: 139 if !ok { 140 log.L().Info("keep alive channel is closed") 141 return nil 142 } 143 case <-ctx.Done(): 144 log.L().Info("ctx is canceled, keepalive will exit now") 145 return nil 146 case newTTL := <-KeepAliveUpdateCh: 147 if newTTL == currentKeepAliveTTL { 148 log.L().Info("ignore same keepalive TTL change", zap.Int64("TTL", newTTL)) 149 continue 150 } 151 152 // create a new lease with new TTL, and overwrite original KV 153 oldLeaseID := leaseID 154 leaseID, err = grantAndPutKV(k, workerEventJSON, newTTL) 155 if err != nil { 156 log.L().Error("meet error when grantAndPutKV keepalive TTL", zap.Error(err)) 157 return err 158 } 159 160 ch, err = cli.KeepAlive(keepAliveCtx, leaseID) 161 if err != nil { 162 log.L().Error("meet error when change keepalive TTL", zap.Error(err)) 163 return terror.ErrHAFailKeepalive.Delegate(err, "failed to keepalive") 164 } 165 currentKeepAliveTTL = newTTL 166 log.L().Info("dynamically changed keepalive TTL to", zap.Int64("ttl in seconds", newTTL)) 167 168 // after new keepalive succeed, we cancel the old keepalive 169 _, err2 := revokeLease(cli, oldLeaseID) 170 if err2 != nil { 171 log.L().Warn("fail to revoke lease", zap.Error(err)) 172 } 173 } 174 } 175 } 176 177 // ATTENTION!!! we must ensure cli.Ctx() not done when we are exiting worker 178 // Do not set cfg.Context when creating cli or do not cancel this Context or it's parent context. 179 // nolint:unparam 180 func revokeLease(cli *clientv3.Client, id clientv3.LeaseID) (*clientv3.LeaseRevokeResponse, error) { 181 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRevokeLeaseTimeout) 182 defer cancel() 183 resp, err := cli.Revoke(ctx, id) 184 return resp, terror.ErrHAFailLeaseOperation.Delegate(err, "failed to revoke lease") 185 } 186 187 // WatchWorkerEvent watches the online and offline of workers from etcd. 188 // this function will output the worker event to evCh, output the error to errCh. 189 func WatchWorkerEvent(ctx context.Context, cli *clientv3.Client, rev int64, outCh chan<- WorkerEvent, errCh chan<- error) { 190 watcher := clientv3.NewWatcher(cli) 191 wCtx, cancel := context.WithCancel(ctx) 192 defer cancel() 193 ch := watcher.Watch(wCtx, common.WorkerKeepAliveKeyAdapter.Path(), clientv3.WithPrefix(), clientv3.WithRev(rev)) 194 195 for { 196 select { 197 case <-ctx.Done(): 198 log.L().Info("watch keepalive worker quit due to context canceled") 199 return 200 case resp, ok := <-ch: 201 if !ok { 202 return 203 } 204 if resp.Canceled { 205 select { 206 case errCh <- terror.ErrHAFailWatchEtcd.Delegate(resp.Err(), "watch worker event canceled"): 207 case <-ctx.Done(): 208 } 209 return 210 } 211 212 for _, ev := range resp.Events { 213 log.L().Info("receive dm-worker keep alive event", zap.String("operation", ev.Type.String()), zap.String("kv", string(ev.Kv.Key))) 214 var ( 215 event WorkerEvent 216 err error 217 ) 218 switch ev.Type { 219 case mvccpb.PUT: 220 event, err = workerEventFromJSON(string(ev.Kv.Value)) 221 case mvccpb.DELETE: 222 event, err = workerEventFromKey(string(ev.Kv.Key)) 223 event.IsDeleted = true 224 default: 225 // this should not happen. 226 log.L().Error("unsupported etcd event type", zap.Reflect("kv", ev.Kv), zap.Reflect("type", ev.Type)) 227 continue 228 } 229 if err != nil { 230 select { 231 case errCh <- err: 232 case <-ctx.Done(): 233 return 234 } 235 } else { 236 select { 237 case outCh <- event: 238 case <-ctx.Done(): 239 return 240 } 241 } 242 } 243 } 244 } 245 } 246 247 // GetKeepAliveWorkers gets current alive workers, 248 // and returns a map{workerName: WorkerEvent}, revision and error. 249 func GetKeepAliveWorkers(cli *clientv3.Client) (map[string]WorkerEvent, int64, error) { 250 ctx, cancel := context.WithTimeout(cli.Ctx(), etcdutil.DefaultRequestTimeout) 251 defer cancel() 252 253 var wwm map[string]WorkerEvent 254 resp, err := cli.Get(ctx, common.WorkerKeepAliveKeyAdapter.Path(), clientv3.WithPrefix()) 255 if err != nil { 256 return wwm, 0, terror.ErrHAFailTxnOperation.Delegate(err, "failed to get keepalive workers") 257 } 258 259 wwm = make(map[string]WorkerEvent, len(resp.Kvs)) 260 for _, kv := range resp.Kvs { 261 w, err := workerEventFromJSON(string(kv.Value)) 262 if err != nil { 263 return wwm, 0, err 264 } 265 wwm[w.WorkerName] = w 266 } 267 return wwm, resp.Header.Revision, nil 268 }