github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/kv/shared_region_worker.go (about) 1 // Copyright 2023 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 kv 15 16 import ( 17 "context" 18 "encoding/hex" 19 "time" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/kvproto/pkg/cdcpb" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/spanz" 27 "github.com/prometheus/client_golang/prometheus" 28 "go.uber.org/zap" 29 ) 30 31 // The magic number here is keep the same with some magic numbers in some 32 // other components in TiCDC, including worker pool task chan size, mounter 33 // chan size etc. 34 // TODO: unified channel buffer mechanism 35 var regionWorkerInputChanSize = 32 36 37 type regionWorkerMetrics struct { 38 metricReceivedEventSize prometheus.Observer 39 metricDroppedEventSize prometheus.Observer 40 41 metricPullEventInitializedCounter prometheus.Counter 42 metricPullEventCommittedCounter prometheus.Counter 43 metricPullEventPrewriteCounter prometheus.Counter 44 metricPullEventCommitCounter prometheus.Counter 45 metricPullEventRollbackCounter prometheus.Counter 46 47 metricSendEventResolvedCounter prometheus.Counter 48 metricSendEventCommitCounter prometheus.Counter 49 metricSendEventCommittedCounter prometheus.Counter 50 51 metricQueueDuration prometheus.Observer 52 53 metricWorkerBusyRatio prometheus.Gauge 54 metricWorkerChannelSize prometheus.Gauge 55 } 56 57 func newRegionWorkerMetrics(changefeedID model.ChangeFeedID, tableID string, storeAddr string) *regionWorkerMetrics { 58 metrics := ®ionWorkerMetrics{} 59 metrics.metricReceivedEventSize = eventSize.WithLabelValues("received") 60 metrics.metricDroppedEventSize = eventSize.WithLabelValues("dropped") 61 62 metrics.metricPullEventInitializedCounter = pullEventCounter. 63 WithLabelValues(cdcpb.Event_INITIALIZED.String(), changefeedID.Namespace, changefeedID.ID) 64 metrics.metricPullEventCommittedCounter = pullEventCounter. 65 WithLabelValues(cdcpb.Event_COMMITTED.String(), changefeedID.Namespace, changefeedID.ID) 66 metrics.metricPullEventPrewriteCounter = pullEventCounter. 67 WithLabelValues(cdcpb.Event_PREWRITE.String(), changefeedID.Namespace, changefeedID.ID) 68 metrics.metricPullEventCommitCounter = pullEventCounter. 69 WithLabelValues(cdcpb.Event_COMMIT.String(), changefeedID.Namespace, changefeedID.ID) 70 metrics.metricPullEventRollbackCounter = pullEventCounter. 71 WithLabelValues(cdcpb.Event_ROLLBACK.String(), changefeedID.Namespace, changefeedID.ID) 72 73 metrics.metricSendEventResolvedCounter = sendEventCounter. 74 WithLabelValues("native-resolved", changefeedID.Namespace, changefeedID.ID) 75 metrics.metricSendEventCommitCounter = sendEventCounter. 76 WithLabelValues("commit", changefeedID.Namespace, changefeedID.ID) 77 metrics.metricSendEventCommittedCounter = sendEventCounter. 78 WithLabelValues("committed", changefeedID.Namespace, changefeedID.ID) 79 80 metrics.metricQueueDuration = regionWorkerQueueDuration. 81 WithLabelValues(changefeedID.Namespace, changefeedID.ID) 82 83 metrics.metricWorkerBusyRatio = workerBusyRatio.WithLabelValues( 84 changefeedID.Namespace, changefeedID.ID, tableID, storeAddr, "event-handler") 85 metrics.metricWorkerChannelSize = workerChannelSize.WithLabelValues( 86 changefeedID.Namespace, changefeedID.ID, tableID, storeAddr, "input") 87 88 return metrics 89 } 90 91 // NOTE: 92 // 1. all contents come from one same TiKV store stream; 93 // 2. eventItem and resolvedTs shouldn't appear simultaneously; 94 type statefulEvent struct { 95 eventItem eventItem 96 resolvedTsBatch resolvedTsBatch 97 stream *requestedStream 98 start time.Time 99 } 100 101 type eventItem struct { 102 // All items come from one same region. 103 item *cdcpb.Event 104 state *regionFeedState 105 } 106 107 // NOTE: all regions must come from the same subscribedTable, and regions will never be empty. 108 type resolvedTsBatch struct { 109 ts uint64 110 regions []*regionFeedState 111 } 112 113 func newEventItem(item *cdcpb.Event, state *regionFeedState, stream *requestedStream) statefulEvent { 114 return statefulEvent{ 115 eventItem: eventItem{item, state}, 116 stream: stream, 117 start: time.Now(), 118 } 119 } 120 121 func newResolvedTsBatch(ts uint64, stream *requestedStream) statefulEvent { 122 return statefulEvent{ 123 resolvedTsBatch: resolvedTsBatch{ts: ts}, 124 stream: stream, 125 start: time.Now(), 126 } 127 } 128 129 type sharedRegionWorker struct { 130 changefeed model.ChangeFeedID 131 client *SharedClient 132 inputCh chan statefulEvent 133 metrics *regionWorkerMetrics 134 } 135 136 func newSharedRegionWorker(c *SharedClient) *sharedRegionWorker { 137 return &sharedRegionWorker{ 138 changefeed: c.changefeed, 139 client: c, 140 inputCh: make(chan statefulEvent, regionWorkerInputChanSize), 141 metrics: newRegionWorkerMetrics(c.changefeed, "shared", "shared"), 142 } 143 } 144 145 func (w *sharedRegionWorker) sendEvent(ctx context.Context, event statefulEvent) error { 146 select { 147 case <-ctx.Done(): 148 return ctx.Err() 149 case w.inputCh <- event: 150 return nil 151 } 152 } 153 154 func (w *sharedRegionWorker) run(ctx context.Context) error { 155 for { 156 var event statefulEvent 157 select { 158 case <-ctx.Done(): 159 return ctx.Err() 160 case event = <-w.inputCh: 161 } 162 163 w.metrics.metricQueueDuration.Observe(float64(time.Since(event.start).Milliseconds())) 164 w.processEvent(ctx, event) 165 } 166 } 167 168 func (w *sharedRegionWorker) handleSingleRegionError(state *regionFeedState, stream *requestedStream) { 169 stepsToRemoved := state.markRemoved() 170 err := state.takeError() 171 if err != nil { 172 w.client.logRegionDetails("region worker get a region error", 173 zap.String("namespace", w.changefeed.Namespace), 174 zap.String("changefeed", w.changefeed.ID), 175 zap.Uint64("streamID", stream.streamID), 176 zap.Any("subscriptionID", state.getRegionID()), 177 zap.Uint64("regionID", state.region.verID.GetID()), 178 zap.Bool("reschedule", stepsToRemoved), 179 zap.Error(err)) 180 } 181 if stepsToRemoved { 182 stream.takeState(SubscriptionID(state.requestID), state.getRegionID()) 183 w.client.onRegionFail(newRegionErrorInfo(state.getRegionInfo(), err)) 184 } 185 } 186 187 func (w *sharedRegionWorker) processEvent(ctx context.Context, event statefulEvent) { 188 if event.eventItem.state != nil { 189 state := event.eventItem.state 190 if state.isStale() { 191 w.handleSingleRegionError(state, event.stream) 192 return 193 } 194 w.metrics.metricReceivedEventSize.Observe(float64(event.eventItem.item.Event.Size())) 195 switch x := event.eventItem.item.Event.(type) { 196 case *cdcpb.Event_Entries_: 197 if err := w.handleEventEntry(ctx, x, state); err != nil { 198 state.markStopped(err) 199 w.handleSingleRegionError(state, event.stream) 200 return 201 } 202 case *cdcpb.Event_ResolvedTs: 203 w.handleResolvedTs(ctx, resolvedTsBatch{ 204 ts: x.ResolvedTs, 205 regions: []*regionFeedState{state}, 206 }) 207 case *cdcpb.Event_Error: 208 state.markStopped(&eventError{err: x.Error}) 209 w.handleSingleRegionError(state, event.stream) 210 return 211 case *cdcpb.Event_Admin_: 212 } 213 } else if len(event.resolvedTsBatch.regions) > 0 { 214 w.handleResolvedTs(ctx, event.resolvedTsBatch) 215 } 216 } 217 218 // NOTE: context.Canceled won't be treated as an error. 219 func (w *sharedRegionWorker) handleEventEntry(ctx context.Context, x *cdcpb.Event_Entries_, state *regionFeedState) error { 220 startTs := state.region.subscribedTable.startTs 221 emit := func(assembled model.RegionFeedEvent) bool { 222 e := newMultiplexingEvent(assembled, state.region.subscribedTable) 223 select { 224 case state.region.subscribedTable.eventCh <- e: 225 return true 226 case <-ctx.Done(): 227 return false 228 } 229 } 230 tableID := state.region.subscribedTable.span.TableID 231 log.Debug("region worker get an Event", 232 zap.String("namespace", w.changefeed.Namespace), 233 zap.String("changefeed", w.changefeed.ID), 234 zap.Any("subscriptionID", state.region.subscribedTable.subscriptionID), 235 zap.Int64("tableID", tableID), 236 zap.Int("rows", len(x.Entries.GetEntries()))) 237 return handleEventEntry(x, startTs, state, w.metrics, emit, w.changefeed, tableID, w.client.logRegionDetails) 238 } 239 240 func handleEventEntry( 241 x *cdcpb.Event_Entries_, 242 startTs uint64, 243 state *regionFeedState, 244 metrics *regionWorkerMetrics, 245 emit func(assembled model.RegionFeedEvent) bool, 246 changefeed model.ChangeFeedID, 247 tableID model.TableID, 248 logRegionDetails func(msg string, fields ...zap.Field), 249 ) error { 250 regionID, regionSpan, _ := state.getRegionMeta() 251 for _, entry := range x.Entries.GetEntries() { 252 // NOTE: from TiKV 7.0.0, entries are already filtered out in TiKV side. 253 // We can remove the check in future. 254 comparableKey := spanz.ToComparableKey(entry.GetKey()) 255 if entry.Type != cdcpb.Event_INITIALIZED && 256 !spanz.KeyInSpan(comparableKey, regionSpan) { 257 metrics.metricDroppedEventSize.Observe(float64(entry.Size())) 258 continue 259 } 260 switch entry.Type { 261 case cdcpb.Event_INITIALIZED: 262 metrics.metricPullEventInitializedCounter.Inc() 263 state.setInitialized() 264 logRegionDetails("region is initialized", 265 zap.String("namespace", changefeed.Namespace), 266 zap.String("changefeed", changefeed.ID), 267 zap.Int64("tableID", tableID), 268 zap.Uint64("regionID", regionID), 269 zap.Uint64("requestID", state.requestID), 270 zap.Stringer("span", &state.region.span)) 271 272 for _, cachedEvent := range state.matcher.matchCachedRow(true) { 273 revent, err := assembleRowEvent(regionID, cachedEvent) 274 if err != nil { 275 return errors.Trace(err) 276 } 277 if !emit(revent) { 278 return nil 279 } 280 metrics.metricSendEventCommitCounter.Inc() 281 } 282 state.matcher.matchCachedRollbackRow(true) 283 case cdcpb.Event_COMMITTED: 284 resolvedTs := state.getLastResolvedTs() 285 if entry.CommitTs <= resolvedTs { 286 logPanic("The CommitTs must be greater than the resolvedTs", 287 zap.String("EventType", "COMMITTED"), 288 zap.Uint64("CommitTs", entry.CommitTs), 289 zap.Uint64("resolvedTs", resolvedTs), 290 zap.Uint64("regionID", regionID)) 291 return errUnreachable 292 } 293 294 metrics.metricPullEventCommittedCounter.Inc() 295 revent, err := assembleRowEvent(regionID, entry) 296 if err != nil { 297 return errors.Trace(err) 298 } 299 if !emit(revent) { 300 return nil 301 } 302 metrics.metricSendEventCommittedCounter.Inc() 303 case cdcpb.Event_PREWRITE: 304 metrics.metricPullEventPrewriteCounter.Inc() 305 state.matcher.putPrewriteRow(entry) 306 case cdcpb.Event_COMMIT: 307 metrics.metricPullEventCommitCounter.Inc() 308 // NOTE: matchRow should always be called even if the event is stale. 309 if !state.matcher.matchRow(entry, state.isInitialized()) { 310 if !state.isInitialized() { 311 state.matcher.cacheCommitRow(entry) 312 continue 313 } 314 return cerror.ErrPrewriteNotMatch.GenWithStackByArgs( 315 hex.EncodeToString(entry.GetKey()), 316 entry.GetStartTs(), entry.GetCommitTs(), 317 entry.GetType(), entry.GetOpType()) 318 } 319 320 // TiKV can send events with StartTs/CommitTs less than startTs. 321 isStaleEvent := entry.CommitTs <= startTs 322 if isStaleEvent { 323 continue 324 } 325 326 // NOTE: state.getLastResolvedTs() will never less than startTs. 327 resolvedTs := state.getLastResolvedTs() 328 if entry.CommitTs <= resolvedTs { 329 logPanic("The CommitTs must be greater than the resolvedTs", 330 zap.String("EventType", "COMMIT"), 331 zap.Uint64("CommitTs", entry.CommitTs), 332 zap.Uint64("resolvedTs", resolvedTs), 333 zap.Uint64("regionID", regionID)) 334 return errUnreachable 335 } 336 337 revent, err := assembleRowEvent(regionID, entry) 338 if err != nil { 339 return errors.Trace(err) 340 } 341 if !emit(revent) { 342 return nil 343 } 344 metrics.metricSendEventCommitCounter.Inc() 345 case cdcpb.Event_ROLLBACK: 346 metrics.metricPullEventRollbackCounter.Inc() 347 if !state.isInitialized() { 348 state.matcher.cacheRollbackRow(entry) 349 continue 350 } 351 state.matcher.rollbackRow(entry) 352 } 353 } 354 return nil 355 } 356 357 func assembleRowEvent(regionID uint64, entry *cdcpb.Event_Row) (model.RegionFeedEvent, error) { 358 var opType model.OpType 359 switch entry.GetOpType() { 360 case cdcpb.Event_Row_DELETE: 361 opType = model.OpTypeDelete 362 case cdcpb.Event_Row_PUT: 363 opType = model.OpTypePut 364 default: 365 return model.RegionFeedEvent{}, cerror.ErrUnknownKVEventType.GenWithStackByArgs(entry.GetOpType(), entry) 366 } 367 368 revent := model.RegionFeedEvent{ 369 RegionID: regionID, 370 Val: &model.RawKVEntry{ 371 OpType: opType, 372 Key: entry.Key, 373 Value: entry.GetValue(), 374 StartTs: entry.StartTs, 375 CRTs: entry.CommitTs, 376 RegionID: regionID, 377 OldValue: entry.GetOldValue(), 378 }, 379 } 380 381 return revent, nil 382 } 383 384 func (w *sharedRegionWorker) handleResolvedTs(ctx context.Context, batch resolvedTsBatch) { 385 if w.client.config.KVClient.AdvanceIntervalInMs > 0 { 386 w.advanceTableSpan(ctx, batch) 387 } else { 388 w.forwardResolvedTsToPullerFrontier(ctx, batch) 389 } 390 } 391 392 func (w *sharedRegionWorker) forwardResolvedTsToPullerFrontier(ctx context.Context, batch resolvedTsBatch) { 393 resolvedSpans := make(map[SubscriptionID]*struct { 394 spans []model.RegionComparableSpan 395 subscribedTable *subscribedTable 396 }) 397 398 for _, state := range batch.regions { 399 if state.isStale() || !state.isInitialized() { 400 continue 401 } 402 403 spansAndChan := resolvedSpans[state.region.subscribedTable.subscriptionID] 404 if spansAndChan == nil { 405 spansAndChan = &struct { 406 spans []model.RegionComparableSpan 407 subscribedTable *subscribedTable 408 }{subscribedTable: state.region.subscribedTable} 409 resolvedSpans[state.region.subscribedTable.subscriptionID] = spansAndChan 410 } 411 412 regionID := state.getRegionID() 413 lastResolvedTs := state.getLastResolvedTs() 414 if batch.ts < lastResolvedTs { 415 log.Debug("The resolvedTs is fallen back in kvclient", 416 zap.String("namespace", w.changefeed.Namespace), 417 zap.String("changefeed", w.changefeed.ID), 418 zap.Uint64("regionID", regionID), 419 zap.Uint64("resolvedTs", batch.ts), 420 zap.Uint64("lastResolvedTs", lastResolvedTs)) 421 continue 422 } 423 state.updateResolvedTs(batch.ts) 424 425 span := model.RegionComparableSpan{Span: state.region.span, Region: regionID} 426 span.Span.TableID = state.region.subscribedTable.span.TableID 427 spansAndChan.spans = append(spansAndChan.spans, span) 428 } 429 430 for subscriptionID, spansAndChan := range resolvedSpans { 431 log.Debug("region worker get a ResolvedTs", 432 zap.String("namespace", w.changefeed.Namespace), 433 zap.String("changefeed", w.changefeed.ID), 434 zap.Any("subscriptionID", subscriptionID), 435 zap.Uint64("ResolvedTs", batch.ts), 436 zap.Int("spanCount", len(spansAndChan.spans))) 437 if len(spansAndChan.spans) > 0 { 438 revent := model.RegionFeedEvent{Resolved: &model.ResolvedSpans{ 439 Spans: spansAndChan.spans, ResolvedTs: batch.ts, 440 }} 441 e := newMultiplexingEvent(revent, spansAndChan.subscribedTable) 442 select { 443 case spansAndChan.subscribedTable.eventCh <- e: 444 w.metrics.metricSendEventResolvedCounter.Add(float64(len(resolvedSpans))) 445 case <-ctx.Done(): 446 } 447 } 448 } 449 } 450 451 func (w *sharedRegionWorker) advanceTableSpan(ctx context.Context, batch resolvedTsBatch) { 452 for _, state := range batch.regions { 453 if state.isStale() || !state.isInitialized() { 454 continue 455 } 456 457 regionID := state.getRegionID() 458 lastResolvedTs := state.getLastResolvedTs() 459 if batch.ts < lastResolvedTs { 460 log.Debug("The resolvedTs is fallen back in kvclient", 461 zap.String("namespace", w.changefeed.Namespace), 462 zap.String("changefeed", w.changefeed.ID), 463 zap.Uint64("regionID", regionID), 464 zap.Uint64("resolvedTs", batch.ts), 465 zap.Uint64("lastResolvedTs", lastResolvedTs)) 466 continue 467 } 468 state.updateResolvedTs(batch.ts) 469 } 470 471 table := batch.regions[0].region.subscribedTable 472 now := time.Now().UnixMilli() 473 lastAdvance := table.lastAdvanceTime.Load() 474 if now-lastAdvance > int64(w.client.config.KVClient.AdvanceIntervalInMs) && table.lastAdvanceTime.CompareAndSwap(lastAdvance, now) { 475 ts := table.rangeLock.ResolvedTs() 476 if ts > table.startTs { 477 revent := model.RegionFeedEvent{ 478 Resolved: &model.ResolvedSpans{ 479 Spans: []model.RegionComparableSpan{{Span: table.span, Region: 0}}, 480 ResolvedTs: ts, 481 }, 482 } 483 e := newMultiplexingEvent(revent, table) 484 select { 485 case table.eventCh <- e: 486 w.metrics.metricSendEventResolvedCounter.Add(float64(len(batch.regions))) 487 case <-ctx.Done(): 488 } 489 } 490 } 491 }