github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/owner/feed_state_manager.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 owner 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/cenkalti/backoff/v4" 21 "github.com/pingcap/errors" 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/cdc/model" 24 cerrors "github.com/pingcap/tiflow/pkg/errors" 25 "github.com/pingcap/tiflow/pkg/upstream" 26 "github.com/pingcap/tiflow/pkg/util" 27 "github.com/tikv/client-go/v2/oracle" 28 pd "github.com/tikv/pd/client" 29 "go.uber.org/zap" 30 ) 31 32 const ( 33 // When errors occurred, and we need to do backoff, we start an exponential backoff 34 // with an interval from 10s to 30min (10s, 20s, 40s, 80s, 160s, 320s, 35 // 600s, 600s, ...). 36 // To avoid thunderherd, a random factor is also added. 37 defaultBackoffInitInterval = 10 * time.Second 38 defaultBackoffMaxInterval = 10 * time.Minute 39 defaultBackoffRandomizationFactor = 0.1 40 defaultBackoffMultiplier = 2.0 41 ) 42 43 // FeedStateManager manages the life cycle of a changefeed, currently it is responsible for: 44 // 1. Handle admin jobs 45 // 2. Handle errors 46 // 3. Handle warnings 47 // 4. Control the status of a changefeed 48 type FeedStateManager interface { 49 // PushAdminJob pushed an admin job to the admin job queue 50 PushAdminJob(job *model.AdminJob) 51 // Tick is the main logic of the FeedStateManager, it will be called periodically 52 // resolvedTs is the resolvedTs of the changefeed 53 // returns true if there is a pending admin job, if so changefeed should not run the tick logic 54 Tick(resolvedTs model.Ts, status *model.ChangeFeedStatus, info *model.ChangeFeedInfo) (adminJobPending bool) 55 // HandleError is called an error occurs in Changefeed.Tick 56 HandleError(errs ...*model.RunningError) 57 // HandleWarning is called a warning occurs in Changefeed.Tick 58 HandleWarning(warnings ...*model.RunningError) 59 // ShouldRunning returns if the changefeed should be running 60 ShouldRunning() bool 61 // ShouldRemoved returns if the changefeed should be removed 62 ShouldRemoved() bool 63 // MarkFinished is call when a changefeed is finished 64 MarkFinished() 65 } 66 67 // feedStateManager manages the ReactorState of a changefeed 68 // when an error or an admin job occurs, the feedStateManager is responsible for controlling the ReactorState 69 type feedStateManager struct { 70 upstream *upstream.Upstream 71 state ChangefeedState 72 73 shouldBeRunning bool 74 // Based on shouldBeRunning = false 75 // shouldBeRemoved = true means the changefeed is removed 76 // shouldBeRemoved = false means the changefeed is paused 77 shouldBeRemoved bool 78 79 adminJobQueue []*model.AdminJob 80 isRetrying bool 81 lastErrorRetryTime time.Time // time of last error for a changefeed 82 lastErrorRetryCheckpointTs model.Ts // checkpoint ts of last retry 83 lastWarningReportCheckpointTs model.Ts // checkpoint ts of last warning report 84 backoffInterval time.Duration // the interval for restarting a changefeed in 'error' state 85 errBackoff *backoff.ExponentialBackOff // an exponential backoff for restarting a changefeed 86 87 // resolvedTs and initCheckpointTs is for checking whether resolved timestamp 88 // has been advanced or not. 89 resolvedTs model.Ts 90 checkpointTs model.Ts 91 checkpointTsAdvanced time.Time 92 93 changefeedErrorStuckDuration time.Duration 94 } 95 96 // NewFeedStateManager creates feedStateManager and initialize the exponential backoff 97 func NewFeedStateManager(up *upstream.Upstream, 98 state ChangefeedState, 99 ) FeedStateManager { 100 m := new(feedStateManager) 101 m.upstream = up 102 m.state = state 103 104 m.errBackoff = backoff.NewExponentialBackOff() 105 m.errBackoff.InitialInterval = defaultBackoffInitInterval 106 m.errBackoff.MaxInterval = defaultBackoffMaxInterval 107 m.errBackoff.Multiplier = defaultBackoffMultiplier 108 m.errBackoff.RandomizationFactor = defaultBackoffRandomizationFactor 109 // backoff will stop once the defaultBackoffMaxElapsedTime has elapsed. 110 m.errBackoff.MaxElapsedTime = *state.GetChangefeedInfo().Config.ChangefeedErrorStuckDuration 111 m.changefeedErrorStuckDuration = *state.GetChangefeedInfo().Config.ChangefeedErrorStuckDuration 112 113 m.resetErrRetry() 114 m.isRetrying = false 115 return m 116 } 117 118 func (m *feedStateManager) shouldRetry() bool { 119 // changefeed should not retry within [m.lastErrorRetryTime, m.lastErrorRetryTime + m.backoffInterval). 120 return time.Since(m.lastErrorRetryTime) >= m.backoffInterval 121 } 122 123 func (m *feedStateManager) shouldFailWhenRetry() bool { 124 // retry the changefeed 125 m.backoffInterval = m.errBackoff.NextBackOff() 126 // NextBackOff() will return -1 once the MaxElapsedTime has elapsed, 127 // set the changefeed to failed state. 128 if m.backoffInterval == m.errBackoff.Stop { 129 return true 130 } 131 132 m.lastErrorRetryTime = time.Now() 133 return false 134 } 135 136 // resetErrRetry reset the error retry related fields 137 func (m *feedStateManager) resetErrRetry() { 138 m.errBackoff.Reset() 139 m.backoffInterval = m.errBackoff.NextBackOff() 140 m.lastErrorRetryTime = time.Unix(0, 0) 141 } 142 143 func (m *feedStateManager) Tick(resolvedTs model.Ts, 144 status *model.ChangeFeedStatus, info *model.ChangeFeedInfo, 145 ) (adminJobPending bool) { 146 m.checkAndInitLastRetryCheckpointTs(status) 147 148 if status != nil { 149 if m.checkpointTs < status.CheckpointTs { 150 m.checkpointTs = status.CheckpointTs 151 m.checkpointTsAdvanced = time.Now() 152 } 153 if m.resolvedTs < resolvedTs { 154 m.resolvedTs = resolvedTs 155 } 156 if m.checkpointTs >= m.resolvedTs { 157 m.checkpointTsAdvanced = time.Now() 158 } 159 } 160 161 m.shouldBeRunning = true 162 defer func() { 163 if !m.shouldBeRunning { 164 m.cleanUp() 165 } 166 }() 167 168 if m.handleAdminJob() { 169 // `handleAdminJob` returns true means that some admin jobs are pending 170 // skip to the next tick until all the admin jobs is handled 171 adminJobPending = true 172 changefeedErrorStuckDuration := util.GetOrZero(m.state.GetChangefeedInfo().Config.ChangefeedErrorStuckDuration) 173 if m.changefeedErrorStuckDuration != changefeedErrorStuckDuration { 174 log.Info("changefeedErrorStuckDuration update", 175 zap.Duration("oldChangefeedErrorStuckDuration", m.changefeedErrorStuckDuration), 176 zap.Duration("newChangefeedErrorStuckDuration", changefeedErrorStuckDuration), 177 ) 178 m.errBackoff.MaxElapsedTime = changefeedErrorStuckDuration 179 m.changefeedErrorStuckDuration = changefeedErrorStuckDuration 180 } 181 return 182 } 183 184 switch info.State { 185 case model.StateUnInitialized: 186 m.patchState(model.StateNormal) 187 return 188 case model.StateRemoved: 189 m.shouldBeRunning = false 190 m.shouldBeRemoved = true 191 return 192 case model.StateStopped, model.StateFailed, model.StateFinished: 193 m.shouldBeRunning = false 194 return 195 case model.StatePending: 196 if !m.shouldRetry() { 197 m.shouldBeRunning = false 198 return 199 } 200 201 if m.shouldFailWhenRetry() { 202 log.Error("The changefeed won't be restarted as it has been experiencing failures for "+ 203 "an extended duration", 204 zap.Duration("maxElapsedTime", m.errBackoff.MaxElapsedTime), 205 zap.String("namespace", m.state.GetID().Namespace), 206 zap.String("changefeed", m.state.GetID().ID), 207 zap.Time("lastRetryTime", m.lastErrorRetryTime), 208 zap.Uint64("lastRetryCheckpointTs", m.lastErrorRetryCheckpointTs), 209 ) 210 m.shouldBeRunning = false 211 m.patchState(model.StateFailed) 212 return 213 } 214 215 // retry the changefeed 216 m.shouldBeRunning = true 217 if status != nil { 218 m.lastErrorRetryCheckpointTs = m.state.GetChangefeedStatus().CheckpointTs 219 } 220 m.patchState(model.StateWarning) 221 log.Info("changefeed retry backoff interval is elapsed,"+ 222 "chengefeed will be restarted", 223 zap.String("namespace", m.state.GetID().Namespace), 224 zap.String("changefeed", m.state.GetID().ID), 225 zap.Time("lastErrorRetryTime", m.lastErrorRetryTime), 226 zap.Duration("nextRetryInterval", m.backoffInterval)) 227 case model.StateNormal, model.StateWarning: 228 m.checkAndChangeState() 229 errs := m.state.TakeProcessorErrors() 230 m.HandleError(errs...) 231 // only handle warnings when there are no errors 232 // otherwise, the warnings will cover the errors 233 if len(errs) == 0 { 234 // warning are come from processors' sink component 235 // they ere not fatal errors, so we don't need to stop the changefeed 236 warnings := m.state.TakeProcessorWarnings() 237 m.HandleWarning(warnings...) 238 } 239 } 240 return 241 } 242 243 func (m *feedStateManager) ShouldRunning() bool { 244 return m.shouldBeRunning 245 } 246 247 func (m *feedStateManager) ShouldRemoved() bool { 248 return m.shouldBeRemoved 249 } 250 251 func (m *feedStateManager) MarkFinished() { 252 if m.state == nil { 253 // when state is nil, it means that Tick has never been called 254 // skip this and wait for the next tick to finish the changefeed 255 return 256 } 257 m.pushAdminJob(&model.AdminJob{ 258 CfID: m.state.GetID(), 259 Type: model.AdminFinish, 260 }) 261 } 262 263 func (m *feedStateManager) PushAdminJob(job *model.AdminJob) { 264 switch job.Type { 265 case model.AdminStop, model.AdminResume, model.AdminRemove: 266 default: 267 log.Panic("Can not handle this job", 268 zap.String("namespace", m.state.GetID().Namespace), 269 zap.String("changefeed", m.state.GetID().ID), 270 zap.String("changefeedState", string(m.state.GetChangefeedInfo().State)), zap.Any("job", job)) 271 } 272 m.pushAdminJob(job) 273 } 274 275 func (m *feedStateManager) handleAdminJob() (jobsPending bool) { 276 job := m.popAdminJob() 277 if job == nil || job.CfID != m.state.GetID() { 278 return false 279 } 280 log.Info("handle admin job", 281 zap.String("namespace", m.state.GetID().Namespace), 282 zap.String("changefeed", m.state.GetID().ID), 283 zap.Any("job", job)) 284 switch job.Type { 285 case model.AdminStop: 286 switch m.state.GetChangefeedInfo().State { 287 case model.StateNormal, model.StateWarning, model.StatePending: 288 default: 289 log.Warn("can not pause the changefeed in the current state", 290 zap.String("namespace", m.state.GetID().Namespace), 291 zap.String("changefeed", m.state.GetID().ID), 292 zap.String("changefeedState", string(m.state.GetChangefeedInfo().State)), zap.Any("job", job)) 293 return 294 } 295 m.shouldBeRunning = false 296 jobsPending = true 297 m.patchState(model.StateStopped) 298 case model.AdminRemove: 299 m.shouldBeRunning = false 300 m.shouldBeRemoved = true 301 jobsPending = true 302 m.state.RemoveChangefeed() 303 checkpointTs := m.state.GetChangefeedInfo().GetCheckpointTs(m.state.GetChangefeedStatus()) 304 305 log.Info("the changefeed is removed", 306 zap.String("namespace", m.state.GetID().Namespace), 307 zap.String("changefeed", m.state.GetID().ID), 308 zap.Uint64("checkpointTs", checkpointTs)) 309 case model.AdminResume: 310 switch m.state.GetChangefeedInfo().State { 311 case model.StateFailed, model.StateStopped, model.StateFinished: 312 default: 313 log.Warn("can not resume the changefeed in the current state", 314 zap.String("namespace", m.state.GetID().Namespace), 315 zap.String("changefeed", m.state.GetID().ID), 316 zap.String("changefeedState", string(m.state.GetChangefeedInfo().State)), zap.Any("job", job)) 317 return 318 } 319 m.shouldBeRunning = true 320 // when the changefeed is manually resumed, we must reset the backoff 321 m.resetErrRetry() 322 m.isRetrying = false 323 jobsPending = true 324 m.patchState(model.StateNormal) 325 m.state.ResumeChnagefeed(job.OverwriteCheckpointTs) 326 327 case model.AdminFinish: 328 switch m.state.GetChangefeedInfo().State { 329 case model.StateNormal, model.StateWarning: 330 default: 331 log.Warn("can not finish the changefeed in the current state", 332 zap.String("namespace", m.state.GetID().Namespace), 333 zap.String("changefeed", m.state.GetID().ID), 334 zap.String("changefeedState", string(m.state.GetChangefeedInfo().State)), 335 zap.Any("job", job)) 336 return 337 } 338 m.shouldBeRunning = false 339 jobsPending = true 340 m.patchState(model.StateFinished) 341 default: 342 log.Warn("Unknown admin job", zap.Any("adminJob", job), 343 zap.String("namespace", m.state.GetID().Namespace), 344 zap.String("changefeed", m.state.GetID().ID), 345 ) 346 } 347 return 348 } 349 350 func (m *feedStateManager) popAdminJob() *model.AdminJob { 351 if len(m.adminJobQueue) == 0 { 352 return nil 353 } 354 job := m.adminJobQueue[0] 355 m.adminJobQueue = m.adminJobQueue[1:] 356 return job 357 } 358 359 func (m *feedStateManager) pushAdminJob(job *model.AdminJob) { 360 m.adminJobQueue = append(m.adminJobQueue, job) 361 } 362 363 func (m *feedStateManager) patchState(feedState model.FeedState) { 364 var updateEpoch bool 365 var adminJobType model.AdminJobType 366 switch feedState { 367 case model.StateNormal, model.StateWarning: 368 adminJobType = model.AdminNone 369 updateEpoch = false 370 case model.StateFinished: 371 adminJobType = model.AdminFinish 372 updateEpoch = true 373 case model.StatePending, model.StateStopped, model.StateFailed: 374 adminJobType = model.AdminStop 375 updateEpoch = true 376 case model.StateRemoved: 377 adminJobType = model.AdminRemove 378 updateEpoch = true 379 default: 380 log.Panic("Unreachable") 381 } 382 epoch := uint64(0) 383 if updateEpoch { 384 if updateEpoch { 385 ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second) 386 defer cancel() 387 epoch = GenerateChangefeedEpoch(ctx, m.upstream.PDClient) 388 } 389 } 390 m.state.UpdateChangefeedState(feedState, adminJobType, epoch) 391 } 392 393 func (m *feedStateManager) cleanUp() { 394 m.state.CleanUpTaskPositions() 395 m.checkpointTs = 0 396 m.checkpointTsAdvanced = time.Time{} 397 m.resolvedTs = 0 398 } 399 400 func (m *feedStateManager) HandleError(errs ...*model.RunningError) { 401 if len(errs) == 0 { 402 return 403 } 404 // if there are a fastFail error in errs, we can just fastFail the changefeed 405 // and no need to patch other error to the changefeed info 406 for _, err := range errs { 407 if cerrors.IsChangefeedGCFastFailErrorCode(errors.RFCErrorCode(err.Code)) || 408 err.ShouldFailChangefeed() { 409 m.state.SetError(err) 410 m.shouldBeRunning = false 411 m.patchState(model.StateFailed) 412 return 413 } 414 } 415 416 // Changing changefeed state from stopped to failed is allowed 417 // but changing changefeed state from stopped to error or normal is not allowed. 418 if m.state.GetChangefeedInfo() != nil && m.state.GetChangefeedInfo().State == model.StateStopped { 419 log.Warn("changefeed is stopped, ignore errors", 420 zap.String("namespace", m.state.GetID().Namespace), 421 zap.String("changefeed", m.state.GetID().ID), 422 zap.Any("errors", errs)) 423 return 424 } 425 426 var lastError *model.RunningError 427 // find the last non nil error 428 // BTW, there shouldn't be any nil error in errs 429 // this is just a safe guard 430 for i := len(errs) - 1; i >= 0; i-- { 431 if errs[i] != nil { 432 lastError = errs[i] 433 break 434 } 435 } 436 // if any error is occurred in this tick, we should set the changefeed state to warning 437 // and stop the changefeed 438 if lastError != nil { 439 log.Warn("changefeed meets an error", zap.Any("error", lastError)) 440 m.shouldBeRunning = false 441 m.patchState(model.StatePending) 442 443 // patch the last error to changefeed info 444 m.state.SetError(lastError) 445 446 // The errBackoff needs to be reset before the first retry. 447 if !m.isRetrying { 448 m.resetErrRetry() 449 m.isRetrying = true 450 } 451 } 452 } 453 454 func (m *feedStateManager) HandleWarning(errs ...*model.RunningError) { 455 if len(errs) == 0 { 456 return 457 } 458 lastError := errs[len(errs)-1] 459 460 if m.state.GetChangefeedStatus() != nil { 461 currTime := m.upstream.PDClock.CurrentTime() 462 ckptTime := oracle.GetTimeFromTS(m.state.GetChangefeedStatus().CheckpointTs) 463 m.lastWarningReportCheckpointTs = m.state.GetChangefeedStatus().CheckpointTs 464 465 checkpointTsStuck := time.Since(m.checkpointTsAdvanced) > m.changefeedErrorStuckDuration 466 if checkpointTsStuck { 467 log.Info("changefeed retry on warning for a very long time and does not resume, "+ 468 "it will be failed", 469 zap.String("namespace", m.state.GetID().Namespace), 470 zap.String("changefeed", m.state.GetID().ID), 471 zap.Uint64("checkpointTs", m.state.GetChangefeedStatus().CheckpointTs), 472 zap.Duration("checkpointTime", currTime.Sub(ckptTime)), 473 ) 474 code, _ := cerrors.RFCCode(cerrors.ErrChangefeedUnretryable) 475 m.HandleError(&model.RunningError{ 476 Time: lastError.Time, 477 Addr: lastError.Addr, 478 Code: string(code), 479 Message: lastError.Message, 480 }) 481 return 482 } 483 } 484 485 m.patchState(model.StateWarning) 486 m.state.SetWarning(lastError) 487 } 488 489 // GenerateChangefeedEpoch generates a unique changefeed epoch. 490 func GenerateChangefeedEpoch(ctx context.Context, pdClient pd.Client) uint64 { 491 phyTs, logical, err := pdClient.GetTS(ctx) 492 if err != nil { 493 log.Warn("generate epoch using local timestamp due to error", zap.Error(err)) 494 return uint64(time.Now().UnixNano()) 495 } 496 return oracle.ComposeTS(phyTs, logical) 497 } 498 499 // checkAndChangeState checks the state of the changefeed and change it if needed. 500 // if the state of the changefeed is warning and the changefeed's checkpointTs is 501 // greater than the lastRetryCheckpointTs, it will change the state to normal. 502 func (m *feedStateManager) checkAndChangeState() { 503 if m.state.GetChangefeedInfo() == nil || m.state.GetChangefeedStatus() == nil { 504 return 505 } 506 if m.state.GetChangefeedInfo().State == model.StateWarning && 507 m.state.GetChangefeedStatus().CheckpointTs > m.lastErrorRetryCheckpointTs && 508 m.state.GetChangefeedStatus().CheckpointTs > m.lastWarningReportCheckpointTs { 509 log.Info("changefeed is recovered from warning state,"+ 510 "its checkpointTs is greater than lastRetryCheckpointTs,"+ 511 "it will be changed to normal state", 512 zap.String("namespace", m.state.GetID().Namespace), 513 zap.String("changefeed", m.state.GetID().ID), 514 zap.Uint64("checkpointTs", m.state.GetChangefeedStatus().CheckpointTs), 515 zap.Uint64("lastRetryCheckpointTs", m.lastErrorRetryCheckpointTs)) 516 m.patchState(model.StateNormal) 517 m.isRetrying = false 518 } 519 } 520 521 // checkAndInitLastRetryCheckpointTs checks the lastRetryCheckpointTs and init it if needed. 522 // It the owner is changed, the lastRetryCheckpointTs will be reset to 0, and we should init 523 // it to the checkpointTs of the changefeed when the changefeed is ticked at the first time. 524 func (m *feedStateManager) checkAndInitLastRetryCheckpointTs(status *model.ChangeFeedStatus) { 525 if status == nil || m.lastErrorRetryCheckpointTs != 0 { 526 return 527 } 528 m.lastWarningReportCheckpointTs = status.CheckpointTs 529 m.lastErrorRetryCheckpointTs = status.CheckpointTs 530 log.Info("init lastRetryCheckpointTs", zap.Uint64("lastRetryCheckpointTs", m.lastErrorRetryCheckpointTs)) 531 }