Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
164 changes: 164 additions & 0 deletions logservice/logpuller/range_reload_aggregator.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,164 @@
// Copyright 2026 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package logpuller

import (
"context"
"sort"
"sync"
"time"

"github.com/pingcap/log"
"github.com/pingcap/ticdc/heartbeatpb"
"github.com/pingcap/ticdc/pkg/common"
"go.uber.org/zap"
)

type rangeReloadTaskSet struct {
subscribedSpan *subscribedSpan
filterLoop bool
priority TaskType
spans []heartbeatpb.TableSpan
flushScheduled bool
}

type rangeReloadAggregator struct {
client *subscriptionClient
debounce time.Duration
flushThreshold int

mu sync.Mutex
tasks map[SubscriptionID]*rangeReloadTaskSet
}

func newRangeReloadAggregator(client *subscriptionClient, debounce time.Duration, flushThreshold int) *rangeReloadAggregator {
if flushThreshold <= 0 {
flushThreshold = rangeReloadImmediateFlushMax
}
return &rangeReloadAggregator{
client: client,
debounce: debounce,
flushThreshold: flushThreshold,
tasks: make(map[SubscriptionID]*rangeReloadTaskSet),
}
}

func (a *rangeReloadAggregator) add(
span heartbeatpb.TableSpan,
subscribedSpan *subscribedSpan,
filterLoop bool,
priority TaskType,
) {
if subscribedSpan == nil || common.IsEmptySpan(span) {
return
}
var immediateFlush bool
a.mu.Lock()
set := a.tasks[subscribedSpan.subID]
if set == nil {
set = &rangeReloadTaskSet{
subscribedSpan: subscribedSpan,
filterLoop: filterLoop,
priority: priority,
spans: make([]heartbeatpb.TableSpan, 0, 4),
}
a.tasks[subscribedSpan.subID] = set
}
if priority < set.priority {
set.priority = priority
}
set.spans = append(set.spans, span)
if len(set.spans) >= a.flushThreshold {
immediateFlush = true
set.flushScheduled = false
} else if !set.flushScheduled {
// Debounce reloads so a split/merge storm can be collapsed into a
// smaller batch of merged spans before hitting region metadata APIs.
set.flushScheduled = true
a.client.retryScheduler.schedule(a.debounce, func(_ context.Context) {
a.flush(subscribedSpan.subID)
})
}
a.mu.Unlock()

if immediateFlush {
a.flush(subscribedSpan.subID)
}
}

func (a *rangeReloadAggregator) flush(subID SubscriptionID) {
var (
set *rangeReloadTaskSet
mergedSpans []heartbeatpb.TableSpan
)
a.mu.Lock()
set = a.tasks[subID]
if set == nil {
a.mu.Unlock()
return
}
delete(a.tasks, subID)
set.flushScheduled = false
mergedSpans = mergeTableSpans(set.spans)
a.mu.Unlock()

if set.subscribedSpan.stopped.Load() {
return
}
log.Debug("subscription client flush aggregated reload spans",
zap.Uint64("subscriptionID", uint64(subID)),
zap.Int("inputSpanCount", len(set.spans)),
zap.Int("mergedSpanCount", len(mergedSpans)))
for _, span := range mergedSpans {
a.client.scheduleRangeTaskAfter(rangeTask{
span: span,
subscribedSpan: set.subscribedSpan,
filterLoop: set.filterLoop,
priority: set.priority,
}, 0)
}
}

func mergeTableSpans(spans []heartbeatpb.TableSpan) []heartbeatpb.TableSpan {
if len(spans) <= 1 {
return append([]heartbeatpb.TableSpan(nil), spans...)
}
copied := append([]heartbeatpb.TableSpan(nil), spans...)
sort.Slice(copied, func(i, j int) bool {
return common.StartCompare(copied[i].StartKey, copied[j].StartKey) < 0
})

merged := make([]heartbeatpb.TableSpan, 0, len(copied))
for _, span := range copied {
if common.IsEmptySpan(span) {
continue
}
if len(merged) == 0 {
merged = append(merged, span)
continue
}
last := merged[len(merged)-1]
if common.EndCompare(last.EndKey, span.StartKey) >= 0 {
// Adjacent retry spans are merged too, because they can be resolved
// by a single BatchLoadRegionsWithKeyRange call later.
if common.EndCompare(span.EndKey, last.EndKey) > 0 {
last.EndKey = span.EndKey
}
merged[len(merged)-1] = last
continue
}
merged = append(merged, span)
}
return merged
}
6 changes: 6 additions & 0 deletions logservice/logpuller/region_event_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,6 +277,9 @@ func handleEventEntries(span *subscribedSpan, state *regionFeedState, entries *c
switch entry.Type {
case cdcpb.Event_INITIALIZED:
state.setInitialized()
if state.worker != nil {
state.worker.client.resetRetryBackoff(SubscriptionID(state.requestID), regionID)
}
log.Debug("region is initialized",
zap.Int64("tableID", span.span.TableID),
zap.Uint64("regionID", regionID),
Expand Down Expand Up @@ -364,6 +367,9 @@ func handleResolvedTs(span *subscribedSpan, state *regionFeedState, resolvedTs u
}

state.updateResolvedTs(resolvedTs)
if state.worker != nil {
state.worker.client.resetRetryBackoff(SubscriptionID(state.requestID), regionID)
}
Comment on lines +370 to +372
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

Resetting the retry backoff on every ResolvedTs event is redundant and potentially expensive, as ResolvedTs events are received very frequently. The backoff state only needs to be reset when a region is successfully initialized or when a new request is sent. Since resetRetryBackoff is already called on Event_INITIALIZED and after doSend, this call should be removed to avoid unnecessary map iterations in retryBackoffManager.


ts := uint64(0)
shouldAdvance := false
Expand Down
5 changes: 5 additions & 0 deletions logservice/logpuller/region_request_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ func newRegionRequestWorker(
}
regionErr = &sendRequestToStoreErr{}
}
worker.client.markStoreFailure(worker.store.storeAddr)
for subID, m := range worker.clearRegionStates() {
for _, state := range m {
state.markStopped(regionErr)
Expand Down Expand Up @@ -186,6 +187,7 @@ func (s *regionRequestWorker) run(ctx context.Context, credential *security.Cred
defer func() {
_ = conn.Conn.Close()
}()
s.client.markStoreSuccess(s.store.storeAddr)

g.Go(func() error {
return s.receiveAndDispatchChangeEvents(conn)
Expand Down Expand Up @@ -221,6 +223,7 @@ func (s *regionRequestWorker) receiveAndDispatchChangeEvents(conn *ConnAndClient
}
return errors.Trace(err)
}
s.client.markStoreSuccess(s.store.storeAddr)
if len(changeEvent.Events) > 0 {
s.dispatchRegionChangeEvents(changeEvent.Events)
}
Expand Down Expand Up @@ -345,6 +348,7 @@ func (s *regionRequestWorker) processRegionSendTask(
zap.Error(err))
return errors.Trace(err)
}
s.client.markStoreSuccess(s.store.storeAddr)
// TODO: add a metric?
return nil
}
Expand Down Expand Up @@ -410,6 +414,7 @@ func (s *regionRequestWorker) processRegionSendTask(
s.requestCache.markDone()
return err
}
s.client.resetRetryBackoff(subID, region.verID.GetID())
s.requestCache.markSent(regionReq)
}
regionReq, err = fetchMoreReq()
Expand Down
Loading
Loading