github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/pkg/orchestrator/etcd_worker.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 orchestrator 15 16 import ( 17 "context" 18 "fmt" 19 "strconv" 20 "time" 21 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 cerrors "github.com/pingcap/ticdc/pkg/errors" 25 "github.com/pingcap/ticdc/pkg/etcd" 26 "github.com/pingcap/ticdc/pkg/orchestrator/util" 27 "go.etcd.io/etcd/clientv3" 28 "go.etcd.io/etcd/clientv3/concurrency" 29 "go.etcd.io/etcd/mvcc/mvccpb" 30 "go.uber.org/zap" 31 "go.uber.org/zap/zapcore" 32 ) 33 34 // EtcdWorker handles all interactions with Etcd 35 type EtcdWorker struct { 36 client *etcd.Client 37 reactor Reactor 38 state ReactorState 39 // rawState is the local cache of the latest Etcd state. 40 rawState map[util.EtcdKey]rawStateEntry 41 // pendingUpdates stores Etcd updates that the Reactor has not been notified of. 42 pendingUpdates []*etcdUpdate 43 // revision is the Etcd revision of the latest event received from Etcd 44 // (which has not necessarily been applied to the ReactorState) 45 revision int64 46 // reactor.Tick() should not be called until revision >= barrierRev. 47 barrierRev int64 48 // prefix is the scope of Etcd watch 49 prefix util.EtcdPrefix 50 // deleteCounter maintains a local copy of a value stored in Etcd used to 51 // keep track of how many deletes have been committed by an EtcdWorker 52 // watching this key prefix. 53 // This mechanism is necessary as a workaround to correctly detect 54 // write-write conflicts when at least a transaction commits a delete, 55 // because deletes in Etcd reset the mod-revision of keys, making it 56 // difficult to use it as a unique version identifier to implement 57 // a `compare-and-swap` semantics, which is essential for implementing 58 // snapshot isolation for Reactor ticks. 59 deleteCounter int64 60 } 61 62 type etcdUpdate struct { 63 key util.EtcdKey 64 value []byte 65 revision int64 66 } 67 68 // rawStateEntry stores the latest version of a key as seen by the EtcdWorker. 69 // modRevision is stored to implement `compare-and-swap` semantics more reliably. 70 type rawStateEntry struct { 71 value []byte 72 modRevision int64 73 } 74 75 // NewEtcdWorker returns a new EtcdWorker 76 func NewEtcdWorker(client *etcd.Client, prefix string, reactor Reactor, initState ReactorState) (*EtcdWorker, error) { 77 return &EtcdWorker{ 78 client: client, 79 reactor: reactor, 80 state: initState, 81 rawState: make(map[util.EtcdKey]rawStateEntry), 82 prefix: util.NormalizePrefix(prefix), 83 barrierRev: -1, // -1 indicates no barrier 84 }, nil 85 } 86 87 const ( 88 etcdRequestProgressDuration = 2 * time.Second 89 deletionCounterKey = "/meta/ticdc-delete-etcd-key-count" 90 ) 91 92 // Run starts the EtcdWorker event loop. 93 // A tick is generated either on a timer whose interval is timerInterval, or on an Etcd event. 94 // If the specified etcd session is Done, this Run function will exit with cerrors.ErrEtcdSessionDone. 95 // And the specified etcd session is nil-safety. 96 func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, timerInterval time.Duration) error { 97 defer worker.cleanUp() 98 99 err := worker.syncRawState(ctx) 100 if err != nil { 101 return errors.Trace(err) 102 } 103 104 ctx1, cancel := context.WithCancel(ctx) 105 defer cancel() 106 107 ticker := time.NewTicker(timerInterval) 108 defer ticker.Stop() 109 110 watchCh := worker.client.Watch(ctx1, worker.prefix.String(), clientv3.WithPrefix(), clientv3.WithRev(worker.revision+1)) 111 var ( 112 pendingPatches [][]DataPatch 113 exiting bool 114 sessionDone <-chan struct{} 115 ) 116 if session != nil { 117 sessionDone = session.Done() 118 } else { 119 // should never be closed 120 sessionDone = make(chan struct{}) 121 } 122 lastReceivedEventTime := time.Now() 123 124 for { 125 var response clientv3.WatchResponse 126 select { 127 case <-ctx.Done(): 128 return ctx.Err() 129 case <-sessionDone: 130 return cerrors.ErrEtcdSessionDone.GenWithStackByArgs() 131 case <-ticker.C: 132 // There is no new event to handle on timer ticks, so we have nothing here. 133 if time.Since(lastReceivedEventTime) > etcdRequestProgressDuration { 134 if err := worker.client.RequestProgress(ctx); err != nil { 135 log.Warn("failed to request progress for etcd watcher", zap.Error(err)) 136 } 137 } 138 case response = <-watchCh: 139 // In this select case, we receive new events from Etcd, and call handleEvent if appropriate. 140 141 if err := response.Err(); err != nil { 142 return errors.Trace(err) 143 } 144 lastReceivedEventTime = time.Now() 145 146 // Check whether the response is stale. 147 if worker.revision >= response.Header.GetRevision() { 148 continue 149 } 150 worker.revision = response.Header.GetRevision() 151 152 // ProgressNotify implies no new events. 153 if response.IsProgressNotify() { 154 continue 155 } 156 157 for _, event := range response.Events { 158 // handleEvent will apply the event to our internal `rawState`. 159 worker.handleEvent(ctx, event) 160 } 161 } 162 163 if len(pendingPatches) > 0 { 164 // Here we have some patches yet to be uploaded to Etcd. 165 pendingPatches, err = worker.applyPatchGroups(ctx, pendingPatches) 166 if err != nil { 167 if cerrors.ErrEtcdTryAgain.Equal(errors.Cause(err)) { 168 continue 169 } 170 return errors.Trace(err) 171 } 172 } else { 173 if exiting { 174 // If exiting is true here, it means that the reactor returned `ErrReactorFinished` last tick, and all pending patches is applied. 175 return nil 176 } 177 if worker.revision < worker.barrierRev { 178 // We hold off notifying the Reactor because barrierRev has not been reached. 179 // This usually happens when a committed write Txn has not been received by Watch. 180 continue 181 } 182 183 // We are safe to update the ReactorState only if there is no pending patch. 184 if err := worker.applyUpdates(); err != nil { 185 return errors.Trace(err) 186 } 187 nextState, err := worker.reactor.Tick(ctx, worker.state) 188 if err != nil { 189 if !cerrors.ErrReactorFinished.Equal(errors.Cause(err)) { 190 return errors.Trace(err) 191 } 192 // normal exit 193 exiting = true 194 } 195 worker.state = nextState 196 pendingPatches = append(pendingPatches, nextState.GetPatches()...) 197 } 198 } 199 } 200 201 func (worker *EtcdWorker) handleEvent(_ context.Context, event *clientv3.Event) { 202 if worker.isDeleteCounterKey(event.Kv.Key) { 203 switch event.Type { 204 case mvccpb.PUT: 205 worker.handleDeleteCounter(event.Kv.Value) 206 case mvccpb.DELETE: 207 log.Warn("deletion counter key deleted", zap.Reflect("event", event)) 208 worker.handleDeleteCounter(nil) 209 } 210 // We return here because the delete-counter is not used for business logic, 211 // and it should not be exposed further to the Reactor. 212 return 213 } 214 215 worker.pendingUpdates = append(worker.pendingUpdates, &etcdUpdate{ 216 key: util.NewEtcdKeyFromBytes(event.Kv.Key), 217 value: event.Kv.Value, 218 revision: event.Kv.ModRevision, 219 }) 220 221 switch event.Type { 222 case mvccpb.PUT: 223 value := event.Kv.Value 224 if value == nil { 225 value = []byte{} 226 } 227 worker.rawState[util.NewEtcdKeyFromBytes(event.Kv.Key)] = rawStateEntry{ 228 value: value, 229 modRevision: event.Kv.ModRevision, 230 } 231 case mvccpb.DELETE: 232 delete(worker.rawState, util.NewEtcdKeyFromBytes(event.Kv.Key)) 233 } 234 } 235 236 func (worker *EtcdWorker) syncRawState(ctx context.Context) error { 237 resp, err := worker.client.Get(ctx, worker.prefix.String(), clientv3.WithPrefix()) 238 if err != nil { 239 return errors.Trace(err) 240 } 241 242 worker.rawState = make(map[util.EtcdKey]rawStateEntry) 243 for _, kv := range resp.Kvs { 244 if worker.isDeleteCounterKey(kv.Key) { 245 worker.handleDeleteCounter(kv.Value) 246 continue 247 } 248 key := util.NewEtcdKeyFromBytes(kv.Key) 249 worker.rawState[key] = rawStateEntry{ 250 value: kv.Value, 251 modRevision: kv.ModRevision, 252 } 253 err := worker.state.Update(key, kv.Value, true) 254 if err != nil { 255 return errors.Trace(err) 256 } 257 } 258 259 worker.revision = resp.Header.Revision 260 return nil 261 } 262 263 func (worker *EtcdWorker) cloneRawState() map[util.EtcdKey][]byte { 264 ret := make(map[util.EtcdKey][]byte) 265 for k, v := range worker.rawState { 266 vCloned := make([]byte, len(v.value)) 267 copy(vCloned, v.value) 268 ret[util.NewEtcdKey(k.String())] = vCloned 269 } 270 return ret 271 } 272 273 func (worker *EtcdWorker) applyPatchGroups(ctx context.Context, patchGroups [][]DataPatch) ([][]DataPatch, error) { 274 for len(patchGroups) > 0 { 275 patches := patchGroups[0] 276 err := worker.applyPatches(ctx, patches) 277 if err != nil { 278 return patchGroups, err 279 } 280 patchGroups = patchGroups[1:] 281 } 282 return patchGroups, nil 283 } 284 285 func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) error { 286 state := worker.cloneRawState() 287 changedSet := make(map[util.EtcdKey]struct{}) 288 for _, patch := range patches { 289 err := patch.Patch(state, changedSet) 290 if err != nil { 291 if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { 292 continue 293 } 294 return errors.Trace(err) 295 } 296 } 297 cmps := make([]clientv3.Cmp, 0, len(changedSet)) 298 ops := make([]clientv3.Op, 0, len(changedSet)) 299 hasDelete := false 300 for key := range changedSet { 301 // make sure someone else has not updated the key after the last snapshot 302 var cmp clientv3.Cmp 303 if entry, ok := worker.rawState[key]; ok { 304 cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "=", entry.modRevision) 305 } else { 306 // if ok is false, it means that the key of this patch is not exist in a committed state 307 // this compare is equivalent to `patch.Key` is not exist 308 cmp = clientv3.Compare(clientv3.ModRevision(key.String()), "=", 0) 309 } 310 cmps = append(cmps, cmp) 311 312 value := state[key] 313 var op clientv3.Op 314 if value != nil { 315 op = clientv3.OpPut(key.String(), string(value)) 316 } else { 317 op = clientv3.OpDelete(key.String()) 318 hasDelete = true 319 } 320 ops = append(ops, op) 321 } 322 323 if hasDelete { 324 ops = append(ops, clientv3.OpPut(worker.prefix.String()+deletionCounterKey, fmt.Sprint(worker.deleteCounter+1))) 325 } 326 if worker.deleteCounter > 0 { 327 cmps = append(cmps, clientv3.Compare(clientv3.Value(worker.prefix.String()+deletionCounterKey), "=", fmt.Sprint(worker.deleteCounter))) 328 } else if worker.deleteCounter == 0 { 329 cmps = append(cmps, clientv3.Compare(clientv3.CreateRevision(worker.prefix.String()+deletionCounterKey), "=", 0)) 330 } else { 331 panic("unreachable") 332 } 333 334 resp, err := worker.client.Txn(ctx).If(cmps...).Then(ops...).Commit() 335 if err != nil { 336 return errors.Trace(err) 337 } 338 339 logEtcdOps(ops, resp.Succeeded) 340 if resp.Succeeded { 341 worker.barrierRev = resp.Header.GetRevision() 342 return nil 343 } 344 345 return cerrors.ErrEtcdTryAgain.GenWithStackByArgs() 346 } 347 348 func (worker *EtcdWorker) applyUpdates() error { 349 for _, update := range worker.pendingUpdates { 350 err := worker.state.Update(update.key, update.value, false) 351 if err != nil { 352 return errors.Trace(err) 353 } 354 } 355 356 worker.pendingUpdates = worker.pendingUpdates[:0] 357 return nil 358 } 359 360 func logEtcdOps(ops []clientv3.Op, commited bool) { 361 if log.GetLevel() != zapcore.DebugLevel || len(ops) == 0 { 362 return 363 } 364 log.Debug("[etcd worker] ==========Update State to ETCD==========") 365 for _, op := range ops { 366 if op.IsDelete() { 367 log.Debug("[etcd worker] delete key", zap.ByteString("key", op.KeyBytes())) 368 } else { 369 log.Debug("[etcd worker] put key", zap.ByteString("key", op.KeyBytes()), zap.ByteString("value", op.ValueBytes())) 370 } 371 } 372 log.Debug("[etcd worker] ============State Commit=============", zap.Bool("committed", commited)) 373 } 374 375 func (worker *EtcdWorker) cleanUp() { 376 worker.rawState = nil 377 worker.revision = 0 378 worker.pendingUpdates = worker.pendingUpdates[:0] 379 } 380 381 func (worker *EtcdWorker) isDeleteCounterKey(key []byte) bool { 382 return string(key) == worker.prefix.String()+deletionCounterKey 383 } 384 385 func (worker *EtcdWorker) handleDeleteCounter(value []byte) { 386 if len(value) == 0 { 387 // The delete counter key has been deleted, resetting the internal counter 388 worker.deleteCounter = 0 389 return 390 } 391 392 var err error 393 worker.deleteCounter, err = strconv.ParseInt(string(value), 10, 64) 394 if err != nil { 395 // This should never happen unless Etcd server has been tampered with. 396 log.Panic("strconv failed. Unexpected Etcd state.", zap.Error(err)) 397 } 398 if worker.deleteCounter <= 0 { 399 log.Panic("unexpected delete counter", zap.Int64("value", worker.deleteCounter)) 400 } 401 }