github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/model/reactor_state.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 model 15 16 import ( 17 "encoding/json" 18 "reflect" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/log" 22 cerrors "github.com/pingcap/ticdc/pkg/errors" 23 "github.com/pingcap/ticdc/pkg/etcd" 24 "github.com/pingcap/ticdc/pkg/orchestrator" 25 "github.com/pingcap/ticdc/pkg/orchestrator/util" 26 "go.uber.org/zap" 27 ) 28 29 // GlobalReactorState represents a global state which stores all key-value pairs in ETCD 30 type GlobalReactorState struct { 31 Owner map[string]struct{} 32 Captures map[CaptureID]*CaptureInfo 33 Changefeeds map[ChangeFeedID]*ChangefeedReactorState 34 pendingPatches [][]orchestrator.DataPatch 35 } 36 37 // NewGlobalState creates a new global state 38 func NewGlobalState() orchestrator.ReactorState { 39 return &GlobalReactorState{ 40 Owner: map[string]struct{}{}, 41 Captures: make(map[CaptureID]*CaptureInfo), 42 Changefeeds: make(map[ChangeFeedID]*ChangefeedReactorState), 43 } 44 } 45 46 // Update implements the ReactorState interface 47 func (s *GlobalReactorState) Update(key util.EtcdKey, value []byte, _ bool) error { 48 k := new(etcd.CDCKey) 49 err := k.Parse(key.String()) 50 if err != nil { 51 return errors.Trace(err) 52 } 53 switch k.Tp { 54 case etcd.CDCKeyTypeOwner: 55 if value != nil { 56 s.Owner[k.OwnerLeaseID] = struct{}{} 57 } else { 58 delete(s.Owner, k.OwnerLeaseID) 59 } 60 return nil 61 case etcd.CDCKeyTypeCapture: 62 if value == nil { 63 log.Info("remote capture offline", zap.String("capture-id", k.CaptureID)) 64 delete(s.Captures, k.CaptureID) 65 return nil 66 } 67 68 var newCaptureInfo CaptureInfo 69 err := newCaptureInfo.Unmarshal(value) 70 if err != nil { 71 return cerrors.ErrUnmarshalFailed.Wrap(err).GenWithStackByArgs() 72 } 73 74 log.Info("remote capture online", zap.String("capture-id", k.CaptureID), zap.Any("info", newCaptureInfo)) 75 s.Captures[k.CaptureID] = &newCaptureInfo 76 case etcd.CDCKeyTypeChangefeedInfo, 77 etcd.CDCKeyTypeChangeFeedStatus, 78 etcd.CDCKeyTypeTaskPosition, 79 etcd.CDCKeyTypeTaskStatus, 80 etcd.CDCKeyTypeTaskWorkload: 81 changefeedState, exist := s.Changefeeds[k.ChangefeedID] 82 if !exist { 83 if value == nil { 84 return nil 85 } 86 changefeedState = NewChangefeedReactorState(k.ChangefeedID) 87 s.Changefeeds[k.ChangefeedID] = changefeedState 88 } 89 if err := changefeedState.UpdateCDCKey(k, value); err != nil { 90 return errors.Trace(err) 91 } 92 if value == nil && !changefeedState.Exist() { 93 s.pendingPatches = append(s.pendingPatches, changefeedState.getPatches()) 94 delete(s.Changefeeds, k.ChangefeedID) 95 } 96 default: 97 log.Warn("receive an unexpected etcd event", zap.String("key", key.String()), zap.ByteString("value", value)) 98 } 99 return nil 100 } 101 102 // GetPatches implements the ReactorState interface 103 func (s *GlobalReactorState) GetPatches() [][]orchestrator.DataPatch { 104 pendingPatches := s.pendingPatches 105 for _, changefeedState := range s.Changefeeds { 106 pendingPatches = append(pendingPatches, changefeedState.getPatches()) 107 } 108 s.pendingPatches = nil 109 return pendingPatches 110 } 111 112 // ChangefeedReactorState represents a changefeed state which stores all key-value pairs of a changefeed in ETCD 113 type ChangefeedReactorState struct { 114 ID ChangeFeedID 115 Info *ChangeFeedInfo 116 Status *ChangeFeedStatus 117 TaskPositions map[CaptureID]*TaskPosition 118 TaskStatuses map[CaptureID]*TaskStatus 119 Workloads map[CaptureID]TaskWorkload 120 121 pendingPatches []orchestrator.DataPatch 122 skipPatchesInThisTick bool 123 } 124 125 // NewChangefeedReactorState creates a new changefeed reactor state 126 func NewChangefeedReactorState(id ChangeFeedID) *ChangefeedReactorState { 127 return &ChangefeedReactorState{ 128 ID: id, 129 TaskPositions: make(map[CaptureID]*TaskPosition), 130 TaskStatuses: make(map[CaptureID]*TaskStatus), 131 Workloads: make(map[CaptureID]TaskWorkload), 132 } 133 } 134 135 // Update implements the ReactorState interface 136 func (s *ChangefeedReactorState) Update(key util.EtcdKey, value []byte, _ bool) error { 137 k := new(etcd.CDCKey) 138 if err := k.Parse(key.String()); err != nil { 139 return errors.Trace(err) 140 } 141 if err := s.UpdateCDCKey(k, value); err != nil { 142 log.Error("failed to update status", zap.String("key", key.String()), zap.ByteString("value", value)) 143 return errors.Trace(err) 144 } 145 return nil 146 } 147 148 // UpdateCDCKey updates the state by a parsed etcd key 149 func (s *ChangefeedReactorState) UpdateCDCKey(key *etcd.CDCKey, value []byte) error { 150 var e interface{} 151 switch key.Tp { 152 case etcd.CDCKeyTypeChangefeedInfo: 153 if key.ChangefeedID != s.ID { 154 return nil 155 } 156 if value == nil { 157 s.Info = nil 158 return nil 159 } 160 s.Info = new(ChangeFeedInfo) 161 e = s.Info 162 case etcd.CDCKeyTypeChangeFeedStatus: 163 if key.ChangefeedID != s.ID { 164 return nil 165 } 166 if value == nil { 167 s.Status = nil 168 return nil 169 } 170 s.Status = new(ChangeFeedStatus) 171 e = s.Status 172 case etcd.CDCKeyTypeTaskPosition: 173 if key.ChangefeedID != s.ID { 174 return nil 175 } 176 if value == nil { 177 delete(s.TaskPositions, key.CaptureID) 178 return nil 179 } 180 position := new(TaskPosition) 181 s.TaskPositions[key.CaptureID] = position 182 e = position 183 case etcd.CDCKeyTypeTaskStatus: 184 if key.ChangefeedID != s.ID { 185 return nil 186 } 187 if value == nil { 188 delete(s.TaskStatuses, key.CaptureID) 189 return nil 190 } 191 status := new(TaskStatus) 192 s.TaskStatuses[key.CaptureID] = status 193 e = status 194 case etcd.CDCKeyTypeTaskWorkload: 195 if key.ChangefeedID != s.ID { 196 return nil 197 } 198 if value == nil { 199 delete(s.Workloads, key.CaptureID) 200 return nil 201 } 202 workload := make(TaskWorkload) 203 s.Workloads[key.CaptureID] = workload 204 e = &workload 205 default: 206 return nil 207 } 208 if err := json.Unmarshal(value, e); err != nil { 209 return errors.Trace(err) 210 } 211 if key.Tp == etcd.CDCKeyTypeChangefeedInfo { 212 if err := s.Info.VerifyAndFix(); err != nil { 213 return errors.Trace(err) 214 } 215 } 216 return nil 217 } 218 219 // Exist returns false if all keys of this changefeed in ETCD is not exist 220 func (s *ChangefeedReactorState) Exist() bool { 221 return s.Info != nil || s.Status != nil || len(s.TaskPositions) != 0 || len(s.TaskStatuses) != 0 || len(s.Workloads) != 0 222 } 223 224 // Active return true if the changefeed is ready to be processed 225 func (s *ChangefeedReactorState) Active(captureID CaptureID) bool { 226 return s.Info != nil && s.Status != nil && s.TaskStatuses[captureID] != nil 227 } 228 229 // GetPatches implements the ReactorState interface 230 func (s *ChangefeedReactorState) GetPatches() [][]orchestrator.DataPatch { 231 return [][]orchestrator.DataPatch{s.getPatches()} 232 } 233 234 func (s *ChangefeedReactorState) getPatches() []orchestrator.DataPatch { 235 pendingPatches := s.pendingPatches 236 s.pendingPatches = nil 237 return pendingPatches 238 } 239 240 // CheckCaptureAlive checks if the capture is alive, if the capture offline, 241 // the etcd worker will exit and throw the ErrLeaseExpired error. 242 func (s *ChangefeedReactorState) CheckCaptureAlive(captureID CaptureID) { 243 k := etcd.CDCKey{ 244 Tp: etcd.CDCKeyTypeCapture, 245 CaptureID: captureID, 246 } 247 key := k.String() 248 patch := &orchestrator.SingleDataPatch{ 249 Key: util.NewEtcdKey(key), 250 Func: func(v []byte) ([]byte, bool, error) { 251 // If v is empty, it means that the key-value pair of capture info is not exist. 252 // The key-value pair of capture info is written with lease, 253 // so if the capture info is not exist, the lease is expired 254 if len(v) == 0 { 255 return v, false, cerrors.ErrLeaseExpired.GenWithStackByArgs() 256 } 257 return v, false, nil 258 }, 259 } 260 s.pendingPatches = append(s.pendingPatches, patch) 261 } 262 263 // CheckChangefeedNormal checks if the changefeed state is runable, 264 // if the changefeed status is not runable, the etcd worker will skip all patch of this tick 265 // the processor should call this function every tick to make sure the changefeed is runable 266 func (s *ChangefeedReactorState) CheckChangefeedNormal() { 267 s.skipPatchesInThisTick = false 268 s.PatchInfo(func(info *ChangeFeedInfo) (*ChangeFeedInfo, bool, error) { 269 if info == nil || info.AdminJobType.IsStopState() { 270 s.skipPatchesInThisTick = true 271 return info, false, cerrors.ErrEtcdTryAgain.GenWithStackByArgs() 272 } 273 return info, false, nil 274 }) 275 s.PatchStatus(func(status *ChangeFeedStatus) (*ChangeFeedStatus, bool, error) { 276 if status == nil { 277 return status, false, nil 278 } 279 if status.AdminJobType.IsStopState() { 280 s.skipPatchesInThisTick = true 281 return status, false, cerrors.ErrEtcdTryAgain.GenWithStackByArgs() 282 } 283 return status, false, nil 284 }) 285 } 286 287 // PatchInfo appends a DataPatch which can modify the ChangeFeedInfo 288 func (s *ChangefeedReactorState) PatchInfo(fn func(*ChangeFeedInfo) (*ChangeFeedInfo, bool, error)) { 289 key := &etcd.CDCKey{ 290 Tp: etcd.CDCKeyTypeChangefeedInfo, 291 ChangefeedID: s.ID, 292 } 293 s.patchAny(key.String(), changefeedInfoTPI, func(e interface{}) (interface{}, bool, error) { 294 // e == nil means that the key is not exist before this patch 295 if e == nil { 296 return fn(nil) 297 } 298 return fn(e.(*ChangeFeedInfo)) 299 }) 300 } 301 302 // PatchStatus appends a DataPatch which can modify the ChangeFeedStatus 303 func (s *ChangefeedReactorState) PatchStatus(fn func(*ChangeFeedStatus) (*ChangeFeedStatus, bool, error)) { 304 key := &etcd.CDCKey{ 305 Tp: etcd.CDCKeyTypeChangeFeedStatus, 306 ChangefeedID: s.ID, 307 } 308 s.patchAny(key.String(), changefeedStatusTPI, func(e interface{}) (interface{}, bool, error) { 309 // e == nil means that the key is not exist before this patch 310 if e == nil { 311 return fn(nil) 312 } 313 return fn(e.(*ChangeFeedStatus)) 314 }) 315 } 316 317 // PatchTaskPosition appends a DataPatch which can modify the TaskPosition of a specified capture 318 func (s *ChangefeedReactorState) PatchTaskPosition(captureID CaptureID, fn func(*TaskPosition) (*TaskPosition, bool, error)) { 319 key := &etcd.CDCKey{ 320 Tp: etcd.CDCKeyTypeTaskPosition, 321 CaptureID: captureID, 322 ChangefeedID: s.ID, 323 } 324 s.patchAny(key.String(), taskPositionTPI, func(e interface{}) (interface{}, bool, error) { 325 // e == nil means that the key is not exist before this patch 326 if e == nil { 327 return fn(nil) 328 } 329 return fn(e.(*TaskPosition)) 330 }) 331 } 332 333 // PatchTaskStatus appends a DataPatch which can modify the TaskStatus of a specified capture 334 func (s *ChangefeedReactorState) PatchTaskStatus(captureID CaptureID, fn func(*TaskStatus) (*TaskStatus, bool, error)) { 335 key := &etcd.CDCKey{ 336 Tp: etcd.CDCKeyTypeTaskStatus, 337 CaptureID: captureID, 338 ChangefeedID: s.ID, 339 } 340 s.patchAny(key.String(), taskStatusTPI, func(e interface{}) (interface{}, bool, error) { 341 // e == nil means that the key is not exist before this patch 342 if e == nil { 343 return fn(nil) 344 } 345 return fn(e.(*TaskStatus)) 346 }) 347 } 348 349 // PatchTaskWorkload appends a DataPatch which can modify the TaskWorkload of a specified capture 350 func (s *ChangefeedReactorState) PatchTaskWorkload(captureID CaptureID, fn func(TaskWorkload) (TaskWorkload, bool, error)) { 351 key := &etcd.CDCKey{ 352 Tp: etcd.CDCKeyTypeTaskWorkload, 353 CaptureID: captureID, 354 ChangefeedID: s.ID, 355 } 356 s.patchAny(key.String(), taskWorkloadTPI, func(e interface{}) (interface{}, bool, error) { 357 // e == nil means that the key is not exist before this patch 358 if e == nil { 359 return fn(nil) 360 } 361 return fn(*e.(*TaskWorkload)) 362 }) 363 } 364 365 var ( 366 taskPositionTPI *TaskPosition 367 taskStatusTPI *TaskStatus 368 taskWorkloadTPI *TaskWorkload 369 changefeedStatusTPI *ChangeFeedStatus 370 changefeedInfoTPI *ChangeFeedInfo 371 ) 372 373 func (s *ChangefeedReactorState) patchAny(key string, tpi interface{}, fn func(interface{}) (interface{}, bool, error)) { 374 patch := &orchestrator.SingleDataPatch{ 375 Key: util.NewEtcdKey(key), 376 Func: func(v []byte) ([]byte, bool, error) { 377 if s.skipPatchesInThisTick { 378 return v, false, cerrors.ErrEtcdIgnore.GenWithStackByArgs() 379 } 380 var e interface{} 381 if v != nil { 382 tp := reflect.TypeOf(tpi) 383 e = reflect.New(tp.Elem()).Interface() 384 err := json.Unmarshal(v, e) 385 if err != nil { 386 return nil, false, errors.Trace(err) 387 } 388 } 389 ne, changed, err := fn(e) 390 if err != nil { 391 return nil, false, errors.Trace(err) 392 } 393 if !changed { 394 return v, false, nil 395 } 396 if reflect.ValueOf(ne).IsNil() { 397 return nil, true, nil 398 } 399 nv, err := json.Marshal(ne) 400 if err != nil { 401 return nil, false, errors.Trace(err) 402 } 403 return nv, true, nil 404 }, 405 } 406 s.pendingPatches = append(s.pendingPatches, patch) 407 }