github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/table_sink_wrapper.go (about) 1 // Copyright 2022 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 sinkmanager 15 16 import ( 17 "context" 18 "sort" 19 "sync" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/log" 25 "github.com/pingcap/tiflow/cdc/model" 26 "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter" 27 "github.com/pingcap/tiflow/cdc/processor/tablepb" 28 "github.com/pingcap/tiflow/cdc/sink/tablesink" 29 cerrors "github.com/pingcap/tiflow/pkg/errors" 30 "github.com/pingcap/tiflow/pkg/retry" 31 "github.com/pingcap/tiflow/pkg/util" 32 "github.com/tikv/client-go/v2/oracle" 33 pd "github.com/tikv/pd/client" 34 "go.uber.org/zap" 35 ) 36 37 var tableSinkWrapperVersion uint64 = 0 38 39 // tableSinkWrapper is a wrapper of TableSink, it is used in SinkManager to manage TableSink. 40 // Because in the SinkManager, we write data to TableSink and RedoManager concurrently, 41 // so current sink node can not be reused. 42 type tableSinkWrapper struct { 43 version uint64 44 45 // changefeed used for logging. 46 changefeed model.ChangeFeedID 47 // tableSpan used for logging. 48 span tablepb.Span 49 50 tableSinkCreator func() (tablesink.TableSink, uint64) 51 52 // tableSink is the underlying sink. 53 tableSink struct { 54 sync.RWMutex 55 s tablesink.TableSink 56 version uint64 // it's generated by `tableSinkCreater`. 57 58 innerMu sync.Mutex 59 advanced time.Time 60 resolvedTs model.ResolvedTs 61 checkpointTs model.ResolvedTs 62 lastSyncedTs model.Ts 63 } 64 65 // state used to control the lifecycle of the table. 66 state *tablepb.TableState 67 68 // startTs is the start ts of the table. 69 startTs model.Ts 70 // targetTs is the upper bound of the table sink. 71 targetTs model.Ts 72 73 // barrierTs is the barrier bound of the table sink. 74 barrierTs atomic.Uint64 75 // receivedSorterResolvedTs is the resolved ts received from the sorter. 76 // We use this to advance the redo log. 77 receivedSorterResolvedTs atomic.Uint64 78 79 // replicateTs is the ts that the table sink has started to replicate. 80 replicateTs model.Ts 81 genReplicateTs func(ctx context.Context) (model.Ts, error) 82 83 // lastCleanTime indicates the last time the table has been cleaned. 84 lastCleanTime time.Time 85 86 // rangeEventCounts is for clean the table sorter. 87 // If rangeEventCounts[i].events is greater than 0, it means there must be 88 // events in the range (rangeEventCounts[i-1].lastPos, rangeEventCounts[i].lastPos]. 89 rangeEventCounts []rangeEventCount 90 rangeEventCountsMu sync.Mutex 91 } 92 93 type rangeEventCount struct { 94 // firstPos and lastPos are used to merge many rangeEventCount into one. 95 firstPos sorter.Position 96 lastPos sorter.Position 97 events int 98 } 99 100 func newRangeEventCount(pos sorter.Position, events int) rangeEventCount { 101 return rangeEventCount{ 102 firstPos: pos, 103 lastPos: pos, 104 events: events, 105 } 106 } 107 108 func newTableSinkWrapper( 109 changefeed model.ChangeFeedID, 110 span tablepb.Span, 111 tableSinkCreater func() (tablesink.TableSink, uint64), 112 state tablepb.TableState, 113 startTs model.Ts, 114 targetTs model.Ts, 115 genReplicateTs func(ctx context.Context) (model.Ts, error), 116 ) *tableSinkWrapper { 117 res := &tableSinkWrapper{ 118 version: atomic.AddUint64(&tableSinkWrapperVersion, 1), 119 changefeed: changefeed, 120 span: span, 121 tableSinkCreator: tableSinkCreater, 122 state: &state, 123 startTs: startTs, 124 targetTs: targetTs, 125 genReplicateTs: genReplicateTs, 126 } 127 128 res.tableSink.version = 0 129 res.tableSink.checkpointTs = model.NewResolvedTs(startTs) 130 res.tableSink.resolvedTs = model.NewResolvedTs(startTs) 131 res.tableSink.advanced = time.Now() 132 133 res.receivedSorterResolvedTs.Store(startTs) 134 res.barrierTs.Store(startTs) 135 return res 136 } 137 138 func (t *tableSinkWrapper) start(ctx context.Context, startTs model.Ts) (err error) { 139 if t.replicateTs != 0 { 140 log.Panic("The table sink has already started", 141 zap.String("namespace", t.changefeed.Namespace), 142 zap.String("changefeed", t.changefeed.ID), 143 zap.Stringer("span", &t.span), 144 zap.Uint64("startTs", startTs), 145 zap.Uint64("oldReplicateTs", t.replicateTs), 146 ) 147 } 148 149 // FIXME(qupeng): it can be re-fetched later instead of fails. 150 if t.replicateTs, err = t.genReplicateTs(ctx); err != nil { 151 return errors.Trace(err) 152 } 153 154 log.Info("Sink is started", 155 zap.String("namespace", t.changefeed.Namespace), 156 zap.String("changefeed", t.changefeed.ID), 157 zap.Stringer("span", &t.span), 158 zap.Uint64("startTs", startTs), 159 zap.Uint64("replicateTs", t.replicateTs), 160 ) 161 162 // This start ts maybe greater than the initial start ts of the table sink. 163 // Because in two phase scheduling, the table sink may be advanced to a later ts. 164 // And we can just continue to replicate the table sink from the new start ts. 165 util.MustCompareAndMonotonicIncrease(&t.receivedSorterResolvedTs, startTs) 166 // the barrierTs should always larger than or equal to the checkpointTs, so we need to update 167 // barrierTs before the checkpointTs is updated. 168 t.updateBarrierTs(startTs) 169 if model.NewResolvedTs(startTs).Greater(t.tableSink.checkpointTs) { 170 t.tableSink.checkpointTs = model.NewResolvedTs(startTs) 171 t.tableSink.resolvedTs = model.NewResolvedTs(startTs) 172 t.tableSink.advanced = time.Now() 173 } 174 t.state.Store(tablepb.TableStateReplicating) 175 return nil 176 } 177 178 func (t *tableSinkWrapper) appendRowChangedEvents(events ...*model.RowChangedEvent) error { 179 t.tableSink.RLock() 180 defer t.tableSink.RUnlock() 181 if t.tableSink.s == nil { 182 // If it's nil it means it's closed. 183 return tablesink.NewSinkInternalError(errors.New("table sink cleared")) 184 } 185 t.tableSink.s.AppendRowChangedEvents(events...) 186 return nil 187 } 188 189 func (t *tableSinkWrapper) updateBarrierTs(ts model.Ts) { 190 util.MustCompareAndMonotonicIncrease(&t.barrierTs, ts) 191 } 192 193 func (t *tableSinkWrapper) updateReceivedSorterResolvedTs(ts model.Ts) { 194 increased := util.CompareAndMonotonicIncrease(&t.receivedSorterResolvedTs, ts) 195 if increased && t.state.Load() == tablepb.TableStatePreparing { 196 // Update the state to `Prepared` when the receivedSorterResolvedTs is updated for the first time. 197 t.state.Store(tablepb.TableStatePrepared) 198 } 199 } 200 201 func (t *tableSinkWrapper) updateResolvedTs(ts model.ResolvedTs) error { 202 t.tableSink.RLock() 203 defer t.tableSink.RUnlock() 204 if t.tableSink.s == nil { 205 // If it's nil it means it's closed. 206 return tablesink.NewSinkInternalError(errors.New("table sink cleared")) 207 } 208 t.tableSink.innerMu.Lock() 209 defer t.tableSink.innerMu.Unlock() 210 t.tableSink.resolvedTs = ts 211 return t.tableSink.s.UpdateResolvedTs(ts) 212 } 213 214 func (t *tableSinkWrapper) getLastSyncedTs() uint64 { 215 t.tableSink.RLock() 216 defer t.tableSink.RUnlock() 217 if t.tableSink.s != nil { 218 return t.tableSink.s.GetLastSyncedTs() 219 } 220 return t.tableSink.lastSyncedTs 221 } 222 223 func (t *tableSinkWrapper) getCheckpointTs() model.ResolvedTs { 224 t.tableSink.RLock() 225 defer t.tableSink.RUnlock() 226 t.tableSink.innerMu.Lock() 227 defer t.tableSink.innerMu.Unlock() 228 229 if t.tableSink.s != nil { 230 checkpointTs := t.tableSink.s.GetCheckpointTs() 231 if t.tableSink.checkpointTs.Less(checkpointTs) { 232 t.tableSink.checkpointTs = checkpointTs 233 t.tableSink.advanced = time.Now() 234 } else if !checkpointTs.Less(t.tableSink.resolvedTs) { 235 t.tableSink.advanced = time.Now() 236 } 237 } 238 return t.tableSink.checkpointTs 239 } 240 241 func (t *tableSinkWrapper) getReceivedSorterResolvedTs() model.Ts { 242 return t.receivedSorterResolvedTs.Load() 243 } 244 245 func (t *tableSinkWrapper) getState() tablepb.TableState { 246 return t.state.Load() 247 } 248 249 // getUpperBoundTs returns the upperbound of the table sink. 250 // It is used by sinkManager to generate sink task. 251 // upperBoundTs should be the minimum of the following two values: 252 // 1. the resolved ts of the sorter 253 // 2. the barrier ts of the table 254 func (t *tableSinkWrapper) getUpperBoundTs() model.Ts { 255 resolvedTs := t.getReceivedSorterResolvedTs() 256 barrierTs := t.barrierTs.Load() 257 if resolvedTs > barrierTs { 258 resolvedTs = barrierTs 259 } 260 return resolvedTs 261 } 262 263 func (t *tableSinkWrapper) markAsClosing() { 264 for { 265 curr := t.state.Load() 266 if curr == tablepb.TableStateStopping || curr == tablepb.TableStateStopped { 267 break 268 } 269 if t.state.CompareAndSwap(curr, tablepb.TableStateStopping) { 270 log.Info("Sink is closing", 271 zap.String("namespace", t.changefeed.Namespace), 272 zap.String("changefeed", t.changefeed.ID), 273 zap.Stringer("span", &t.span)) 274 break 275 } 276 } 277 } 278 279 func (t *tableSinkWrapper) markAsClosed() { 280 for { 281 curr := t.state.Load() 282 if curr == tablepb.TableStateStopped { 283 return 284 } 285 if t.state.CompareAndSwap(curr, tablepb.TableStateStopped) { 286 log.Info("Sink is closed", 287 zap.String("namespace", t.changefeed.Namespace), 288 zap.String("changefeed", t.changefeed.ID), 289 zap.Stringer("span", &t.span)) 290 return 291 } 292 } 293 } 294 295 func (t *tableSinkWrapper) asyncStop() bool { 296 t.markAsClosing() 297 if t.asyncCloseAndClearTableSink() { 298 t.markAsClosed() 299 return true 300 } 301 return false 302 } 303 304 // Return true means the internal table sink has been initialized. 305 func (t *tableSinkWrapper) initTableSink() bool { 306 t.tableSink.Lock() 307 defer t.tableSink.Unlock() 308 if t.tableSink.s == nil { 309 t.tableSink.s, t.tableSink.version = t.tableSinkCreator() 310 if t.tableSink.s != nil { 311 t.tableSink.advanced = time.Now() 312 return true 313 } 314 return false 315 } 316 return true 317 } 318 319 func (t *tableSinkWrapper) asyncCloseTableSink() bool { 320 t.tableSink.RLock() 321 defer t.tableSink.RUnlock() 322 if t.tableSink.s == nil { 323 return true 324 } 325 return t.tableSink.s.AsyncClose() 326 } 327 328 func (t *tableSinkWrapper) closeTableSink() { 329 t.tableSink.RLock() 330 defer t.tableSink.RUnlock() 331 if t.tableSink.s == nil { 332 return 333 } 334 t.tableSink.s.Close() 335 } 336 337 func (t *tableSinkWrapper) asyncCloseAndClearTableSink() bool { 338 closed := t.asyncCloseTableSink() 339 if closed { 340 t.doTableSinkClear() 341 } 342 return closed 343 } 344 345 func (t *tableSinkWrapper) closeAndClearTableSink() { 346 t.closeTableSink() 347 t.doTableSinkClear() 348 } 349 350 func (t *tableSinkWrapper) doTableSinkClear() { 351 t.tableSink.Lock() 352 defer t.tableSink.Unlock() 353 if t.tableSink.s == nil { 354 return 355 } 356 checkpointTs := t.tableSink.s.GetCheckpointTs() 357 t.tableSink.innerMu.Lock() 358 if t.tableSink.checkpointTs.Less(checkpointTs) { 359 t.tableSink.checkpointTs = checkpointTs 360 } 361 t.tableSink.resolvedTs = checkpointTs 362 t.tableSink.lastSyncedTs = t.tableSink.s.GetLastSyncedTs() 363 t.tableSink.advanced = time.Now() 364 t.tableSink.innerMu.Unlock() 365 t.tableSink.s = nil 366 t.tableSink.version = 0 367 } 368 369 func (t *tableSinkWrapper) checkTableSinkHealth() (err error) { 370 t.tableSink.RLock() 371 defer t.tableSink.RUnlock() 372 if t.tableSink.s != nil { 373 err = t.tableSink.s.CheckHealth() 374 } 375 return 376 } 377 378 // When the attached sink fail, there can be some events that have already been 379 // committed at downstream but we don't know. So we need to update `replicateTs` 380 // of the table so that we can re-send those events later. 381 func (t *tableSinkWrapper) restart(ctx context.Context) (err error) { 382 if t.replicateTs, err = t.genReplicateTs(ctx); err != nil { 383 return errors.Trace(err) 384 } 385 log.Info("Sink is restarted", 386 zap.String("namespace", t.changefeed.Namespace), 387 zap.String("changefeed", t.changefeed.ID), 388 zap.Stringer("span", &t.span), 389 zap.Uint64("replicateTs", t.replicateTs)) 390 return nil 391 } 392 393 func (t *tableSinkWrapper) updateRangeEventCounts(eventCount rangeEventCount) { 394 t.rangeEventCountsMu.Lock() 395 defer t.rangeEventCountsMu.Unlock() 396 397 countsLen := len(t.rangeEventCounts) 398 if countsLen == 0 { 399 t.rangeEventCounts = append(t.rangeEventCounts, eventCount) 400 return 401 } 402 if t.rangeEventCounts[countsLen-1].lastPos.Compare(eventCount.lastPos) < 0 { 403 // If two rangeEventCounts are close enough, we can merge them into one record 404 // to save memory usage. When merging B into A, A.lastPos will be updated but 405 // A.firstPos will be kept so that we can determine whether to continue to merge 406 // more events or not based on timeDiff(C.lastPos, A.firstPos). 407 lastPhy := oracle.ExtractPhysical(t.rangeEventCounts[countsLen-1].firstPos.CommitTs) 408 currPhy := oracle.ExtractPhysical(eventCount.lastPos.CommitTs) 409 if (currPhy - lastPhy) >= 1000 { // 1000 means 1000ms. 410 t.rangeEventCounts = append(t.rangeEventCounts, eventCount) 411 } else { 412 t.rangeEventCounts[countsLen-1].lastPos = eventCount.lastPos 413 t.rangeEventCounts[countsLen-1].events += eventCount.events 414 } 415 } 416 } 417 418 func (t *tableSinkWrapper) cleanRangeEventCounts(upperBound sorter.Position, minEvents int) bool { 419 t.rangeEventCountsMu.Lock() 420 defer t.rangeEventCountsMu.Unlock() 421 422 idx := sort.Search(len(t.rangeEventCounts), func(i int) bool { 423 return t.rangeEventCounts[i].lastPos.Compare(upperBound) > 0 424 }) 425 if len(t.rangeEventCounts) == 0 || idx == 0 { 426 return false 427 } 428 429 count := 0 430 for _, events := range t.rangeEventCounts[0:idx] { 431 count += events.events 432 } 433 shouldClean := count >= minEvents 434 435 if !shouldClean { 436 // To reduce sorter.CleanByTable calls. 437 t.rangeEventCounts[idx-1].events = count 438 t.rangeEventCounts = t.rangeEventCounts[idx-1:] 439 } else { 440 t.rangeEventCounts = t.rangeEventCounts[idx:] 441 } 442 return shouldClean 443 } 444 445 func (t *tableSinkWrapper) sinkMaybeStuck(stuckCheck time.Duration) (bool, uint64) { 446 t.getCheckpointTs() 447 448 t.tableSink.RLock() 449 defer t.tableSink.RUnlock() 450 t.tableSink.innerMu.Lock() 451 defer t.tableSink.innerMu.Unlock() 452 453 // What these conditions mean: 454 // 1. the table sink has been associated with a valid sink; 455 // 2. its checkpoint hasn't been advanced for a while; 456 version := t.tableSink.version 457 advanced := t.tableSink.advanced 458 if version > 0 && time.Since(advanced) > stuckCheck { 459 return true, version 460 } 461 return false, uint64(0) 462 } 463 464 func handleRowChangedEvents( 465 changefeed model.ChangeFeedID, span tablepb.Span, 466 events ...*model.PolymorphicEvent, 467 ) ([]*model.RowChangedEvent, uint64) { 468 size := 0 469 rowChangedEvents := make([]*model.RowChangedEvent, 0, len(events)) 470 for _, e := range events { 471 if e == nil || e.Row == nil { 472 log.Warn("skip emit nil event", 473 zap.String("namespace", changefeed.Namespace), 474 zap.String("changefeed", changefeed.ID), 475 zap.Stringer("span", &span), 476 zap.Any("event", e)) 477 continue 478 } 479 480 rowEvent := e.Row 481 // Some transactions could generate empty row change event, such as 482 // begin; insert into t (id) values (1); delete from t where id=1; commit; 483 // Just ignore these row changed events. 484 if len(rowEvent.Columns) == 0 && len(rowEvent.PreColumns) == 0 { 485 log.Warn("skip emit empty row event", 486 zap.Stringer("span", &span), 487 zap.String("namespace", changefeed.Namespace), 488 zap.String("changefeed", changefeed.ID), 489 zap.Any("event", e)) 490 continue 491 } 492 493 size += rowEvent.ApproximateBytes() 494 rowChangedEvents = append(rowChangedEvents, rowEvent) 495 } 496 return rowChangedEvents, uint64(size) 497 } 498 499 func genReplicateTs(ctx context.Context, pdClient pd.Client) (model.Ts, error) { 500 backoffBaseDelayInMs := int64(100) 501 totalRetryDuration := 10 * time.Second 502 var replicateTs model.Ts 503 err := retry.Do(ctx, func() error { 504 phy, logic, err := pdClient.GetTS(ctx) 505 if err != nil { 506 return errors.Trace(err) 507 } 508 replicateTs = oracle.ComposeTS(phy, logic) 509 return nil 510 }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), 511 retry.WithTotalRetryDuratoin(totalRetryDuration), 512 retry.WithIsRetryableErr(cerrors.IsRetryableError)) 513 if err != nil { 514 return model.Ts(0), errors.Trace(err) 515 } 516 return replicateTs, nil 517 }