From 84dced12e649c86cc48ca46d16a3492120634999 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 20 Jan 2026 16:17:49 +0800 Subject: [PATCH 01/34] improve memory control v1 Signed-off-by: dongmen <414110582@qq.com> --- .../eventcollector/event_collector.go | 20 +- pkg/common/event/congestion_control.go | 141 +++++++++++- pkg/common/event/congestion_control_test.go | 32 ++- pkg/eventservice/dispatcher_stat.go | 12 +- pkg/eventservice/event_broker.go | 33 ++- pkg/eventservice/event_broker_test.go | 30 +++ pkg/eventservice/scan_window.go | 213 ++++++++++++++++++ 7 files changed, 465 insertions(+), 16 deletions(-) create mode 100644 pkg/eventservice/scan_window.go diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index e5cb6cdd97..a573e36e4e 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -600,6 +600,8 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge // collect path-level available memory and total available memory for each changefeed changefeedPathMemory := make(map[common.ChangeFeedID]map[common.DispatcherID]uint64) changefeedTotalMemory := make(map[common.ChangeFeedID]uint64) + changefeedUsedMemory := make(map[common.ChangeFeedID]uint64) + changefeedMaxMemory := make(map[common.ChangeFeedID]uint64) // collect from main dynamic stream for _, quota := range c.ds.GetMetrics().MemoryControl.AreaMemoryMetrics { @@ -617,6 +619,8 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge } // store total available memory from AreaMemoryMetric changefeedTotalMemory[cfID] = uint64(quota.AvailableMemory()) + changefeedUsedMemory[cfID] = uint64(quota.MemoryUsage()) + changefeedMaxMemory[cfID] = uint64(quota.MaxMemory()) } // collect from redo dynamic stream and take minimum @@ -643,6 +647,16 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge } else { changefeedTotalMemory[cfID] = uint64(quota.AvailableMemory()) } + if existing, exists := changefeedUsedMemory[cfID]; exists { + changefeedUsedMemory[cfID] = min(existing, uint64(quota.MemoryUsage())) + } else { + changefeedUsedMemory[cfID] = uint64(quota.MemoryUsage()) + } + if existing, exists := changefeedMaxMemory[cfID]; exists { + changefeedMaxMemory[cfID] = min(existing, uint64(quota.MaxMemory())) + } else { + changefeedMaxMemory[cfID] = uint64(quota.MaxMemory()) + } } if len(changefeedPathMemory) == 0 { @@ -679,7 +693,7 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge // build congestion control messages for each node result := make(map[node.ID]*event.CongestionControl) for nodeID, changefeedDispatchers := range nodeDispatcherMemory { - congestionControl := event.NewCongestionControl() + congestionControl := event.NewCongestionControlWithVersion(event.CongestionControlVersion2) for changefeedID, dispatcherMemory := range changefeedDispatchers { if len(dispatcherMemory) == 0 { @@ -689,9 +703,11 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge // get total available memory directly from AreaMemoryMetric totalAvailable := uint64(changefeedTotalMemory[changefeedID]) if totalAvailable > 0 { - congestionControl.AddAvailableMemoryWithDispatchers( + congestionControl.AddAvailableMemoryWithDispatchersAndUsage( changefeedID.ID(), totalAvailable, + changefeedUsedMemory[changefeedID], + changefeedMaxMemory[changefeedID], dispatcherMemory, ) } diff --git a/pkg/common/event/congestion_control.go b/pkg/common/event/congestion_control.go index 5f50b80e25..be71fb7801 100644 --- a/pkg/common/event/congestion_control.go +++ b/pkg/common/event/congestion_control.go @@ -21,12 +21,17 @@ import ( "github.com/pingcap/ticdc/pkg/common" ) -const CongestionControlVersion1 = 1 +const ( + CongestionControlVersion1 = 1 + CongestionControlVersion2 = 2 +) type AvailableMemory struct { Version byte // 1 byte, it should be the same as CongestionControlVersion Gid common.GID // GID is the internal representation of ChangeFeedID - Available uint64 // in bytes, used to report the Available memory + Available uint64 // in bytes, used to report the available memory + Used uint64 // in bytes, used to report the used memory + Max uint64 // in bytes, used to report the max memory DispatcherCount uint32 // used to report the number of dispatchers DispatcherAvailable map[common.DispatcherID]uint64 // in bytes, used to report the memory usage of each dispatcher } @@ -41,18 +46,44 @@ func NewAvailableMemory(gid common.GID, available uint64) AvailableMemory { } func (m AvailableMemory) Marshal() []byte { + return m.marshalV1() +} + +func (m *AvailableMemory) Unmarshal(buf *bytes.Buffer) { + m.unmarshalV1(buf) +} + +func (m AvailableMemory) GetSize() int { + return m.sizeV1() +} + +func (m AvailableMemory) marshalV1() []byte { buf := bytes.NewBuffer(make([]byte, 0)) buf.Write(m.Gid.Marshal()) - binary.Write(buf, binary.BigEndian, m.Available) - binary.Write(buf, binary.BigEndian, m.DispatcherCount) + _ = binary.Write(buf, binary.BigEndian, m.Available) + _ = binary.Write(buf, binary.BigEndian, m.DispatcherCount) for dispatcherID, available := range m.DispatcherAvailable { buf.Write(dispatcherID.Marshal()) - binary.Write(buf, binary.BigEndian, available) + _ = binary.Write(buf, binary.BigEndian, available) } return buf.Bytes() } -func (m *AvailableMemory) Unmarshal(buf *bytes.Buffer) { +func (m AvailableMemory) marshalV2() []byte { + buf := bytes.NewBuffer(make([]byte, 0)) + buf.Write(m.Gid.Marshal()) + _ = binary.Write(buf, binary.BigEndian, m.Available) + _ = binary.Write(buf, binary.BigEndian, m.Used) + _ = binary.Write(buf, binary.BigEndian, m.Max) + _ = binary.Write(buf, binary.BigEndian, m.DispatcherCount) + for dispatcherID, available := range m.DispatcherAvailable { + buf.Write(dispatcherID.Marshal()) + _ = binary.Write(buf, binary.BigEndian, available) + } + return buf.Bytes() +} + +func (m *AvailableMemory) unmarshalV1(buf *bytes.Buffer) { m.Gid.Unmarshal(buf.Next(m.Gid.GetSize())) m.Available = binary.BigEndian.Uint64(buf.Next(8)) m.DispatcherCount = binary.BigEndian.Uint32(buf.Next(4)) @@ -64,7 +95,21 @@ func (m *AvailableMemory) Unmarshal(buf *bytes.Buffer) { } } -func (m AvailableMemory) GetSize() int { +func (m *AvailableMemory) unmarshalV2(buf *bytes.Buffer) { + m.Gid.Unmarshal(buf.Next(m.Gid.GetSize())) + m.Available = binary.BigEndian.Uint64(buf.Next(8)) + m.Used = binary.BigEndian.Uint64(buf.Next(8)) + m.Max = binary.BigEndian.Uint64(buf.Next(8)) + m.DispatcherCount = binary.BigEndian.Uint32(buf.Next(4)) + m.DispatcherAvailable = make(map[common.DispatcherID]uint64) + for range m.DispatcherCount { + dispatcherID := common.DispatcherID{} + dispatcherID.Unmarshal(buf.Next(dispatcherID.GetSize())) + m.DispatcherAvailable[dispatcherID] = binary.BigEndian.Uint64(buf.Next(8)) + } +} + +func (m AvailableMemory) sizeV1() int { // changefeedID size + changefeed available size size := m.Gid.GetSize() + 8 size += 4 // dispatcher count @@ -76,6 +121,18 @@ func (m AvailableMemory) GetSize() int { return size } +func (m AvailableMemory) sizeV2() int { + // changefeedID size + changefeed available size + used size + max size + size := m.Gid.GetSize() + 8 + 8 + 8 + size += 4 // dispatcher count + for range m.DispatcherCount { + dispatcherID := &common.DispatcherID{} + // dispatcherID size + dispatcher available size + size += dispatcherID.GetSize() + 8 + } + return size +} + type CongestionControl struct { version int clusterID uint64 @@ -90,11 +147,22 @@ func NewCongestionControl() *CongestionControl { } } +func NewCongestionControlWithVersion(version int) *CongestionControl { + return &CongestionControl{ + version: version, + } +} + func (c *CongestionControl) GetSize() int { size := 8 // clusterID size += 4 // changefeed count for _, mem := range c.availables { - size += mem.GetSize() + switch c.version { + case CongestionControlVersion2: + size += mem.sizeV2() + default: + size += mem.sizeV1() + } } return size } @@ -109,6 +177,11 @@ func (c *CongestionControl) Marshal() ([]byte, error) { if err != nil { return nil, err } + case CongestionControlVersion2: + payload, err = c.encodeV2() + if err != nil { + return nil, err + } default: return nil, fmt.Errorf("unsupported CongestionControl version: %d", c.version) } @@ -123,7 +196,19 @@ func (c *CongestionControl) encodeV1() ([]byte, error) { _ = binary.Write(buf, binary.BigEndian, c.changefeedCount) for _, item := range c.availables { - data := item.Marshal() + data := item.marshalV1() + buf.Write(data) + } + return buf.Bytes(), nil +} + +func (c *CongestionControl) encodeV2() ([]byte, error) { + buf := bytes.NewBuffer(make([]byte, 0)) + _ = binary.Write(buf, binary.BigEndian, c.clusterID) + _ = binary.Write(buf, binary.BigEndian, c.changefeedCount) + + for _, item := range c.availables { + data := item.marshalV2() buf.Write(data) } return buf.Bytes(), nil @@ -143,6 +228,8 @@ func (c *CongestionControl) Unmarshal(data []byte) error { switch version { case CongestionControlVersion1: return c.decodeV1(payload) + case CongestionControlVersion2: + return c.decodeV2(payload) default: return fmt.Errorf("unsupported CongestionControl version: %d", version) } @@ -155,7 +242,20 @@ func (c *CongestionControl) decodeV1(data []byte) error { c.availables = make([]AvailableMemory, 0, c.changefeedCount) for i := uint32(0); i < c.changefeedCount; i++ { var item AvailableMemory - item.Unmarshal(buf) + item.unmarshalV1(buf) + c.availables = append(c.availables, item) + } + return nil +} + +func (c *CongestionControl) decodeV2(data []byte) error { + buf := bytes.NewBuffer(data) + c.clusterID = binary.BigEndian.Uint64(buf.Next(8)) + c.changefeedCount = binary.BigEndian.Uint32(buf.Next(4)) + c.availables = make([]AvailableMemory, 0, c.changefeedCount) + for i := uint32(0); i < c.changefeedCount; i++ { + var item AvailableMemory + item.unmarshalV2(buf) c.availables = append(c.availables, item) } return nil @@ -174,6 +274,23 @@ func (c *CongestionControl) AddAvailableMemoryWithDispatchers(gid common.GID, av c.availables = append(c.availables, availMem) } +func (c *CongestionControl) AddAvailableMemoryWithDispatchersAndUsage( + gid common.GID, + available uint64, + used uint64, + max uint64, + dispatcherAvailable map[common.DispatcherID]uint64, +) { + c.changefeedCount++ + availMem := NewAvailableMemory(gid, available) + availMem.Version = CongestionControlVersion2 + availMem.Used = used + availMem.Max = max + availMem.DispatcherAvailable = dispatcherAvailable + availMem.DispatcherCount = uint32(len(dispatcherAvailable)) + c.availables = append(c.availables, availMem) +} + func (c *CongestionControl) GetAvailables() []AvailableMemory { return c.availables } @@ -181,3 +298,7 @@ func (c *CongestionControl) GetAvailables() []AvailableMemory { func (c *CongestionControl) GetClusterID() uint64 { return c.clusterID } + +func (c *CongestionControl) GetVersion() int { + return c.version +} diff --git a/pkg/common/event/congestion_control_test.go b/pkg/common/event/congestion_control_test.go index 159a309894..56c239d6f2 100644 --- a/pkg/common/event/congestion_control_test.go +++ b/pkg/common/event/congestion_control_test.go @@ -142,8 +142,6 @@ func TestCongestionControlMarshalUnmarshal(t *testing.T) { require.Equal(t, uint64(4096), availables[1].Available) // Test case 4: CongestionControl with AvailableMemoryWithDispatchers - // Note: DispatcherAvailable field is not properly serialized/deserialized in current implementation - // So we only test the basic GID and Available fields control4 := NewCongestionControl() control4.clusterID = 22222 gid4 := common.NewGID() @@ -170,6 +168,36 @@ func TestCongestionControlMarshalUnmarshal(t *testing.T) { require.Len(t, availableMem.DispatcherAvailable, 2) } +func TestCongestionControlV2(t *testing.T) { + t.Parallel() + + control := NewCongestionControlWithVersion(CongestionControlVersion2) + control.clusterID = 33333 + gid := common.NewGID() + dispatcherAvailable := map[common.DispatcherID]uint64{ + common.NewDispatcherID(): 100, + common.NewDispatcherID(): 200, + } + control.AddAvailableMemoryWithDispatchersAndUsage(gid, 3000, 1200, 4000, dispatcherAvailable) + + data, err := control.Marshal() + require.NoError(t, err) + + // Verify header version + require.Equal(t, uint16(CongestionControlVersion2), binary.BigEndian.Uint16(data[6:8]), "version") + + var decoded CongestionControl + err = decoded.Unmarshal(data) + require.NoError(t, err) + require.Equal(t, CongestionControlVersion2, decoded.version) + require.Equal(t, control.clusterID, decoded.clusterID) + require.Len(t, decoded.availables, 1) + require.Equal(t, uint64(3000), decoded.availables[0].Available) + require.Equal(t, uint64(1200), decoded.availables[0].Used) + require.Equal(t, uint64(4000), decoded.availables[0].Max) + require.Len(t, decoded.availables[0].DispatcherAvailable, 2) +} + func TestCongestionControlMarshalUnmarshalEdgeCases(t *testing.T) { t.Parallel() diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 1d58658d28..d4fd0d781b 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -427,12 +427,22 @@ type changefeedStatus struct { dispatchers sync.Map // common.DispatcherID -> *atomic.Pointer[dispatcherStat] availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) + minCheckpointTs atomic.Uint64 + scanInterval atomic.Int64 + lastAdjustTime atomic.Time + usageWindow *memoryUsageWindow + syncPointEnabled atomic.Bool + syncPointInterval atomic.Int64 } func newChangefeedStatus(changefeedID common.ChangeFeedID) *changefeedStatus { - return &changefeedStatus{ + status := &changefeedStatus{ changefeedID: changefeedID, + usageWindow: newMemoryUsageWindow(memoryUsageWindowDuration), } + status.scanInterval.Store(int64(defaultScanInterval)) + status.lastAdjustTime.Store(time.Time{}) + return status } func (c *changefeedStatus) addDispatcher(id common.DispatcherID, dispatcher *atomic.Pointer[dispatcherStat]) { diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 2d67273fc3..70adc822a3 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -404,6 +404,10 @@ func (c *eventBroker) getScanTaskDataRange(task scanTask) (bool, common.DataRang return false, common.DataRange{} } dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, ddlState.ResolvedTs) + scanMaxTs := task.changefeedStat.getScanMaxTs() + if scanMaxTs > 0 { + dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, scanMaxTs) + } if dataRange.CommitTsEnd <= dataRange.CommitTsStart { updateMetricEventServiceSkipResolvedTsCount(task.info.GetMode()) @@ -922,6 +926,7 @@ func (c *eventBroker) addDispatcher(info DispatcherInfo) error { changefeedID := info.GetChangefeedID() status := c.getOrSetChangefeedStatus(changefeedID) + status.updateSyncPointConfig(info) dispatcher := newDispatcherStat(info, uint64(len(c.taskChan)), uint64(len(c.messageCh)), nil, status) dispatcherPtr := &atomic.Pointer[dispatcherStat]{} dispatcherPtr.Store(dispatcher) @@ -1017,6 +1022,7 @@ func (c *eventBroker) removeDispatcher(dispatcherInfo DispatcherInfo) { stat := statPtr.(*atomic.Pointer[dispatcherStat]).Load() stat.changefeedStat.removeDispatcher(id) + stat.changefeedStat.refreshMinCheckpointTs() c.metricsCollector.metricDispatcherCount.Dec() changefeedID := dispatcherInfo.GetChangefeedID() @@ -1097,6 +1103,7 @@ func (c *eventBroker) resetDispatcher(dispatcherInfo DispatcherInfo) error { } } status := c.getOrSetChangefeedStatus(changefeedID) + status.updateSyncPointConfig(dispatcherInfo) newStat := newDispatcherStat(dispatcherInfo, uint64(len(c.taskChan)), uint64(len(c.messageCh)), tableInfo, status) newStat.copyStatistics(oldStat) @@ -1147,6 +1154,8 @@ func (c *eventBroker) getOrSetChangefeedStatus(changefeedID common.ChangeFeedID) func (c *eventBroker) handleDispatcherHeartbeat(heartbeat *DispatcherHeartBeatWithServerID) { responseMap := make(map[string]*event.DispatcherHeartbeatResponse) + changedChangefeeds := make(map[*changefeedStatus]struct{}) + now := time.Now().Unix() for _, dp := range heartbeat.heartbeat.DispatcherProgresses { dispatcherPtr := c.getDispatcher(dp.DispatcherID) // Can't find the dispatcher, it means the dispatcher is removed. @@ -1165,7 +1174,11 @@ func (c *eventBroker) handleDispatcherHeartbeat(heartbeat *DispatcherHeartBeatWi dispatcher.checkpointTs.Store(dp.CheckpointTs) } // Update the last received heartbeat time to the current time. - dispatcher.lastReceivedHeartbeatTime.Store(time.Now().Unix()) + dispatcher.lastReceivedHeartbeatTime.Store(now) + changedChangefeeds[dispatcher.changefeedStat] = struct{}{} + } + for status := range changedChangefeeds { + status.refreshMinCheckpointTs() } c.sendDispatcherResponse(responseMap) } @@ -1177,14 +1190,27 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC } holder := make(map[common.GID]uint64, len(availables)) + type usageInfo struct { + used uint64 + max uint64 + } + usage := make(map[common.GID]usageInfo, len(availables)) dispatcherAvailable := make(map[common.DispatcherID]uint64, len(availables)) + hasUsage := m.GetVersion() >= event.CongestionControlVersion2 for _, item := range availables { holder[item.Gid] = item.Available + if hasUsage && item.Max > 0 { + usage[item.Gid] = usageInfo{ + used: item.Used, + max: item.Max, + } + } for dispatcherID, available := range item.DispatcherAvailable { dispatcherAvailable[dispatcherID] = available } } + now := time.Now() c.changefeedMap.Range(func(k, v interface{}) bool { changefeedID := k.(common.ChangeFeedID) changefeed := v.(*changefeedStatus) @@ -1193,6 +1219,11 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC changefeed.availableMemoryQuota.Store(from, atomic.NewUint64(available)) metrics.EventServiceAvailableMemoryQuotaGaugeVec.WithLabelValues(changefeedID.String()).Set(float64(available)) } + if hasUsage { + if info, ok := usage[changefeedID.ID()]; ok { + changefeed.updateMemoryUsage(now, info.used, info.max) + } + } return true }) diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index 2982e3feac..f69c3dac4d 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/ticdc/pkg/pdutil" "github.com/pingcap/ticdc/pkg/util" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/atomic" "go.uber.org/zap" ) @@ -171,6 +172,35 @@ func TestOnNotify(t *testing.T) { log.Info("Pass case 6") } +func TestScanRangeCappedByScanWindow(t *testing.T) { + broker, _, _, _ := newEventBrokerForTest() + // Close the broker, so we can catch all message in the test. + broker.close() + + info := newMockDispatcherInfoForTest(t) + info.epoch = 1 + changefeedStatus := broker.getOrSetChangefeedStatus(info.GetChangefeedID()) + changefeedStatus.updateSyncPointConfig(info) + + disp := newDispatcherStat(info, 1, 1, nil, changefeedStatus) + disp.seq.Store(1) + + dispPtr := &atomic.Pointer[dispatcherStat]{} + dispPtr.Store(disp) + changefeedStatus.addDispatcher(disp.id, dispPtr) + + baseTime := time.Now() + baseTs := oracle.GoTimeToTS(baseTime) + disp.checkpointTs.Store(baseTs) + disp.receivedResolvedTs.Store(oracle.GoTimeToTS(baseTime.Add(20 * time.Second))) + disp.eventStoreCommitTs.Store(oracle.GoTimeToTS(baseTime.Add(15 * time.Second))) + changefeedStatus.refreshMinCheckpointTs() + + needScan, dataRange := broker.getScanTaskDataRange(disp) + require.True(t, needScan) + require.Equal(t, oracle.GoTimeToTS(baseTime.Add(defaultScanInterval)), dataRange.CommitTsEnd) +} + func TestCURDDispatcher(t *testing.T) { broker, _, _, _ := newEventBrokerForTest() defer broker.close() diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go new file mode 100644 index 0000000000..502411b60f --- /dev/null +++ b/pkg/eventservice/scan_window.go @@ -0,0 +1,213 @@ +// Copyright 2025 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 eventservice + +import ( + "sync" + "time" + + "github.com/tikv/client-go/v2/oracle" + "go.uber.org/atomic" +) + +const ( + defaultScanInterval = 5 * time.Second + minScanInterval = 1 * time.Second + maxScanInterval = 30 * time.Minute + scanIntervalAdjustCooldown = 30 * time.Second + memoryUsageWindowDuration = 30 * time.Second + memoryUsageHighThreshold = 1.10 + memoryUsageCriticalThreshold = 1.50 + memoryUsageLowThreshold = 0.50 +) + +type memoryUsageSample struct { + ts time.Time + ratio float64 +} + +type memoryUsageWindow struct { + window time.Duration + mu sync.Mutex + samples []memoryUsageSample +} + +func newMemoryUsageWindow(window time.Duration) *memoryUsageWindow { + return &memoryUsageWindow{ + window: window, + } +} + +func (w *memoryUsageWindow) addSample(now time.Time, ratio float64) { + if ratio < 0 { + ratio = 0 + } + w.mu.Lock() + defer w.mu.Unlock() + + w.samples = append(w.samples, memoryUsageSample{ts: now, ratio: ratio}) + w.pruneLocked(now) +} + +func (w *memoryUsageWindow) average(now time.Time) (float64, bool) { + w.mu.Lock() + defer w.mu.Unlock() + + w.pruneLocked(now) + if len(w.samples) == 0 { + return 0, false + } + fullWindow := now.Sub(w.samples[0].ts) >= w.window + var sum float64 + for _, sample := range w.samples { + sum += sample.ratio + } + return sum / float64(len(w.samples)), fullWindow +} + +func (w *memoryUsageWindow) pruneLocked(now time.Time) { + cutoff := now.Add(-w.window) + idx := 0 + for idx < len(w.samples) && w.samples[idx].ts.Before(cutoff) { + idx++ + } + if idx > 0 { + w.samples = w.samples[idx:] + } +} + +func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { + if max == 0 || c.usageWindow == nil { + return + } + ratio := float64(used) / float64(max) + c.usageWindow.addSample(now, ratio) + avg, full := c.usageWindow.average(now) + if !full { + return + } + c.adjustScanInterval(now, avg) +} + +func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { + lastAdjust := c.lastAdjustTime.Load() + if !lastAdjust.IsZero() && now.Sub(lastAdjust) < scanIntervalAdjustCooldown { + return + } + + current := time.Duration(c.scanInterval.Load()) + if current <= 0 { + current = defaultScanInterval + } + maxInterval := c.maxScanInterval() + if maxInterval < minScanInterval { + maxInterval = minScanInterval + } + + newInterval := current + switch { + case avg > memoryUsageCriticalThreshold: + newInterval = minScanInterval + case avg > memoryUsageHighThreshold: + newInterval = maxDuration(current/2, minScanInterval) + case avg < memoryUsageLowThreshold: + newInterval = minDuration(current*2, maxInterval) + } + + if newInterval != current { + c.scanInterval.Store(int64(newInterval)) + c.lastAdjustTime.Store(now) + } +} + +func (c *changefeedStatus) maxScanInterval() time.Duration { + if !c.syncPointEnabled.Load() { + return maxScanInterval + } + interval := time.Duration(c.syncPointInterval.Load()) + if interval <= 0 { + return maxScanInterval + } + if interval < maxScanInterval { + return interval + } + return maxScanInterval +} + +func (c *changefeedStatus) refreshMinCheckpointTs() { + minCheckpoint := ^uint64(0) + c.dispatchers.Range(func(_ any, value any) bool { + dispatcher := value.(*atomic.Pointer[dispatcherStat]).Load() + if dispatcher == nil || dispatcher.isRemoved.Load() || dispatcher.seq.Load() == 0 { + return true + } + checkpoint := dispatcher.checkpointTs.Load() + if checkpoint < minCheckpoint { + minCheckpoint = checkpoint + } + return true + }) + + if minCheckpoint == ^uint64(0) { + c.minCheckpointTs.Store(0) + return + } + c.minCheckpointTs.Store(minCheckpoint) +} + +func (c *changefeedStatus) getScanMaxTs() uint64 { + baseTs := c.minCheckpointTs.Load() + if baseTs == 0 { + return 0 + } + interval := time.Duration(c.scanInterval.Load()) + if interval <= 0 { + interval = defaultScanInterval + } + return oracle.GoTimeToTS(oracle.GetTimeFromTS(baseTs).Add(interval)) +} + +func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { + if !info.SyncPointEnabled() { + return + } + c.syncPointEnabled.Store(true) + interval := info.GetSyncPointInterval() + if interval <= 0 { + return + } + for { + current := time.Duration(c.syncPointInterval.Load()) + if current != 0 && interval >= current { + return + } + if c.syncPointInterval.CompareAndSwap(int64(current), int64(interval)) { + return + } + } +} + +func minDuration(a time.Duration, b time.Duration) time.Duration { + if a < b { + return a + } + return b +} + +func maxDuration(a time.Duration, b time.Duration) time.Duration { + if a > b { + return a + } + return b +} From 1d6d73c3a4b327f0a0310eec4e09fe509c3f1818 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 20 Jan 2026 17:34:53 +0800 Subject: [PATCH 02/34] fix bug Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 25 +++++++++++++++++++++---- pkg/eventservice/scan_window.go | 14 +++++++------- 2 files changed, 28 insertions(+), 11 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 70adc822a3..8a5895e37c 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -180,6 +180,10 @@ func newEventBroker( return c.metricsCollector.Run(ctx) }) + g.Go(func() error { + return c.refreshMinSentResolvedTs(ctx) + }) + log.Info("new event broker created", zap.Uint64("id", id), zap.Uint64("scanLimitInBytes", c.scanLimitInBytes)) return c } @@ -254,6 +258,23 @@ func (c *eventBroker) sendDDL(ctx context.Context, remoteID node.ID, e *event.DD zap.Uint64("seq", e.Seq), zap.Int64("mode", d.info.GetMode())) } +func (c *eventBroker) refreshMinSentResolvedTs(ctx context.Context) error { + ticker := time.NewTicker(time.Second * 1) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return context.Cause(ctx) + case <-ticker.C: + c.changefeedMap.Range(func(key, value interface{}) bool { + status := value.(*changefeedStatus) + status.refreshMinSentResolvedTs() + return true + }) + } + } +} + func (c *eventBroker) sendSignalResolvedTs(d *dispatcherStat) { // Can't send resolvedTs if there was a interrupted scan task happened before. // d.lastScannedStartTs.Load() != 0 indicates that there was a interrupted scan task happened before. @@ -1022,7 +1043,6 @@ func (c *eventBroker) removeDispatcher(dispatcherInfo DispatcherInfo) { stat := statPtr.(*atomic.Pointer[dispatcherStat]).Load() stat.changefeedStat.removeDispatcher(id) - stat.changefeedStat.refreshMinCheckpointTs() c.metricsCollector.metricDispatcherCount.Dec() changefeedID := dispatcherInfo.GetChangefeedID() @@ -1177,9 +1197,6 @@ func (c *eventBroker) handleDispatcherHeartbeat(heartbeat *DispatcherHeartBeatWi dispatcher.lastReceivedHeartbeatTime.Store(now) changedChangefeeds[dispatcher.changefeedStat] = struct{}{} } - for status := range changedChangefeeds { - status.refreshMinCheckpointTs() - } c.sendDispatcherResponse(responseMap) } diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 502411b60f..401517c217 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -145,25 +145,25 @@ func (c *changefeedStatus) maxScanInterval() time.Duration { return maxScanInterval } -func (c *changefeedStatus) refreshMinCheckpointTs() { - minCheckpoint := ^uint64(0) +func (c *changefeedStatus) refreshMinSentResolvedTs() { + minSentResolvedTs := ^uint64(0) c.dispatchers.Range(func(_ any, value any) bool { dispatcher := value.(*atomic.Pointer[dispatcherStat]).Load() if dispatcher == nil || dispatcher.isRemoved.Load() || dispatcher.seq.Load() == 0 { return true } - checkpoint := dispatcher.checkpointTs.Load() - if checkpoint < minCheckpoint { - minCheckpoint = checkpoint + sentResolvedTs := dispatcher.sentResolvedTs.Load() + if sentResolvedTs < minSentResolvedTs { + minSentResolvedTs = sentResolvedTs } return true }) - if minCheckpoint == ^uint64(0) { + if minSentResolvedTs == ^uint64(0) { c.minCheckpointTs.Store(0) return } - c.minCheckpointTs.Store(minCheckpoint) + c.minCheckpointTs.Store(minSentResolvedTs) } func (c *changefeedStatus) getScanMaxTs() uint64 { From 381e57dcbdaaab75f80cc647709938e722275f8d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 14:34:27 +0800 Subject: [PATCH 03/34] add debug log Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 3 +- pkg/eventservice/event_broker.go | 12 ++++++++ pkg/eventservice/scan_window.go | 47 +++++++++++++++++++++++++++-- 3 files changed, 58 insertions(+), 4 deletions(-) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index d4fd0d781b..23de227263 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -427,12 +427,13 @@ type changefeedStatus struct { dispatchers sync.Map // common.DispatcherID -> *atomic.Pointer[dispatcherStat] availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) - minCheckpointTs atomic.Uint64 + minSentTs atomic.Uint64 scanInterval atomic.Int64 lastAdjustTime atomic.Time usageWindow *memoryUsageWindow syncPointEnabled atomic.Bool syncPointInterval atomic.Int64 + lastUsageLogTime atomic.Int64 } func newChangefeedStatus(changefeedID common.ChangeFeedID) *changefeedStatus { diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 8a5895e37c..a5d5a6a39e 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -425,9 +425,21 @@ func (c *eventBroker) getScanTaskDataRange(task scanTask) (bool, common.DataRang return false, common.DataRange{} } dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, ddlState.ResolvedTs) + commitTsEndBeforeWindow := dataRange.CommitTsEnd scanMaxTs := task.changefeedStat.getScanMaxTs() if scanMaxTs > 0 { dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, scanMaxTs) + if dataRange.CommitTsEnd < commitTsEndBeforeWindow { + log.Info("scan window capped", + zap.Stringer("changefeedID", task.changefeedStat.changefeedID), + zap.Stringer("dispatcherID", task.id), + zap.Uint64("baseTs", task.changefeedStat.minSentTs.Load()), + zap.Uint64("scanMaxTs", scanMaxTs), + zap.Uint64("beforeEndTs", commitTsEndBeforeWindow), + zap.Uint64("afterEndTs", dataRange.CommitTsEnd), + zap.Duration("scanInterval", time.Duration(task.changefeedStat.scanInterval.Load())), + ) + } } if dataRange.CommitTsEnd <= dataRange.CommitTsStart { diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 401517c217..e476d0ea47 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -17,8 +17,10 @@ import ( "sync" "time" + "github.com/pingcap/log" "github.com/tikv/client-go/v2/oracle" "go.uber.org/atomic" + "go.uber.org/zap" ) const ( @@ -30,6 +32,7 @@ const ( memoryUsageHighThreshold = 1.10 memoryUsageCriticalThreshold = 1.50 memoryUsageLowThreshold = 0.50 + usageLogInterval = time.Minute ) type memoryUsageSample struct { @@ -89,6 +92,7 @@ func (w *memoryUsageWindow) pruneLocked(now time.Time) { func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { if max == 0 || c.usageWindow == nil { + c.logUsageMissing(now, used, max) return } ratio := float64(used) / float64(max) @@ -128,6 +132,14 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { if newInterval != current { c.scanInterval.Store(int64(newInterval)) c.lastAdjustTime.Store(now) + log.Info("scan interval adjusted", + zap.Stringer("changefeedID", c.changefeedID), + zap.Duration("oldInterval", current), + zap.Duration("newInterval", newInterval), + zap.Duration("maxInterval", maxInterval), + zap.Float64("avgUsage", avg), + zap.Bool("syncPointEnabled", c.syncPointEnabled.Load()), + ) } } @@ -160,14 +172,14 @@ func (c *changefeedStatus) refreshMinSentResolvedTs() { }) if minSentResolvedTs == ^uint64(0) { - c.minCheckpointTs.Store(0) + c.storeMinSentTs(0) return } - c.minCheckpointTs.Store(minSentResolvedTs) + c.storeMinSentTs(minSentResolvedTs) } func (c *changefeedStatus) getScanMaxTs() uint64 { - baseTs := c.minCheckpointTs.Load() + baseTs := c.minSentTs.Load() if baseTs == 0 { return 0 } @@ -178,6 +190,35 @@ func (c *changefeedStatus) getScanMaxTs() uint64 { return oracle.GoTimeToTS(oracle.GetTimeFromTS(baseTs).Add(interval)) } +func (c *changefeedStatus) storeMinSentTs(value uint64) { + prev := c.minSentTs.Load() + if prev == value { + return + } + c.minSentTs.Store(value) + log.Info("scan window base updated", + zap.Stringer("changefeedID", c.changefeedID), + zap.Uint64("oldBaseTs", prev), + zap.Uint64("newBaseTs", value), + ) +} + +func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint64) { + last := c.lastUsageLogTime.Load() + nowUnix := now.Unix() + if nowUnix-last < int64(usageLogInterval.Seconds()) { + return + } + if !c.lastUsageLogTime.CompareAndSwap(last, nowUnix) { + return + } + log.Info("memory usage max is zero", + zap.Stringer("changefeedID", c.changefeedID), + zap.Uint64("used", used), + zap.Uint64("max", max), + ) +} + func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { if !info.SyncPointEnabled() { return From d2852f64086e580eb9dedb48b91a17ecd443515e Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 15:23:50 +0800 Subject: [PATCH 04/34] add debug log 2 Signed-off-by: dongmen <414110582@qq.com> --- .../eventcollector/event_collector.go | 21 +++++++++++++++++++ pkg/eventservice/event_broker.go | 21 +++++++++++++++++++ 2 files changed, 42 insertions(+) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index a573e36e4e..14bd1ca357 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -113,6 +113,8 @@ type EventCollector struct { dispatcherMap sync.Map // key: dispatcherID, value: dispatcherStat changefeedMap sync.Map // key: changefeedID.GID, value: *changefeedStat + lastCongestionLogTime int64 + mc messaging.MessageCenter logCoordinatorClient *LogCoordinatorClient @@ -602,6 +604,8 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge changefeedTotalMemory := make(map[common.ChangeFeedID]uint64) changefeedUsedMemory := make(map[common.ChangeFeedID]uint64) changefeedMaxMemory := make(map[common.ChangeFeedID]uint64) + totalDispatchers := 0 + dispatchersWithoutService := 0 // collect from main dynamic stream for _, quota := range c.ds.GetMetrics().MemoryControl.AreaMemoryMetrics { @@ -669,7 +673,9 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge c.dispatcherMap.Range(func(k, v interface{}) bool { stat := v.(*dispatcherStat) eventServiceID := stat.connState.getEventServiceID() + totalDispatchers++ if eventServiceID == "" { + dispatchersWithoutService++ return true } @@ -692,6 +698,7 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge // build congestion control messages for each node result := make(map[node.ID]*event.CongestionControl) + changefeedsInMessages := 0 for nodeID, changefeedDispatchers := range nodeDispatcherMemory { congestionControl := event.NewCongestionControlWithVersion(event.CongestionControlVersion2) @@ -710,6 +717,7 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge changefeedMaxMemory[changefeedID], dispatcherMemory, ) + changefeedsInMessages++ } } @@ -718,6 +726,19 @@ func (c *EventCollector) newCongestionControlMessages() map[node.ID]*event.Conge } } + nowUnix := time.Now().Unix() + lastLog := atomic.LoadInt64(&c.lastCongestionLogTime) + if nowUnix-lastLog >= int64(time.Minute.Seconds()) && + atomic.CompareAndSwapInt64(&c.lastCongestionLogTime, lastLog, nowUnix) { + log.Info("congestion control build summary", + zap.Int("dispatchers", totalDispatchers), + zap.Int("dispatchersWithoutService", dispatchersWithoutService), + zap.Int("changefeedsWithMetrics", len(changefeedPathMemory)), + zap.Int("changefeedsInMessages", changefeedsInMessages), + zap.Int("nodes", len(result)), + ) + } + return result } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index a5d5a6a39e..48d067d224 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -51,6 +51,7 @@ const ( maxReadyEventIntervalSeconds = 10 // defaultSendResolvedTsInterval use to control whether to send a resolvedTs event to the dispatcher when its scan is skipped. defaultSendResolvedTsInterval = time.Second * 2 + congestionControlLogInterval = time.Minute ) // eventBroker get event from the eventStore, and send the event to the dispatchers. @@ -93,6 +94,8 @@ type eventBroker struct { scanRateLimiter *rate.Limiter scanLimitInBytes uint64 + + lastCongestionLogTime atomic.Int64 } func newEventBroker( @@ -1226,6 +1229,8 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC usage := make(map[common.GID]usageInfo, len(availables)) dispatcherAvailable := make(map[common.DispatcherID]uint64, len(availables)) hasUsage := m.GetVersion() >= event.CongestionControlVersion2 + usageCount := 0 + zeroMaxCount := 0 for _, item := range availables { holder[item.Gid] = item.Available if hasUsage && item.Max > 0 { @@ -1233,6 +1238,9 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC used: item.Used, max: item.Max, } + usageCount++ + } else if hasUsage { + zeroMaxCount++ } for dispatcherID, available := range item.DispatcherAvailable { dispatcherAvailable[dispatcherID] = available @@ -1265,6 +1273,19 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC } return true }) + + nowUnix := now.Unix() + lastLog := c.lastCongestionLogTime.Load() + if nowUnix-lastLog >= int64(congestionControlLogInterval.Seconds()) && + c.lastCongestionLogTime.CompareAndSwap(lastLog, nowUnix) { + log.Info("congestion control received", + zap.Uint16("version", uint16(m.GetVersion())), + zap.Int("changefeedCount", len(availables)), + zap.Int("usageCount", usageCount), + zap.Int("zeroMaxCount", zeroMaxCount), + zap.Int("dispatcherCount", len(dispatcherAvailable)), + ) + } } func (c *eventBroker) sendDispatcherResponse(responseMap map[string]*event.DispatcherHeartbeatResponse) { From f1f9fe1b1c474cf55c92f71fd2b77907669b7671 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 17:02:56 +0800 Subject: [PATCH 05/34] add debug log 3 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 38 +++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index e476d0ea47..412b929bea 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -90,6 +90,22 @@ func (w *memoryUsageWindow) pruneLocked(now time.Time) { } } +func (w *memoryUsageWindow) span(now time.Time) time.Duration { + w.mu.Lock() + defer w.mu.Unlock() + w.pruneLocked(now) + if len(w.samples) == 0 { + return 0 + } + return now.Sub(w.samples[0].ts) +} + +func (w *memoryUsageWindow) count() int { + w.mu.Lock() + defer w.mu.Unlock() + return len(w.samples) +} + func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { if max == 0 || c.usageWindow == nil { c.logUsageMissing(now, used, max) @@ -98,6 +114,7 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin ratio := float64(used) / float64(max) c.usageWindow.addSample(now, ratio) avg, full := c.usageWindow.average(now) + c.logUsageWindow(now, avg, full) if !full { return } @@ -219,6 +236,27 @@ func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint6 ) } +func (c *changefeedStatus) logUsageWindow(now time.Time, avg float64, full bool) { + last := c.lastUsageLogTime.Load() + nowUnix := now.Unix() + if nowUnix-last < int64(usageLogInterval.Seconds()) { + return + } + if !c.lastUsageLogTime.CompareAndSwap(last, nowUnix) { + return + } + windowSpan := c.usageWindow.span(now) + log.Info("scan window usage", + zap.Stringer("changefeedID", c.changefeedID), + zap.Int("sampleCount", c.usageWindow.count()), + zap.Duration("windowSpan", windowSpan), + zap.Bool("fullWindow", full), + zap.Float64("avgUsage", avg), + zap.Duration("scanInterval", time.Duration(c.scanInterval.Load())), + zap.Duration("maxInterval", c.maxScanInterval()), + ) +} + func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { if !info.SyncPointEnabled() { return From 62e1546813311f8baaf932e94e946eec73b01ee5 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 17:20:35 +0800 Subject: [PATCH 06/34] add debug log 4 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 22 ++++++++++-------- pkg/eventservice/scan_window.go | 35 +++++++++++++++++++---------- 2 files changed, 36 insertions(+), 21 deletions(-) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 23de227263..03c1b47980 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -426,14 +426,15 @@ type changefeedStatus struct { dispatchers sync.Map // common.DispatcherID -> *atomic.Pointer[dispatcherStat] - availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) - minSentTs atomic.Uint64 - scanInterval atomic.Int64 - lastAdjustTime atomic.Time - usageWindow *memoryUsageWindow - syncPointEnabled atomic.Bool - syncPointInterval atomic.Int64 - lastUsageLogTime atomic.Int64 + availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) + minSentTs atomic.Uint64 + scanInterval atomic.Int64 + lastAdjustTime atomic.Time + usageWindow *memoryUsageWindow + syncPointEnabled atomic.Bool + syncPointInterval atomic.Int64 + lastUsageLogTime atomic.Time + lastUsageWindowLogTime atomic.Time } func newChangefeedStatus(changefeedID common.ChangeFeedID) *changefeedStatus { @@ -442,7 +443,10 @@ func newChangefeedStatus(changefeedID common.ChangeFeedID) *changefeedStatus { usageWindow: newMemoryUsageWindow(memoryUsageWindowDuration), } status.scanInterval.Store(int64(defaultScanInterval)) - status.lastAdjustTime.Store(time.Time{}) + status.lastAdjustTime.Store(time.Now()) + status.lastUsageLogTime.Store(time.Now()) + status.lastUsageWindowLogTime.Store(time.Now()) + return status } diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 412b929bea..8d9aa4be80 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -107,10 +107,18 @@ func (w *memoryUsageWindow) count() int { } func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { + + log.Info("fizz update memory usage", + zap.Stringer("changefeedID", c.changefeedID), + zap.Uint64("used", used), + zap.Uint64("max", max), + ) + if max == 0 || c.usageWindow == nil { c.logUsageMissing(now, used, max) return } + ratio := float64(used) / float64(max) c.usageWindow.addSample(now, ratio) avg, full := c.usageWindow.average(now) @@ -122,8 +130,16 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin } func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { + log.Info("fizz adjust scan interval", + zap.Stringer("changefeedID", c.changefeedID), + zap.Float64("avgUsage", avg), + ) lastAdjust := c.lastAdjustTime.Load() - if !lastAdjust.IsZero() && now.Sub(lastAdjust) < scanIntervalAdjustCooldown { + if now.Sub(lastAdjust) < scanIntervalAdjustCooldown { + log.Info("fizz adjust scan interval cooldown", + zap.Stringer("changefeedID", c.changefeedID), + zap.Duration("cooldown", scanIntervalAdjustCooldown), + ) return } @@ -222,13 +238,11 @@ func (c *changefeedStatus) storeMinSentTs(value uint64) { func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint64) { last := c.lastUsageLogTime.Load() - nowUnix := now.Unix() - if nowUnix-last < int64(usageLogInterval.Seconds()) { - return - } - if !c.lastUsageLogTime.CompareAndSwap(last, nowUnix) { + if now.Sub(last) < usageLogInterval { return } + c.lastUsageLogTime.Store(now) + log.Info("memory usage max is zero", zap.Stringer("changefeedID", c.changefeedID), zap.Uint64("used", used), @@ -237,14 +251,11 @@ func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint6 } func (c *changefeedStatus) logUsageWindow(now time.Time, avg float64, full bool) { - last := c.lastUsageLogTime.Load() - nowUnix := now.Unix() - if nowUnix-last < int64(usageLogInterval.Seconds()) { - return - } - if !c.lastUsageLogTime.CompareAndSwap(last, nowUnix) { + last := c.lastUsageWindowLogTime.Load() + if now.Sub(last) < usageLogInterval { return } + c.lastUsageWindowLogTime.Store(now) windowSpan := c.usageWindow.span(now) log.Info("scan window usage", zap.Stringer("changefeedID", c.changefeedID), From f714b8d1f1c144c66e624ff94f6b9f9ab38379f9 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 17:35:17 +0800 Subject: [PATCH 07/34] add debug log 5 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 8d9aa4be80..332c298432 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -123,9 +123,9 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin c.usageWindow.addSample(now, ratio) avg, full := c.usageWindow.average(now) c.logUsageWindow(now, avg, full) - if !full { - return - } + // if !full { + // return + // } c.adjustScanInterval(now, avg) } From 2f046434ef90aff3b99b0dd6c0a5c814bc72870d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 18:03:24 +0800 Subject: [PATCH 08/34] adjust = 0.7 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 2 +- pkg/eventservice/scan_window.go | 53 ++++++-------------------------- 2 files changed, 11 insertions(+), 44 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 48d067d224..33eccef911 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -433,7 +433,7 @@ func (c *eventBroker) getScanTaskDataRange(task scanTask) (bool, common.DataRang if scanMaxTs > 0 { dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, scanMaxTs) if dataRange.CommitTsEnd < commitTsEndBeforeWindow { - log.Info("scan window capped", + log.Debug("fizz scan window capped", zap.Stringer("changefeedID", task.changefeedStat.changefeedID), zap.Stringer("dispatcherID", task.id), zap.Uint64("baseTs", task.changefeedStat.minSentTs.Load()), diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 332c298432..2463f0bda3 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -29,9 +29,9 @@ const ( maxScanInterval = 30 * time.Minute scanIntervalAdjustCooldown = 30 * time.Second memoryUsageWindowDuration = 30 * time.Second - memoryUsageHighThreshold = 1.10 - memoryUsageCriticalThreshold = 1.50 - memoryUsageLowThreshold = 0.50 + memoryUsageHighThreshold = 0.7 + memoryUsageCriticalThreshold = 0.9 + memoryUsageLowThreshold = 0.2 usageLogInterval = time.Minute ) @@ -63,20 +63,19 @@ func (w *memoryUsageWindow) addSample(now time.Time, ratio float64) { w.pruneLocked(now) } -func (w *memoryUsageWindow) average(now time.Time) (float64, bool) { +func (w *memoryUsageWindow) average(now time.Time) float64 { w.mu.Lock() defer w.mu.Unlock() w.pruneLocked(now) if len(w.samples) == 0 { - return 0, false + return 0 } - fullWindow := now.Sub(w.samples[0].ts) >= w.window var sum float64 for _, sample := range w.samples { sum += sample.ratio } - return sum / float64(len(w.samples)), fullWindow + return sum / float64(len(w.samples)) } func (w *memoryUsageWindow) pruneLocked(now time.Time) { @@ -107,13 +106,6 @@ func (w *memoryUsageWindow) count() int { } func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { - - log.Info("fizz update memory usage", - zap.Stringer("changefeedID", c.changefeedID), - zap.Uint64("used", used), - zap.Uint64("max", max), - ) - if max == 0 || c.usageWindow == nil { c.logUsageMissing(now, used, max) return @@ -121,25 +113,18 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin ratio := float64(used) / float64(max) c.usageWindow.addSample(now, ratio) - avg, full := c.usageWindow.average(now) - c.logUsageWindow(now, avg, full) - // if !full { - // return - // } + avg := c.usageWindow.average(now) c.adjustScanInterval(now, avg) } func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { - log.Info("fizz adjust scan interval", + log.Info("fizz enter adjust scan interval", zap.Stringer("changefeedID", c.changefeedID), zap.Float64("avgUsage", avg), ) + lastAdjust := c.lastAdjustTime.Load() - if now.Sub(lastAdjust) < scanIntervalAdjustCooldown { - log.Info("fizz adjust scan interval cooldown", - zap.Stringer("changefeedID", c.changefeedID), - zap.Duration("cooldown", scanIntervalAdjustCooldown), - ) + if time.Since(lastAdjust) < scanIntervalAdjustCooldown { return } @@ -250,24 +235,6 @@ func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint6 ) } -func (c *changefeedStatus) logUsageWindow(now time.Time, avg float64, full bool) { - last := c.lastUsageWindowLogTime.Load() - if now.Sub(last) < usageLogInterval { - return - } - c.lastUsageWindowLogTime.Store(now) - windowSpan := c.usageWindow.span(now) - log.Info("scan window usage", - zap.Stringer("changefeedID", c.changefeedID), - zap.Int("sampleCount", c.usageWindow.count()), - zap.Duration("windowSpan", windowSpan), - zap.Bool("fullWindow", full), - zap.Float64("avgUsage", avg), - zap.Duration("scanInterval", time.Duration(c.scanInterval.Load())), - zap.Duration("maxInterval", c.maxScanInterval()), - ) -} - func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { if !info.SyncPointEnabled() { return From 75b2d16b0fecd91700c2770b32d80f1e5697dec1 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 18:19:09 +0800 Subject: [PATCH 09/34] adjust 5 Signed-off-by: dongmen <414110582@qq.com> --- utils/dynstream/memory_control.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/dynstream/memory_control.go b/utils/dynstream/memory_control.go index 9782d7954e..c9915342a7 100644 --- a/utils/dynstream/memory_control.go +++ b/utils/dynstream/memory_control.go @@ -33,7 +33,7 @@ const ( // For now, we only use it in event collector. MemoryControlForEventCollector = 1 - defaultReleaseMemoryRatio = 0.4 + defaultReleaseMemoryRatio = 0.9 defaultDeadlockDuration = 5 * time.Second defaultReleaseMemoryThreshold = 256 ) @@ -166,7 +166,7 @@ func (as *areaMemStat[A, P, T, D, H]) checkDeadlock() bool { hasEventComeButNotOut := time.Since(as.lastAppendEventTime.Load().(time.Time)) < defaultDeadlockDuration && time.Since(as.lastSizeDecreaseTime.Load().(time.Time)) > defaultDeadlockDuration - memoryHighWaterMark := as.memoryUsageRatio() > (1 - defaultReleaseMemoryRatio) + memoryHighWaterMark := as.memoryUsageRatio() > defaultReleaseMemoryRatio return hasEventComeButNotOut && memoryHighWaterMark } From 8dcb0a201fb82ef8b17e48cad3f5b85409016f57 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 21 Jan 2026 18:34:06 +0800 Subject: [PATCH 10/34] for debug Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 2463f0bda3..ea28fb01bd 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -205,6 +205,9 @@ func (c *changefeedStatus) getScanMaxTs() uint64 { if interval <= 0 { interval = defaultScanInterval } + + interval = defaultScanInterval + return oracle.GoTimeToTS(oracle.GetTimeFromTS(baseTs).Add(interval)) } From 994ef9b271151a1aa17db8d1f617142a8321f707 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 22 Jan 2026 16:06:33 +0800 Subject: [PATCH 11/34] adjust 7 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 1 + pkg/eventservice/event_broker.go | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 03c1b47980..6329adca23 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -71,6 +71,7 @@ type dispatcherStat struct { nextSyncPoint atomic.Uint64 syncPointInterval time.Duration txnAtomicity config.AtomicityLevel + lastSyncPoint atomic.Uint64 // ============================================================================= // ================== below are fields need copied when reset ================== diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 33eccef911..8cf40c071c 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -571,6 +571,7 @@ func (c *eventBroker) emitSyncPointEventIfNeeded(ts uint64, d *dispatcherStat, r syncPointEvent := newWrapSyncPointEvent(remoteID, e) c.getMessageCh(d.messageWorkerIndex, common.IsRedoMode(d.info.GetMode())) <- syncPointEvent + d.lastSyncPoint.Store(commitTs) } } @@ -596,6 +597,14 @@ func (c *eventBroker) doScan(ctx context.Context, task scanTask) { if task.isRemoved.Load() { return } + + if task.enableSyncPoint { + lastSyncPoint := task.lastSyncPoint.Load() + if task.sentResolvedTs.Load() > lastSyncPoint { + return + } + } + // If the target is not ready to send, we don't need to scan the event store. // To avoid the useless scan task. if !c.msgSender.IsReadyToSend(remoteID) { From 40736105246c364cd337409ef01e57a4f05a18b6 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 22 Jan 2026 16:11:04 +0800 Subject: [PATCH 12/34] fix bug Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 8cf40c071c..39e593b0c1 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -600,7 +600,7 @@ func (c *eventBroker) doScan(ctx context.Context, task scanTask) { if task.enableSyncPoint { lastSyncPoint := task.lastSyncPoint.Load() - if task.sentResolvedTs.Load() > lastSyncPoint { + if task.sentResolvedTs.Load() > lastSyncPoint && task.changefeedStat.minSentTs.Load() < lastSyncPoint { return } } From 87c5e6f1a3aa2536c88f05d1f2f63a9cf0b7e472 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 22 Jan 2026 16:21:26 +0800 Subject: [PATCH 13/34] fix bug 2 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 12 ++++++------ utils/dynstream/memory_control.go | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 39e593b0c1..980d576cac 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -598,12 +598,12 @@ func (c *eventBroker) doScan(ctx context.Context, task scanTask) { return } - if task.enableSyncPoint { - lastSyncPoint := task.lastSyncPoint.Load() - if task.sentResolvedTs.Load() > lastSyncPoint && task.changefeedStat.minSentTs.Load() < lastSyncPoint { - return - } - } + // if task.enableSyncPoint { + // lastSyncPoint := task.lastSyncPoint.Load() + // if task.sentResolvedTs.Load() > lastSyncPoint && task.changefeedStat.minSentTs.Load() < lastSyncPoint { + // return + // } + // } // If the target is not ready to send, we don't need to scan the event store. // To avoid the useless scan task. diff --git a/utils/dynstream/memory_control.go b/utils/dynstream/memory_control.go index c9915342a7..6f54c26e7a 100644 --- a/utils/dynstream/memory_control.go +++ b/utils/dynstream/memory_control.go @@ -33,7 +33,7 @@ const ( // For now, we only use it in event collector. MemoryControlForEventCollector = 1 - defaultReleaseMemoryRatio = 0.9 + defaultReleaseMemoryRatio = 0.6 defaultDeadlockDuration = 5 * time.Second defaultReleaseMemoryThreshold = 256 ) From fcf76f22a55e010a5d3c657c2fa2ff4570718f54 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 22 Jan 2026 16:24:18 +0800 Subject: [PATCH 14/34] adjust 10 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 980d576cac..549c8b4ac4 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -1174,6 +1174,11 @@ func (c *eventBroker) resetDispatcher(dispatcherInfo DispatcherInfo) error { oldStat.isRemoved.Store(true) } + if newStat.epoch > 1 { + newStat.changefeedStat.scanInterval.Store(int64(defaultScanInterval)) + newStat.changefeedStat.lastAdjustTime.Store(time.Now()) + } + log.Info("reset dispatcher", zap.Stringer("changefeedID", newStat.changefeedStat.changefeedID), zap.Stringer("dispatcherID", newStat.id), zap.Int64("tableID", newStat.info.GetTableSpan().GetTableID()), From 0853113486521371cdaf5472f9f4396e3a50c40d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 22 Jan 2026 16:38:24 +0800 Subject: [PATCH 15/34] adjust 11 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 8 +++++--- pkg/eventservice/event_broker.go | 7 ------- pkg/eventservice/scan_window.go | 7 ++++--- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 6329adca23..528627b290 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -427,9 +427,11 @@ type changefeedStatus struct { dispatchers sync.Map // common.DispatcherID -> *atomic.Pointer[dispatcherStat] - availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) - minSentTs atomic.Uint64 - scanInterval atomic.Int64 + availableMemoryQuota sync.Map // nodeID -> atomic.Uint64 (memory quota in bytes) + minSentTs atomic.Uint64 + scanInterval atomic.Int64 + lastRatio atomic.Float64 + lastAdjustTime atomic.Time usageWindow *memoryUsageWindow syncPointEnabled atomic.Bool diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 549c8b4ac4..d81be4f9a3 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -598,13 +598,6 @@ func (c *eventBroker) doScan(ctx context.Context, task scanTask) { return } - // if task.enableSyncPoint { - // lastSyncPoint := task.lastSyncPoint.Load() - // if task.sentResolvedTs.Load() > lastSyncPoint && task.changefeedStat.minSentTs.Load() < lastSyncPoint { - // return - // } - // } - // If the target is not ready to send, we don't need to scan the event store. // To avoid the useless scan task. if !c.msgSender.IsReadyToSend(remoteID) { diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index ea28fb01bd..76be9cad9c 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -113,8 +113,8 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin ratio := float64(used) / float64(max) c.usageWindow.addSample(now, ratio) - avg := c.usageWindow.average(now) - c.adjustScanInterval(now, avg) + //avg := c.usageWindow.average(now) + c.adjustScanInterval(now, ratio) } func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { @@ -124,7 +124,8 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { ) lastAdjust := c.lastAdjustTime.Load() - if time.Since(lastAdjust) < scanIntervalAdjustCooldown { + // slow grow, fast shrink + if c.lastRatio.Load() < avg && time.Since(lastAdjust) < scanIntervalAdjustCooldown { return } From 9139d93a2faa45c72fc4df63626afb8e149c24e6 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Fri, 23 Jan 2026 12:54:50 +0800 Subject: [PATCH 16/34] allow scan interval grow larger Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 76be9cad9c..7356f41592 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -32,6 +32,7 @@ const ( memoryUsageHighThreshold = 0.7 memoryUsageCriticalThreshold = 0.9 memoryUsageLowThreshold = 0.2 + memoryUsageVeryLowThreshold = 0.05 usageLogInterval = time.Minute ) @@ -113,8 +114,8 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin ratio := float64(used) / float64(max) c.usageWindow.addSample(now, ratio) - //avg := c.usageWindow.average(now) - c.adjustScanInterval(now, ratio) + avg := c.usageWindow.average(now) + c.adjustScanInterval(now, avg) } func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { @@ -123,12 +124,6 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { zap.Float64("avgUsage", avg), ) - lastAdjust := c.lastAdjustTime.Load() - // slow grow, fast shrink - if c.lastRatio.Load() < avg && time.Since(lastAdjust) < scanIntervalAdjustCooldown { - return - } - current := time.Duration(c.scanInterval.Load()) if current <= 0 { current = defaultScanInterval @@ -146,6 +141,9 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { newInterval = maxDuration(current/2, minScanInterval) case avg < memoryUsageLowThreshold: newInterval = minDuration(current*2, maxInterval) + case avg < memoryUsageVeryLowThreshold: + maxInterval = maxScanInterval + newInterval = minDuration(current*2, maxInterval) } if newInterval != current { From 274878f678e63fa521d6f19b1e81f8a1e8760e61 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Fri, 23 Jan 2026 16:11:05 +0800 Subject: [PATCH 17/34] allow scan interval grow larger 2 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 7356f41592..8ddcb1cec6 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -32,7 +32,7 @@ const ( memoryUsageHighThreshold = 0.7 memoryUsageCriticalThreshold = 0.9 memoryUsageLowThreshold = 0.2 - memoryUsageVeryLowThreshold = 0.05 + memoryUsageVeryLowThreshold = 0.1 usageLogInterval = time.Minute ) From 36be880a8a2eb351fa4ee73193e56cb108de5d04 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Fri, 23 Jan 2026 17:20:11 +0800 Subject: [PATCH 18/34] remove verbose log Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 8ddcb1cec6..5beab5ff3b 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -33,7 +33,6 @@ const ( memoryUsageCriticalThreshold = 0.9 memoryUsageLowThreshold = 0.2 memoryUsageVeryLowThreshold = 0.1 - usageLogInterval = time.Minute ) type memoryUsageSample struct { @@ -108,7 +107,6 @@ func (w *memoryUsageWindow) count() int { func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { if max == 0 || c.usageWindow == nil { - c.logUsageMissing(now, used, max) return } @@ -223,20 +221,6 @@ func (c *changefeedStatus) storeMinSentTs(value uint64) { ) } -func (c *changefeedStatus) logUsageMissing(now time.Time, used uint64, max uint64) { - last := c.lastUsageLogTime.Load() - if now.Sub(last) < usageLogInterval { - return - } - c.lastUsageLogTime.Store(now) - - log.Info("memory usage max is zero", - zap.Stringer("changefeedID", c.changefeedID), - zap.Uint64("used", used), - zap.Uint64("max", max), - ) -} - func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { if !info.SyncPointEnabled() { return From 96b9ae80ed1696f3c4a33b78d0615ab39ed1c8b1 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Fri, 23 Jan 2026 18:16:33 +0800 Subject: [PATCH 19/34] fix bug Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker_test.go | 4 ++-- pkg/eventservice/scan_window.go | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index f69c3dac4d..712b6d0b22 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -191,10 +191,10 @@ func TestScanRangeCappedByScanWindow(t *testing.T) { baseTime := time.Now() baseTs := oracle.GoTimeToTS(baseTime) - disp.checkpointTs.Store(baseTs) + disp.sentResolvedTs.Store(baseTs) disp.receivedResolvedTs.Store(oracle.GoTimeToTS(baseTime.Add(20 * time.Second))) disp.eventStoreCommitTs.Store(oracle.GoTimeToTS(baseTime.Add(15 * time.Second))) - changefeedStatus.refreshMinCheckpointTs() + changefeedStatus.refreshMinSentResolvedTs() needScan, dataRange := broker.getScanTaskDataRange(disp) require.True(t, needScan) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 5beab5ff3b..1b70fcadc9 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -137,11 +137,13 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { newInterval = minScanInterval case avg > memoryUsageHighThreshold: newInterval = maxDuration(current/2, minScanInterval) - case avg < memoryUsageLowThreshold: - newInterval = minDuration(current*2, maxInterval) case avg < memoryUsageVeryLowThreshold: + // When memory pressure is very low, allow the scan interval to grow beyond + // the sync point interval cap. maxInterval = maxScanInterval newInterval = minDuration(current*2, maxInterval) + case avg < memoryUsageLowThreshold: + newInterval = minDuration(current*2, maxInterval) } if newInterval != current { From c9df6b0f44c2c83b9b5dd458369d5e6be15aa84f Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 26 Jan 2026 15:03:48 +0800 Subject: [PATCH 20/34] comment hardcode scan interval Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 1b70fcadc9..0f0533cebe 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -205,7 +205,7 @@ func (c *changefeedStatus) getScanMaxTs() uint64 { interval = defaultScanInterval } - interval = defaultScanInterval + //interval = defaultScanInterval return oracle.GoTimeToTS(oracle.GetTimeFromTS(baseTs).Add(interval)) } From 2ebeef6b63b92a35a3fcb2290d7d944b37d03fb3 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 26 Jan 2026 16:45:54 +0800 Subject: [PATCH 21/34] add cool down Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 0f0533cebe..e44715587f 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -117,10 +117,9 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin } func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { - log.Info("fizz enter adjust scan interval", - zap.Stringer("changefeedID", c.changefeedID), - zap.Float64("avgUsage", avg), - ) + if time.Since(c.lastAdjustTime.Load()) < scanIntervalAdjustCooldown { + return + } current := time.Duration(c.scanInterval.Load()) if current <= 0 { @@ -205,8 +204,6 @@ func (c *changefeedStatus) getScanMaxTs() uint64 { interval = defaultScanInterval } - //interval = defaultScanInterval - return oracle.GoTimeToTS(oracle.GetTimeFromTS(baseTs).Add(interval)) } From abc00152696e779390c77d6d8c5a78db4b5108d6 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 26 Jan 2026 18:01:54 +0800 Subject: [PATCH 22/34] add metrics Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 11 +++++++++++ pkg/eventservice/scan_window.go | 3 +++ pkg/metrics/event_service.go | 16 ++++++++++++++++ 3 files changed, 30 insertions(+) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index a31a007c50..f34e08462a 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -1012,6 +1012,9 @@ func (c *eventBroker) addDispatcher(info DispatcherInfo) error { status.removeDispatcher(id) if status.isEmpty() { c.changefeedMap.Delete(changefeedID) + metrics.EventServiceAvailableMemoryQuotaGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowBaseTsGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowIntervalGaugeVec.DeleteLabelValues(changefeedID.String()) } c.sendNotReusableEvent(node.ID(info.GetServerID()), dispatcher) return nil @@ -1032,6 +1035,9 @@ func (c *eventBroker) addDispatcher(info DispatcherInfo) error { status.removeDispatcher(id) if status.isEmpty() { c.changefeedMap.Delete(changefeedID) + metrics.EventServiceAvailableMemoryQuotaGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowBaseTsGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowIntervalGaugeVec.DeleteLabelValues(changefeedID.String()) } return err } @@ -1082,6 +1088,8 @@ func (c *eventBroker) removeDispatcher(dispatcherInfo DispatcherInfo) { ) c.changefeedMap.Delete(changefeedID) metrics.EventServiceAvailableMemoryQuotaGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowBaseTsGaugeVec.DeleteLabelValues(changefeedID.String()) + metrics.EventServiceScanWindowIntervalGaugeVec.DeleteLabelValues(changefeedID.String()) } c.eventStore.UnregisterDispatcher(changefeedID, id) @@ -1180,6 +1188,7 @@ func (c *eventBroker) resetDispatcher(dispatcherInfo DispatcherInfo) error { if newStat.epoch > 1 { newStat.changefeedStat.scanInterval.Store(int64(defaultScanInterval)) + metrics.EventServiceScanWindowIntervalGaugeVec.WithLabelValues(changefeedID.String()).Set(defaultScanInterval.Seconds()) newStat.changefeedStat.lastAdjustTime.Store(time.Now()) } @@ -1201,6 +1210,8 @@ func (c *eventBroker) getOrSetChangefeedStatus(changefeedID common.ChangeFeedID) stat = newChangefeedStatus(changefeedID) log.Info("new changefeed status", zap.Stringer("changefeedID", changefeedID)) c.changefeedMap.Store(changefeedID, stat) + metrics.EventServiceScanWindowBaseTsGaugeVec.WithLabelValues(changefeedID.String()).Set(0) + metrics.EventServiceScanWindowIntervalGaugeVec.WithLabelValues(changefeedID.String()).Set(defaultScanInterval.Seconds()) } return stat.(*changefeedStatus) } diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index e44715587f..9ec8b21bd7 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/metrics" "github.com/tikv/client-go/v2/oracle" "go.uber.org/atomic" "go.uber.org/zap" @@ -147,6 +148,7 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { if newInterval != current { c.scanInterval.Store(int64(newInterval)) + metrics.EventServiceScanWindowIntervalGaugeVec.WithLabelValues(c.changefeedID.String()).Set(newInterval.Seconds()) c.lastAdjustTime.Store(now) log.Info("scan interval adjusted", zap.Stringer("changefeedID", c.changefeedID), @@ -213,6 +215,7 @@ func (c *changefeedStatus) storeMinSentTs(value uint64) { return } c.minSentTs.Store(value) + metrics.EventServiceScanWindowBaseTsGaugeVec.WithLabelValues(c.changefeedID.String()).Set(float64(value)) log.Info("scan window base updated", zap.Stringer("changefeedID", c.changefeedID), zap.Uint64("oldBaseTs", prev), diff --git a/pkg/metrics/event_service.go b/pkg/metrics/event_service.go index 763aa90f10..d7b47b04cb 100644 --- a/pkg/metrics/event_service.go +++ b/pkg/metrics/event_service.go @@ -55,6 +55,20 @@ var ( Name: "resolved_ts_lag", Help: "resolved ts lag of eventService in seconds", }, []string{"type"}) + EventServiceScanWindowBaseTsGaugeVec = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "event_service", + Name: "scan_window_base_ts", + Help: "The base ts of the scan window for each changefeed", + }, []string{"changefeed"}) + EventServiceScanWindowIntervalGaugeVec = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "event_service", + Name: "scan_window_interval", + Help: "The scan window interval in seconds for each changefeed", + }, []string{"changefeed"}) EventServiceScanDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "ticdc", @@ -178,6 +192,8 @@ func initEventServiceMetrics(registry *prometheus.Registry) { registry.MustRegister(EventServiceSendEventDuration) registry.MustRegister(EventServiceResolvedTsGauge) registry.MustRegister(EventServiceResolvedTsLagGauge) + registry.MustRegister(EventServiceScanWindowBaseTsGaugeVec) + registry.MustRegister(EventServiceScanWindowIntervalGaugeVec) registry.MustRegister(EventServiceScanDuration) registry.MustRegister(EventServiceScannedCount) registry.MustRegister(EventServiceDispatcherGauge) From 3d506ab7de26a322c1bad6a4a6b9cf59cd31dc92 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 26 Jan 2026 18:44:32 +0800 Subject: [PATCH 23/34] adjust scan window algorithm Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 13 +-- pkg/eventservice/event_broker_test.go | 2 + pkg/eventservice/scan_window.go | 129 ++++++++++++++++++++++---- pkg/eventservice/scan_window_test.go | 118 +++++++++++++++++++++++ 4 files changed, 237 insertions(+), 25 deletions(-) create mode 100644 pkg/eventservice/scan_window_test.go diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index f34e08462a..4c4643e16e 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -1187,9 +1187,10 @@ func (c *eventBroker) resetDispatcher(dispatcherInfo DispatcherInfo) error { } if newStat.epoch > 1 { + now := time.Now() newStat.changefeedStat.scanInterval.Store(int64(defaultScanInterval)) metrics.EventServiceScanWindowIntervalGaugeVec.WithLabelValues(changefeedID.String()).Set(defaultScanInterval.Seconds()) - newStat.changefeedStat.lastAdjustTime.Store(time.Now()) + newStat.changefeedStat.lastAdjustTime.Store(now) } log.Info("reset dispatcher", @@ -1280,14 +1281,14 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC c.changefeedMap.Range(func(k, v interface{}) bool { changefeedID := k.(common.ChangeFeedID) changefeed := v.(*changefeedStatus) - available, ok := holder[changefeedID.ID()] + availableInMsg, ok := holder[changefeedID.ID()] if ok { - changefeed.availableMemoryQuota.Store(from, atomic.NewUint64(available)) - metrics.EventServiceAvailableMemoryQuotaGaugeVec.WithLabelValues(changefeedID.String()).Set(float64(available)) + changefeed.availableMemoryQuota.Store(from, atomic.NewUint64(availableInMsg)) + metrics.EventServiceAvailableMemoryQuotaGaugeVec.WithLabelValues(changefeedID.String()).Set(float64(availableInMsg)) } if hasUsage { - if info, ok := usage[changefeedID.ID()]; ok { - changefeed.updateMemoryUsage(now, info.used, info.max) + if info, okUsage := usage[changefeedID.ID()]; okUsage && ok { + changefeed.updateMemoryUsage(now, info.used, info.max, availableInMsg) } } return true diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index 6adba64f96..8ebdaedc08 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -199,6 +199,8 @@ func TestScanRangeCappedByScanWindow(t *testing.T) { needScan, dataRange := broker.getScanTaskDataRange(disp) require.True(t, needScan) require.Equal(t, oracle.GoTimeToTS(baseTime.Add(defaultScanInterval)), dataRange.CommitTsEnd) +} + func TestDoScanSkipWhenChangefeedStatusNotFound(t *testing.T) { broker, _, _, _ := newEventBrokerForTest() broker.close() diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 9ec8b21bd7..59fe3cb318 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -47,6 +47,15 @@ type memoryUsageWindow struct { samples []memoryUsageSample } +type memoryUsageStats struct { + avg float64 + max float64 + first float64 + last float64 + span time.Duration + cnt int +} + func newMemoryUsageWindow(window time.Duration) *memoryUsageWindow { return &memoryUsageWindow{ window: window, @@ -79,6 +88,35 @@ func (w *memoryUsageWindow) average(now time.Time) float64 { return sum / float64(len(w.samples)) } +func (w *memoryUsageWindow) stats(now time.Time) memoryUsageStats { + w.mu.Lock() + defer w.mu.Unlock() + + w.pruneLocked(now) + if len(w.samples) == 0 { + return memoryUsageStats{} + } + + sum := 0.0 + firstRatio := w.samples[0].ratio + maxRatio := firstRatio + for _, sample := range w.samples { + sum += sample.ratio + if sample.ratio > maxRatio { + maxRatio = sample.ratio + } + } + + return memoryUsageStats{ + avg: sum / float64(len(w.samples)), + max: maxRatio, + first: firstRatio, + last: w.samples[len(w.samples)-1].ratio, + span: now.Sub(w.samples[0].ts), + cnt: len(w.samples), + } +} + func (w *memoryUsageWindow) pruneLocked(now time.Time) { cutoff := now.Add(-w.window) idx := 0 @@ -106,22 +144,37 @@ func (w *memoryUsageWindow) count() int { return len(w.samples) } -func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64) { +func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64, available uint64) { if max == 0 || c.usageWindow == nil { return } - ratio := float64(used) / float64(max) - c.usageWindow.addSample(now, ratio) - avg := c.usageWindow.average(now) - c.adjustScanInterval(now, avg) -} + pressure := float64(used) / float64(max) + if pressure < 0 { + pressure = 0 + } + if pressure > 1 { + pressure = 1 + } -func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { - if time.Since(c.lastAdjustTime.Load()) < scanIntervalAdjustCooldown { - return + availableRatio := float64(available) / float64(max) + if availableRatio < 0 { + availableRatio = 0 + } + if availableRatio > 1 { + availableRatio = 1 + } + pressureFromAvailable := 1 - availableRatio + if pressureFromAvailable > pressure { + pressure = pressureFromAvailable } + c.usageWindow.addSample(now, pressure) + stats := c.usageWindow.stats(now) + c.adjustScanInterval(now, stats) +} + +func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageStats) { current := time.Duration(c.scanInterval.Load()) if current <= 0 { current = defaultScanInterval @@ -131,19 +184,45 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { maxInterval = minScanInterval } + const ( + minTrendSamples = 4 + increasingTrendEpsilon = 0.02 + increasingTrendStrongDelta = 0.05 + ) + + trendDelta := usage.last - usage.first + isIncreasing := usage.cnt >= minTrendSamples && trendDelta > increasingTrendEpsilon + + allowedToIncrease := now.Sub(c.lastAdjustTime.Load()) >= scanIntervalAdjustCooldown && + usage.cnt > 0 && + usage.span >= memoryUsageWindowDuration && + !isIncreasing + newInterval := current switch { - case avg > memoryUsageCriticalThreshold: - newInterval = minScanInterval - case avg > memoryUsageHighThreshold: + case usage.last > memoryUsageCriticalThreshold || usage.max > memoryUsageCriticalThreshold: + newInterval = maxDuration(current/4, minScanInterval) + case usage.last > memoryUsageHighThreshold || usage.max > memoryUsageHighThreshold: + newInterval = maxDuration(current/2, minScanInterval) + case isIncreasing && trendDelta >= increasingTrendStrongDelta: + // When pressure keeps increasing, it usually indicates downstream can't keep up. + // Decrease scan interval proactively to avoid sudden memory quota exhaustion. newInterval = maxDuration(current/2, minScanInterval) - case avg < memoryUsageVeryLowThreshold: - // When memory pressure is very low, allow the scan interval to grow beyond - // the sync point interval cap. + case isIncreasing: + // Mild damping for increasing pressure. + newInterval = maxDuration(scaleDuration(current, 9, 10), minScanInterval) + case allowedToIncrease && usage.max < memoryUsageVeryLowThreshold && usage.avg < memoryUsageVeryLowThreshold: + // When memory pressure stays very low for a full window, allow the scan interval + // to grow beyond the sync point interval cap, but increase slowly to avoid burst. maxInterval = maxScanInterval - newInterval = minDuration(current*2, maxInterval) - case avg < memoryUsageLowThreshold: - newInterval = minDuration(current*2, maxInterval) + newInterval = minDuration(scaleDuration(current, 3, 2), maxInterval) + case allowedToIncrease && usage.max < memoryUsageLowThreshold && usage.avg < memoryUsageLowThreshold: + newInterval = minDuration(scaleDuration(current, 5, 4), maxInterval) + } + + // Prevent rapid oscillation: always apply decreases immediately, but throttle increases. + if newInterval > current && !allowedToIncrease { + return } if newInterval != current { @@ -155,7 +234,12 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, avg float64) { zap.Duration("oldInterval", current), zap.Duration("newInterval", newInterval), zap.Duration("maxInterval", maxInterval), - zap.Float64("avgUsage", avg), + zap.Float64("avgUsage", usage.avg), + zap.Float64("maxUsage", usage.max), + zap.Float64("firstUsage", usage.first), + zap.Float64("lastUsage", usage.last), + zap.Float64("trendDelta", trendDelta), + zap.Int("usageSamples", usage.cnt), zap.Bool("syncPointEnabled", c.syncPointEnabled.Load()), ) } @@ -256,3 +340,10 @@ func maxDuration(a time.Duration, b time.Duration) time.Duration { } return b } + +func scaleDuration(d time.Duration, numerator int64, denominator int64) time.Duration { + if numerator <= 0 || denominator <= 0 { + return d + } + return time.Duration(int64(d) * numerator / denominator) +} diff --git a/pkg/eventservice/scan_window_test.go b/pkg/eventservice/scan_window_test.go new file mode 100644 index 0000000000..8513627974 --- /dev/null +++ b/pkg/eventservice/scan_window_test.go @@ -0,0 +1,118 @@ +// Copyright 2025 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 eventservice + +import ( + "testing" + "time" + + "github.com/pingcap/ticdc/pkg/common" + "github.com/stretchr/testify/require" +) + +func TestAdjustScanIntervalVeryLowBypassesSyncPointCap(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + status.syncPointEnabled.Store(true) + status.syncPointInterval.Store(int64(1 * time.Minute)) + + now := time.Now() + status.lastAdjustTime.Store(now.Add(-scanIntervalAdjustCooldown - time.Second)) + + // Start from the sync point capped max interval, then allow it to grow slowly. + status.scanInterval.Store(int64(1 * time.Minute)) + + // Maintain a very low pressure for a full window to allow bypassing the sync point cap. + for i := 0; i <= int(memoryUsageWindowDuration/time.Second); i++ { + status.updateMemoryUsage(now.Add(time.Duration(i)*time.Second), 0, 100, 100) + } + require.Equal(t, int64(90*time.Second), status.scanInterval.Load()) +} + +func TestAdjustScanIntervalLowRespectsSyncPointCap(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + status.syncPointEnabled.Store(true) + status.syncPointInterval.Store(int64(1 * time.Minute)) + + now := time.Now() + status.lastAdjustTime.Store(now.Add(-scanIntervalAdjustCooldown - time.Second)) + + status.scanInterval.Store(int64(40 * time.Second)) + + for i := 0; i <= int(memoryUsageWindowDuration/time.Second); i++ { + status.updateMemoryUsage(now.Add(time.Duration(i)*time.Second), 15, 100, 100) + } + require.Equal(t, int64(50*time.Second), status.scanInterval.Load()) +} + +func TestAdjustScanIntervalDecreaseIgnoresCooldown(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + now := time.Now() + status.lastAdjustTime.Store(now) + + status.scanInterval.Store(int64(40 * time.Second)) + status.updateMemoryUsage(now.Add(memoryUsageWindowDuration), 80, 100, 100) + require.Equal(t, int64(20*time.Second), status.scanInterval.Load()) +} + +func TestAdjustScanIntervalUsesAvailableAsPressureSignal(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + now := time.Now() + status.lastAdjustTime.Store(now) + + status.scanInterval.Store(int64(40 * time.Second)) + + // used/max suggests low pressure, but available indicates full. + status.updateMemoryUsage(now.Add(memoryUsageWindowDuration), 10, 100, 0) + require.Equal(t, int64(10*time.Second), status.scanInterval.Load()) +} + +func TestAdjustScanIntervalDecreasesWhenUsageIncreasing(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + now := time.Now() + status.lastAdjustTime.Store(now) + + status.scanInterval.Store(int64(40 * time.Second)) + + status.updateMemoryUsage(now, 10, 100, 100) + status.updateMemoryUsage(now.Add(1*time.Second), 11, 100, 100) + status.updateMemoryUsage(now.Add(2*time.Second), 12, 100, 100) + status.updateMemoryUsage(now.Add(3*time.Second), 13, 100, 100) + require.Equal(t, int64(36*time.Second), status.scanInterval.Load()) +} + +func TestAdjustScanIntervalDecreasesMoreWhenUsageIncreasingFast(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + now := time.Now() + status.lastAdjustTime.Store(now) + + status.scanInterval.Store(int64(40 * time.Second)) + + status.updateMemoryUsage(now, 10, 100, 100) + status.updateMemoryUsage(now.Add(1*time.Second), 20, 100, 100) + status.updateMemoryUsage(now.Add(2*time.Second), 30, 100, 100) + status.updateMemoryUsage(now.Add(3*time.Second), 40, 100, 100) + require.Equal(t, int64(20*time.Second), status.scanInterval.Load()) +} From a0febabb7d3143ada1f68f288362834f8393804a Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 26 Jan 2026 18:58:42 +0800 Subject: [PATCH 24/34] adjust scan window algorithm 2 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 2 ++ pkg/eventservice/scan_window.go | 26 ++++++++++++++++---------- pkg/eventservice/scan_window_test.go | 15 ++++++++------- 3 files changed, 26 insertions(+), 17 deletions(-) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 528627b290..eb8f72c265 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -433,6 +433,7 @@ type changefeedStatus struct { lastRatio atomic.Float64 lastAdjustTime atomic.Time + lastTrendAdjustTime atomic.Time usageWindow *memoryUsageWindow syncPointEnabled atomic.Bool syncPointInterval atomic.Int64 @@ -447,6 +448,7 @@ func newChangefeedStatus(changefeedID common.ChangeFeedID) *changefeedStatus { } status.scanInterval.Store(int64(defaultScanInterval)) status.lastAdjustTime.Store(time.Now()) + status.lastTrendAdjustTime.Store(time.Now()) status.lastUsageLogTime.Store(time.Now()) status.lastUsageWindowLogTime.Store(time.Now()) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 59fe3cb318..e7f21501fb 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -29,6 +29,7 @@ const ( minScanInterval = 1 * time.Second maxScanInterval = 30 * time.Minute scanIntervalAdjustCooldown = 30 * time.Second + scanTrendAdjustCooldown = 5 * time.Second memoryUsageWindowDuration = 30 * time.Second memoryUsageHighThreshold = 0.7 memoryUsageCriticalThreshold = 0.9 @@ -185,32 +186,34 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageSt } const ( - minTrendSamples = 4 - increasingTrendEpsilon = 0.02 - increasingTrendStrongDelta = 0.05 + minTrendSamples = 4 + increasingTrendEpsilon = 0.02 + increasingTrendStartRatio = 0.3 ) trendDelta := usage.last - usage.first isIncreasing := usage.cnt >= minTrendSamples && trendDelta > increasingTrendEpsilon + isAboveTrendStart := usage.last > increasingTrendStartRatio + canAdjustOnTrend := now.Sub(c.lastTrendAdjustTime.Load()) >= scanTrendAdjustCooldown + shouldDampOnTrend := isAboveTrendStart && isIncreasing && canAdjustOnTrend allowedToIncrease := now.Sub(c.lastAdjustTime.Load()) >= scanIntervalAdjustCooldown && usage.cnt > 0 && usage.span >= memoryUsageWindowDuration && - !isIncreasing + !(isAboveTrendStart && isIncreasing) + adjustedOnTrend := false newInterval := current switch { case usage.last > memoryUsageCriticalThreshold || usage.max > memoryUsageCriticalThreshold: newInterval = maxDuration(current/4, minScanInterval) case usage.last > memoryUsageHighThreshold || usage.max > memoryUsageHighThreshold: newInterval = maxDuration(current/2, minScanInterval) - case isIncreasing && trendDelta >= increasingTrendStrongDelta: - // When pressure keeps increasing, it usually indicates downstream can't keep up. - // Decrease scan interval proactively to avoid sudden memory quota exhaustion. - newInterval = maxDuration(current/2, minScanInterval) - case isIncreasing: - // Mild damping for increasing pressure. + case shouldDampOnTrend: + // When pressure is above a safe level and still increasing, it usually indicates + // downstream can't keep up. Decrease scan interval gradually to avoid quota exhaustion. newInterval = maxDuration(scaleDuration(current, 9, 10), minScanInterval) + adjustedOnTrend = true case allowedToIncrease && usage.max < memoryUsageVeryLowThreshold && usage.avg < memoryUsageVeryLowThreshold: // When memory pressure stays very low for a full window, allow the scan interval // to grow beyond the sync point interval cap, but increase slowly to avoid burst. @@ -229,6 +232,9 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageSt c.scanInterval.Store(int64(newInterval)) metrics.EventServiceScanWindowIntervalGaugeVec.WithLabelValues(c.changefeedID.String()).Set(newInterval.Seconds()) c.lastAdjustTime.Store(now) + if adjustedOnTrend { + c.lastTrendAdjustTime.Store(now) + } log.Info("scan interval adjusted", zap.Stringer("changefeedID", c.changefeedID), zap.Duration("oldInterval", current), diff --git a/pkg/eventservice/scan_window_test.go b/pkg/eventservice/scan_window_test.go index 8513627974..32e33c495b 100644 --- a/pkg/eventservice/scan_window_test.go +++ b/pkg/eventservice/scan_window_test.go @@ -98,21 +98,22 @@ func TestAdjustScanIntervalDecreasesWhenUsageIncreasing(t *testing.T) { status.updateMemoryUsage(now.Add(1*time.Second), 11, 100, 100) status.updateMemoryUsage(now.Add(2*time.Second), 12, 100, 100) status.updateMemoryUsage(now.Add(3*time.Second), 13, 100, 100) - require.Equal(t, int64(36*time.Second), status.scanInterval.Load()) + require.Equal(t, int64(40*time.Second), status.scanInterval.Load()) } -func TestAdjustScanIntervalDecreasesMoreWhenUsageIncreasingFast(t *testing.T) { +func TestAdjustScanIntervalDecreasesWhenUsageIncreasingAboveThirtyPercent(t *testing.T) { t.Parallel() status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) now := time.Now() status.lastAdjustTime.Store(now) + status.lastTrendAdjustTime.Store(now.Add(-scanTrendAdjustCooldown - time.Second)) status.scanInterval.Store(int64(40 * time.Second)) - status.updateMemoryUsage(now, 10, 100, 100) - status.updateMemoryUsage(now.Add(1*time.Second), 20, 100, 100) - status.updateMemoryUsage(now.Add(2*time.Second), 30, 100, 100) - status.updateMemoryUsage(now.Add(3*time.Second), 40, 100, 100) - require.Equal(t, int64(20*time.Second), status.scanInterval.Load()) + status.updateMemoryUsage(now, 31, 100, 100) + status.updateMemoryUsage(now.Add(1*time.Second), 32, 100, 100) + status.updateMemoryUsage(now.Add(2*time.Second), 33, 100, 100) + status.updateMemoryUsage(now.Add(3*time.Second), 34, 100, 100) + require.Equal(t, int64(36*time.Second), status.scanInterval.Load()) } From 460b74d733a9ec9040247e07e16b9b21fc9756a8 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 27 Jan 2026 09:48:19 +0800 Subject: [PATCH 25/34] adjust scan window algorithm 3 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 9 +++++++-- pkg/eventservice/scan_window_test.go | 21 +++++++++++++++++++++ 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index e7f21501fb..23e6c7d0c2 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -189,6 +189,10 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageSt minTrendSamples = 4 increasingTrendEpsilon = 0.02 increasingTrendStartRatio = 0.3 + + minIncreaseSamples = 10 + minIncreaseSpanNumerator = 4 + minIncreaseSpanDenominator = 5 ) trendDelta := usage.last - usage.first @@ -197,9 +201,10 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageSt canAdjustOnTrend := now.Sub(c.lastTrendAdjustTime.Load()) >= scanTrendAdjustCooldown shouldDampOnTrend := isAboveTrendStart && isIncreasing && canAdjustOnTrend + minIncreaseSpan := memoryUsageWindowDuration * minIncreaseSpanNumerator / minIncreaseSpanDenominator allowedToIncrease := now.Sub(c.lastAdjustTime.Load()) >= scanIntervalAdjustCooldown && - usage.cnt > 0 && - usage.span >= memoryUsageWindowDuration && + usage.cnt >= minIncreaseSamples && + usage.span >= minIncreaseSpan && !(isAboveTrendStart && isIncreasing) adjustedOnTrend := false diff --git a/pkg/eventservice/scan_window_test.go b/pkg/eventservice/scan_window_test.go index 32e33c495b..a2d8d19561 100644 --- a/pkg/eventservice/scan_window_test.go +++ b/pkg/eventservice/scan_window_test.go @@ -85,6 +85,27 @@ func TestAdjustScanIntervalUsesAvailableAsPressureSignal(t *testing.T) { require.Equal(t, int64(10*time.Second), status.scanInterval.Load()) } +func TestAdjustScanIntervalIncreaseWithJitteredSamples(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + status.syncPointEnabled.Store(true) + status.syncPointInterval.Store(int64(1 * time.Minute)) + + start := time.Now() + status.lastAdjustTime.Store(start.Add(-scanIntervalAdjustCooldown - time.Second)) + + status.scanInterval.Store(int64(40 * time.Second)) + + // Use a >1s interval to simulate heartbeat jitter, so the window span will be + // slightly less than memoryUsageWindowDuration. + step := 1100 * time.Millisecond + for i := 0; i < 28; i++ { + status.updateMemoryUsage(start.Add(time.Duration(i)*step), 15, 100, 100) + } + require.Equal(t, int64(50*time.Second), status.scanInterval.Load()) +} + func TestAdjustScanIntervalDecreasesWhenUsageIncreasing(t *testing.T) { t.Parallel() From f8c34a14dd688b93b8a225cf86bf9a4468ecb481 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 2 Feb 2026 11:46:44 +0800 Subject: [PATCH 26/34] add comment Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/scan_window.go | 46 ++++++++++++++++++++++++++------- 1 file changed, 36 insertions(+), 10 deletions(-) diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index 23e6c7d0c2..b2fac8d930 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -175,6 +175,21 @@ func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uin c.adjustScanInterval(now, stats) } +// adjustScanInterval dynamically adjusts the scan interval based on memory pressure. +// +// Algorithm overview: +// - "Fast brake, slow accelerate": Decreases are applied immediately when memory +// pressure is high, while increases require cooldown periods and stable conditions. +// - Tiered response: Different thresholds trigger different adjustment magnitudes. +// - Trend prediction: Detects rising memory pressure early and proactively reduces +// the interval before hitting critical thresholds. +// +// Thresholds and actions: +// - Critical (>80%): Reduce interval to 1/4 (aggressive) +// - High (>60%): Reduce interval to 1/2 +// - Trend damping (>30% AND rising): Reduce interval by 10% +// - Low (<40% max AND avg): Increase interval by 25% +// - Very low (<20% max AND avg): Increase interval by 50%, may exceed normal cap func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageStats) { current := time.Duration(c.scanInterval.Load()) if current <= 0 { @@ -185,50 +200,61 @@ func (c *changefeedStatus) adjustScanInterval(now time.Time, usage memoryUsageSt maxInterval = minScanInterval } + // Constants for trend detection and increase eligibility. const ( - minTrendSamples = 4 - increasingTrendEpsilon = 0.02 - increasingTrendStartRatio = 0.3 + minTrendSamples = 4 // Minimum samples needed to detect a valid trend + increasingTrendEpsilon = 0.02 // Minimum delta to consider as "increasing" + increasingTrendStartRatio = 0.3 // Threshold (30%) above which trend damping kicks in - minIncreaseSamples = 10 - minIncreaseSpanNumerator = 4 + minIncreaseSamples = 10 // Minimum samples needed before allowing increase + minIncreaseSpanNumerator = 4 // Observation span must be at least 4/5 of window minIncreaseSpanDenominator = 5 ) + // Trend detection: check if memory usage is rising over the observation window. + // This enables proactive intervention before hitting high thresholds. trendDelta := usage.last - usage.first isIncreasing := usage.cnt >= minTrendSamples && trendDelta > increasingTrendEpsilon isAboveTrendStart := usage.last > increasingTrendStartRatio canAdjustOnTrend := now.Sub(c.lastTrendAdjustTime.Load()) >= scanTrendAdjustCooldown shouldDampOnTrend := isAboveTrendStart && isIncreasing && canAdjustOnTrend + // Increase eligibility: conservative conditions to prevent oscillation. + // Requires: cooldown passed, enough samples, sufficient observation span, + // and NOT in an increasing trend situation (to avoid fighting against pressure). minIncreaseSpan := memoryUsageWindowDuration * minIncreaseSpanNumerator / minIncreaseSpanDenominator allowedToIncrease := now.Sub(c.lastAdjustTime.Load()) >= scanIntervalAdjustCooldown && usage.cnt >= minIncreaseSamples && usage.span >= minIncreaseSpan && !(isAboveTrendStart && isIncreasing) + // Determine the new interval based on memory pressure levels. + // Priority order: critical > high > trend damping > very low > low adjustedOnTrend := false newInterval := current switch { case usage.last > memoryUsageCriticalThreshold || usage.max > memoryUsageCriticalThreshold: + // Critical pressure: aggressive reduction to 1/4 newInterval = maxDuration(current/4, minScanInterval) case usage.last > memoryUsageHighThreshold || usage.max > memoryUsageHighThreshold: + // High pressure: reduce to 1/2 newInterval = maxDuration(current/2, minScanInterval) case shouldDampOnTrend: - // When pressure is above a safe level and still increasing, it usually indicates - // downstream can't keep up. Decrease scan interval gradually to avoid quota exhaustion. + // Trend damping: pressure is moderate (>30%) but rising. Reduce by 10% to + // preemptively slow down before downstream gets overwhelmed. newInterval = maxDuration(scaleDuration(current, 9, 10), minScanInterval) adjustedOnTrend = true case allowedToIncrease && usage.max < memoryUsageVeryLowThreshold && usage.avg < memoryUsageVeryLowThreshold: - // When memory pressure stays very low for a full window, allow the scan interval - // to grow beyond the sync point interval cap, but increase slowly to avoid burst. + // Very low pressure (<20%): increase by 50%, allowed to exceed sync point cap. maxInterval = maxScanInterval newInterval = minDuration(scaleDuration(current, 3, 2), maxInterval) case allowedToIncrease && usage.max < memoryUsageLowThreshold && usage.avg < memoryUsageLowThreshold: + // Low pressure (<40%): increase by 25%, capped by sync point interval. newInterval = minDuration(scaleDuration(current, 5, 4), maxInterval) } - // Prevent rapid oscillation: always apply decreases immediately, but throttle increases. + // Anti-oscillation guard: decreases are always applied immediately, + // but increases are blocked if cooldown conditions aren't met. if newInterval > current && !allowedToIncrease { return } From 1a86e53168e2ac250b6615d95d32f56a00e7cd5a Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 2 Feb 2026 16:49:19 +0800 Subject: [PATCH 27/34] remove useless codes Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker_test.go | 35 ++++++++ pkg/eventservice/scan_window.go | 89 ++++++++++---------- pkg/eventservice/scan_window_test.go | 110 +++++++++++++++++++++++++ tools/workload/Makefile | 4 +- tools/workload/app.go | 12 ++- tools/workload/config.go | 24 +++++- tools/workload/go.mod | 1 + tools/workload/go.sum | 2 + tools/workload/readme.md | 45 ++++++++++ tools/workload/statistics.go | 24 +++++- utils/dynstream/memory_control_test.go | 65 ++++++++++----- 11 files changed, 337 insertions(+), 74 deletions(-) diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index 8ebdaedc08..ea07153f1f 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -201,6 +201,41 @@ func TestScanRangeCappedByScanWindow(t *testing.T) { require.Equal(t, oracle.GoTimeToTS(baseTime.Add(defaultScanInterval)), dataRange.CommitTsEnd) } +func TestHandleCongestionControlV2AdjustsScanInterval(t *testing.T) { + broker, _, _, _ := newEventBrokerForTest() + defer broker.close() + + changefeedID := common.NewChangefeedID4Test("default", "test") + status := broker.getOrSetChangefeedStatus(changefeedID) + + status.scanInterval.Store(int64(40 * time.Second)) + status.lastAdjustTime.Store(time.Now()) + + control := event.NewCongestionControlWithVersion(event.CongestionControlVersion2) + // Simulate critical pressure by setting available to 0. + control.AddAvailableMemoryWithDispatchersAndUsage(changefeedID.ID(), 0, 10, 100, nil) + broker.handleCongestionControl(node.ID("event-collector-1"), control) + + require.Equal(t, int64(10*time.Second), status.scanInterval.Load()) +} + +func TestHandleCongestionControlV1DoesNotAdjustScanInterval(t *testing.T) { + broker, _, _, _ := newEventBrokerForTest() + defer broker.close() + + changefeedID := common.NewChangefeedID4Test("default", "test") + status := broker.getOrSetChangefeedStatus(changefeedID) + + status.scanInterval.Store(int64(40 * time.Second)) + status.lastAdjustTime.Store(time.Now()) + + control := event.NewCongestionControl() + control.AddAvailableMemoryWithDispatchers(changefeedID.ID(), 0, nil) + broker.handleCongestionControl(node.ID("event-collector-1"), control) + + require.Equal(t, int64(40*time.Second), status.scanInterval.Load()) +} + func TestDoScanSkipWhenChangefeedStatusNotFound(t *testing.T) { broker, _, _, _ := newEventBrokerForTest() broker.close() diff --git a/pkg/eventservice/scan_window.go b/pkg/eventservice/scan_window.go index b2fac8d930..502a4dbc9e 100644 --- a/pkg/eventservice/scan_window.go +++ b/pkg/eventservice/scan_window.go @@ -25,16 +25,51 @@ import ( ) const ( - defaultScanInterval = 5 * time.Second - minScanInterval = 1 * time.Second - maxScanInterval = 30 * time.Minute - scanIntervalAdjustCooldown = 30 * time.Second - scanTrendAdjustCooldown = 5 * time.Second - memoryUsageWindowDuration = 30 * time.Second - memoryUsageHighThreshold = 0.7 + // defaultScanInterval is the initial scan interval used when starting up + // or when the current interval is invalid. + defaultScanInterval = 5 * time.Second + + // minScanInterval is the minimum allowed scan interval. Even under critical + // memory pressure, the interval will never go below this value. + minScanInterval = 1 * time.Second + + // maxScanInterval is the maximum allowed scan interval. Even under very low + // memory pressure, the interval will never exceed this value. + maxScanInterval = 30 * time.Minute + + // scanIntervalAdjustCooldown is the minimum time that must pass between + // scan interval increases. This prevents oscillation by enforcing a waiting + // period before allowing another increase. Decreases are not affected by + // this cooldown and are applied immediately. + scanIntervalAdjustCooldown = 30 * time.Second + + // scanTrendAdjustCooldown is the minimum time between trend-based interval + // adjustments. This is shorter than the general cooldown because trend + // adjustments need to be more responsive to rising memory pressure. + scanTrendAdjustCooldown = 5 * time.Second + + // memoryUsageWindowDuration is the duration of the sliding window for + // collecting memory usage samples. Samples older than this duration are + // pruned from the window. + memoryUsageWindowDuration = 30 * time.Second + + // memoryUsageHighThreshold (70%) triggers a moderate reduction of the scan + // interval to 1/2 of its current value when memory usage exceeds this level. + memoryUsageHighThreshold = 0.7 + + // memoryUsageCriticalThreshold (90%) triggers an aggressive reduction of + // the scan interval to 1/4 of its current value when memory usage exceeds + // this level. memoryUsageCriticalThreshold = 0.9 - memoryUsageLowThreshold = 0.2 - memoryUsageVeryLowThreshold = 0.1 + + // memoryUsageLowThreshold (20%) allows the scan interval to be increased + // by 25% when both max and average memory usage are below this level. + memoryUsageLowThreshold = 0.2 + + // memoryUsageVeryLowThreshold (10%) allows the scan interval to be increased + // by 50% when both max and average memory usage are below this level. This + // increase may exceed the normal sync point interval cap. + memoryUsageVeryLowThreshold = 0.1 ) type memoryUsageSample struct { @@ -74,21 +109,6 @@ func (w *memoryUsageWindow) addSample(now time.Time, ratio float64) { w.pruneLocked(now) } -func (w *memoryUsageWindow) average(now time.Time) float64 { - w.mu.Lock() - defer w.mu.Unlock() - - w.pruneLocked(now) - if len(w.samples) == 0 { - return 0 - } - var sum float64 - for _, sample := range w.samples { - sum += sample.ratio - } - return sum / float64(len(w.samples)) -} - func (w *memoryUsageWindow) stats(now time.Time) memoryUsageStats { w.mu.Lock() defer w.mu.Unlock() @@ -129,22 +149,6 @@ func (w *memoryUsageWindow) pruneLocked(now time.Time) { } } -func (w *memoryUsageWindow) span(now time.Time) time.Duration { - w.mu.Lock() - defer w.mu.Unlock() - w.pruneLocked(now) - if len(w.samples) == 0 { - return 0 - } - return now.Sub(w.samples[0].ts) -} - -func (w *memoryUsageWindow) count() int { - w.mu.Lock() - defer w.mu.Unlock() - return len(w.samples) -} - func (c *changefeedStatus) updateMemoryUsage(now time.Time, used uint64, max uint64, available uint64) { if max == 0 || c.usageWindow == nil { return @@ -337,11 +341,6 @@ func (c *changefeedStatus) storeMinSentTs(value uint64) { } c.minSentTs.Store(value) metrics.EventServiceScanWindowBaseTsGaugeVec.WithLabelValues(c.changefeedID.String()).Set(float64(value)) - log.Info("scan window base updated", - zap.Stringer("changefeedID", c.changefeedID), - zap.Uint64("oldBaseTs", prev), - zap.Uint64("newBaseTs", value), - ) } func (c *changefeedStatus) updateSyncPointConfig(info DispatcherInfo) { diff --git a/pkg/eventservice/scan_window_test.go b/pkg/eventservice/scan_window_test.go index a2d8d19561..6c6c0555e0 100644 --- a/pkg/eventservice/scan_window_test.go +++ b/pkg/eventservice/scan_window_test.go @@ -17,8 +17,11 @@ import ( "testing" "time" + "github.com/pingcap/ticdc/eventpb" "github.com/pingcap/ticdc/pkg/common" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" + "go.uber.org/atomic" ) func TestAdjustScanIntervalVeryLowBypassesSyncPointCap(t *testing.T) { @@ -138,3 +141,110 @@ func TestAdjustScanIntervalDecreasesWhenUsageIncreasingAboveThirtyPercent(t *tes status.updateMemoryUsage(now.Add(3*time.Second), 34, 100, 100) require.Equal(t, int64(36*time.Second), status.scanInterval.Load()) } + +func TestRefreshMinSentResolvedTsMinAndSkipRules(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + + removed := &dispatcherStat{} + removed.seq.Store(1) + removed.sentResolvedTs.Store(150) + removed.isRemoved.Store(true) + + uninitialized := &dispatcherStat{} + uninitialized.seq.Store(0) + uninitialized.sentResolvedTs.Store(10) + + first := &dispatcherStat{} + first.seq.Store(1) + first.sentResolvedTs.Store(200) + + second := &dispatcherStat{} + second.seq.Store(1) + second.sentResolvedTs.Store(50) + + removedPtr := &atomic.Pointer[dispatcherStat]{} + removedPtr.Store(removed) + status.addDispatcher(common.NewDispatcherID(), removedPtr) + + uninitializedPtr := &atomic.Pointer[dispatcherStat]{} + uninitializedPtr.Store(uninitialized) + status.addDispatcher(common.NewDispatcherID(), uninitializedPtr) + + firstPtr := &atomic.Pointer[dispatcherStat]{} + firstPtr.Store(first) + status.addDispatcher(common.NewDispatcherID(), firstPtr) + + secondPtr := &atomic.Pointer[dispatcherStat]{} + secondPtr.Store(second) + status.addDispatcher(common.NewDispatcherID(), secondPtr) + + status.refreshMinSentResolvedTs() + require.Equal(t, uint64(50), status.minSentTs.Load()) + + second.isRemoved.Store(true) + status.refreshMinSentResolvedTs() + require.Equal(t, uint64(200), status.minSentTs.Load()) + + first.seq.Store(0) + status.refreshMinSentResolvedTs() + require.Equal(t, uint64(0), status.minSentTs.Load()) +} + +func TestGetScanMaxTsFallbackInterval(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + + baseTime := time.Unix(1234, 0) + baseTs := oracle.GoTimeToTS(baseTime) + status.minSentTs.Store(baseTs) + + status.scanInterval.Store(0) + require.Equal(t, oracle.GoTimeToTS(baseTime.Add(defaultScanInterval)), status.getScanMaxTs()) + + status.scanInterval.Store(int64(10 * time.Second)) + require.Equal(t, oracle.GoTimeToTS(baseTime.Add(10*time.Second)), status.getScanMaxTs()) + + status.minSentTs.Store(0) + require.Equal(t, uint64(0), status.getScanMaxTs()) +} + +func TestUpdateSyncPointConfigUsesMinimumInterval(t *testing.T) { + t.Parallel() + + status := newChangefeedStatus(common.NewChangefeedID4Test("default", "test")) + + disabled := newMockDispatcherInfo(t, 0, common.NewDispatcherID(), 1, eventpb.ActionType_ACTION_TYPE_REGISTER) + disabled.enableSyncPoint = false + disabled.syncPointInterval = 2 * time.Minute + status.updateSyncPointConfig(disabled) + require.False(t, status.syncPointEnabled.Load()) + require.Equal(t, int64(0), status.syncPointInterval.Load()) + + first := newMockDispatcherInfo(t, 0, common.NewDispatcherID(), 1, eventpb.ActionType_ACTION_TYPE_REGISTER) + first.enableSyncPoint = true + first.syncPointInterval = 2 * time.Minute + status.updateSyncPointConfig(first) + require.True(t, status.syncPointEnabled.Load()) + require.Equal(t, int64(2*time.Minute), status.syncPointInterval.Load()) + + second := newMockDispatcherInfo(t, 0, common.NewDispatcherID(), 1, eventpb.ActionType_ACTION_TYPE_REGISTER) + second.enableSyncPoint = true + second.syncPointInterval = 1 * time.Minute + status.updateSyncPointConfig(second) + require.Equal(t, int64(1*time.Minute), status.syncPointInterval.Load()) + + third := newMockDispatcherInfo(t, 0, common.NewDispatcherID(), 1, eventpb.ActionType_ACTION_TYPE_REGISTER) + third.enableSyncPoint = true + third.syncPointInterval = 3 * time.Minute + status.updateSyncPointConfig(third) + require.Equal(t, int64(1*time.Minute), status.syncPointInterval.Load()) + + invalid := newMockDispatcherInfo(t, 0, common.NewDispatcherID(), 1, eventpb.ActionType_ACTION_TYPE_REGISTER) + invalid.enableSyncPoint = true + invalid.syncPointInterval = 0 + status.updateSyncPointConfig(invalid) + require.Equal(t, int64(1*time.Minute), status.syncPointInterval.Load()) +} diff --git a/tools/workload/Makefile b/tools/workload/Makefile index 498c32262e..9b8f819c87 100644 --- a/tools/workload/Makefile +++ b/tools/workload/Makefile @@ -7,10 +7,10 @@ all: build build: @echo "Building $(BINARY_NAME)..." @mkdir -p $(OUTPUT_DIR) - @go build -o $(OUTPUT_DIR)/$(BINARY_NAME) . + @go build -mod=vendor -o $(OUTPUT_DIR)/$(BINARY_NAME) . clean: @echo "Cleaning up..." @rm -rf $(OUTPUT_DIR) -.PHONY: all build clean \ No newline at end of file +.PHONY: all build clean diff --git a/tools/workload/app.go b/tools/workload/app.go index 47f9f0629b..3c37b7f57d 100644 --- a/tools/workload/app.go +++ b/tools/workload/app.go @@ -56,6 +56,10 @@ type WorkloadStats struct { QueryCount atomic.Uint64 ErrorCount atomic.Uint64 CreatedTableNum atomic.Int32 + DDLExecuted atomic.Uint64 + DDLSucceeded atomic.Uint64 + DDLSkipped atomic.Uint64 + DDLFailed atomic.Uint64 } // WorkloadApp is the main structure of the application @@ -161,12 +165,12 @@ func (app *WorkloadApp) executeWorkload(wg *sync.WaitGroup) error { zap.Int("dbCount", len(app.DBManager.GetDBs())), zap.Int("tableCount", app.Config.TableCount)) - if !app.Config.SkipCreateTable && app.Config.Action == "prepare" { - app.handlePrepareAction(insertConcurrency, wg) - return nil + if app.Config.Action == "ddl" || app.Config.OnlyDDL { + return app.handleDDLExecution(wg) } - if app.Config.OnlyDDL { + if !app.Config.SkipCreateTable && app.Config.Action == "prepare" { + app.handlePrepareAction(insertConcurrency, wg) return nil } diff --git a/tools/workload/config.go b/tools/workload/config.go index e79bc63ea2..7fca1636f6 100644 --- a/tools/workload/config.go +++ b/tools/workload/config.go @@ -17,6 +17,7 @@ import ( "flag" "fmt" "sync" + "time" ) // WorkloadConfig saves all the configurations for the workload @@ -44,6 +45,9 @@ type WorkloadConfig struct { Action string SkipCreateTable bool OnlyDDL bool + DDLConfigPath string + DDLWorker int + DDLTimeout time.Duration // Special workload config RowSize int @@ -85,6 +89,9 @@ func NewWorkloadConfig() *WorkloadConfig { Action: "prepare", SkipCreateTable: false, OnlyDDL: false, + DDLConfigPath: "", + DDLWorker: 1, + DDLTimeout: 2 * time.Minute, // For large row workload RowSize: 10240, @@ -115,7 +122,7 @@ func (c *WorkloadConfig) ParseFlags() error { flag.Float64Var(&c.PercentageForUpdate, "percentage-for-update", c.PercentageForUpdate, "percentage for update: [0, 1.0]") flag.Float64Var(&c.PercentageForDelete, "percentage-for-delete", c.PercentageForDelete, "percentage for delete: [0, 1.0]") flag.BoolVar(&c.SkipCreateTable, "skip-create-table", c.SkipCreateTable, "do not create tables") - flag.StringVar(&c.Action, "action", c.Action, "action of the workload: [prepare, insert, update, delete, write, cleanup]") + flag.StringVar(&c.Action, "action", c.Action, "action of the workload: [prepare, insert, update, delete, write, ddl, cleanup]") flag.StringVar(&c.WorkloadType, "workload-type", c.WorkloadType, "workload type: [bank, sysbench, large_row, shop_item, uuu, bank2, bank_update, crawler, dc]") flag.StringVar(&c.DBHost, "database-host", c.DBHost, "database host") flag.StringVar(&c.DBUser, "database-user", c.DBUser, "database user") @@ -123,6 +130,9 @@ func (c *WorkloadConfig) ParseFlags() error { flag.StringVar(&c.DBName, "database-db-name", c.DBName, "database db name") flag.IntVar(&c.DBPort, "database-port", c.DBPort, "database port") flag.BoolVar(&c.OnlyDDL, "only-ddl", c.OnlyDDL, "only generate ddl") + flag.StringVar(&c.DDLConfigPath, "ddl-config", c.DDLConfigPath, "ddl config file path, must be .toml") + flag.IntVar(&c.DDLWorker, "ddl-worker", c.DDLWorker, "ddl worker concurrency") + flag.DurationVar(&c.DDLTimeout, "ddl-timeout", c.DDLTimeout, "timeout for each ddl statement") flag.StringVar(&c.LogFile, "log-file", c.LogFile, "log file path") flag.StringVar(&c.LogLevel, "log-level", c.LogLevel, "log file path") // For large row workload @@ -146,6 +156,18 @@ func (c *WorkloadConfig) ParseFlags() error { c.PercentageForUpdate, c.PercentageForDelete) } + if c.Action == "ddl" || c.OnlyDDL { + if c.DDLConfigPath == "" { + return fmt.Errorf("ddl requires -ddl-config") + } + if c.DDLWorker <= 0 { + return fmt.Errorf("ddl-worker must be > 0") + } + if c.DDLTimeout <= 0 { + return fmt.Errorf("ddl-timeout must be > 0") + } + } + return nil } diff --git a/tools/workload/go.mod b/tools/workload/go.mod index 049ab6afad..55f7985927 100644 --- a/tools/workload/go.mod +++ b/tools/workload/go.mod @@ -3,6 +3,7 @@ module workload go 1.25.5 require ( + github.com/BurntSushi/toml v1.5.0 github.com/go-sql-driver/mysql v1.8.1 github.com/google/uuid v1.6.0 github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f diff --git a/tools/workload/go.sum b/tools/workload/go.sum index a53ae788d5..80d8ecf199 100644 --- a/tools/workload/go.sum +++ b/tools/workload/go.sum @@ -1,5 +1,7 @@ filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= +github.com/BurntSushi/toml v1.5.0 h1:W5quZX/G/csjUnuI8SUYlsHs9M38FC7znL0lIO+DvMg= +github.com/BurntSushi/toml v1.5.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= diff --git a/tools/workload/readme.md b/tools/workload/readme.md index 5b27407923..7a49cb3120 100644 --- a/tools/workload/readme.md +++ b/tools/workload/readme.md @@ -17,6 +17,51 @@ make ## Common Usage Scenarios +### 0. DDL Workload + +Run DDL workload based on a TOML config file: + +```bash +./workload -action ddl \ + -database-host 127.0.0.1 \ + -database-port 4000 \ + -database-db-name test \ + -ddl-config ./ddl.toml \ + -ddl-worker 1 \ + -ddl-timeout 2m +``` + +`ddl.toml` example (fixed mode): + +```toml +mode = "fixed" + +tables = [ + "test.sbtest1", + "test.sbtest2", +] + +[rate_per_minute] +add_column = 10 +drop_column = 10 +add_index = 5 +drop_index = 5 +truncate_table = 1 +``` + +`ddl.toml` example (random mode, omit `tables`): + +```toml +mode = "random" + +[rate_per_minute] +add_column = 10 +drop_column = 10 +add_index = 5 +drop_index = 5 +truncate_table = 0 +``` + ### 1. Sysbench-style Data Insertion Insert test data using sysbench-compatible schema: diff --git a/tools/workload/statistics.go b/tools/workload/statistics.go index 2835af3949..815727ed08 100644 --- a/tools/workload/statistics.go +++ b/tools/workload/statistics.go @@ -27,12 +27,18 @@ type statistics struct { queryCount uint64 flushedRowCount uint64 errCount uint64 + ddlExecuted uint64 + ddlSucceeded uint64 + ddlSkipped uint64 + ddlFailed uint64 // QPS qps int // row/s rps int // error/s eps int + // ddl/s + ddls int } // calculateStats calculates the statistics @@ -40,33 +46,45 @@ func (app *WorkloadApp) calculateStats( lastQueryCount, lastFlushed, lastErrors uint64, + lastDDLExecuted uint64, reportInterval time.Duration, ) statistics { currentFlushed := app.Stats.FlushedRowCount.Load() currentErrors := app.Stats.ErrorCount.Load() currentQueryCount := app.Stats.QueryCount.Load() + currentDDLExecuted := app.Stats.DDLExecuted.Load() return statistics{ queryCount: currentQueryCount, flushedRowCount: currentFlushed, errCount: currentErrors, + ddlExecuted: currentDDLExecuted, + ddlSucceeded: app.Stats.DDLSucceeded.Load(), + ddlSkipped: app.Stats.DDLSkipped.Load(), + ddlFailed: app.Stats.DDLFailed.Load(), qps: int(currentQueryCount-lastQueryCount) / int(reportInterval.Seconds()), rps: int(currentFlushed-lastFlushed) / int(reportInterval.Seconds()), eps: int(currentErrors-lastErrors) / int(reportInterval.Seconds()), + ddls: int(currentDDLExecuted-lastDDLExecuted) / int(reportInterval.Seconds()), } } // printStats prints the statistics func (app *WorkloadApp) printStats(stats statistics) { status := fmt.Sprintf( - "Total Write Rows: %d, Total Queries: %d, Total Created Tables: %d, Total Errors: %d, QPS: %d, Row/s: %d, Error/s: %d", + "Total Write Rows: %d, Total Queries: %d, Total Created Tables: %d, Total Errors: %d, Total DDL Executed: %d, Total DDL Succeeded: %d, Total DDL Skipped: %d, Total DDL Failed: %d, QPS: %d, Row/s: %d, Error/s: %d, DDL/s: %d", stats.flushedRowCount, stats.queryCount, app.Stats.CreatedTableNum.Load(), stats.errCount, + stats.ddlExecuted, + stats.ddlSucceeded, + stats.ddlSkipped, + stats.ddlFailed, stats.qps, stats.rps, stats.eps, + stats.ddls, ) plog.Info(status) } @@ -85,14 +103,16 @@ func (app *WorkloadApp) reportMetrics() { lastQueryCount uint64 lastFlushed uint64 lastErrorCount uint64 + lastDDLCount uint64 ) for range ticker.C { - stats := app.calculateStats(lastQueryCount, lastFlushed, lastErrorCount, reportInterval) + stats := app.calculateStats(lastQueryCount, lastFlushed, lastErrorCount, lastDDLCount, reportInterval) // Update last values for next iteration lastQueryCount = stats.queryCount lastFlushed = stats.flushedRowCount lastErrorCount = stats.errCount + lastDDLCount = stats.ddlExecuted // Print statistics app.printStats(stats) diff --git a/utils/dynstream/memory_control_test.go b/utils/dynstream/memory_control_test.go index 7c8f7226f3..b3c8a1edd3 100644 --- a/utils/dynstream/memory_control_test.go +++ b/utils/dynstream/memory_control_test.go @@ -272,6 +272,25 @@ func TestReleaseMemory(t *testing.T) { } feedbackChan := make(chan Feedback[int, string, any], 10) + calcExpectedReleasedPaths := func( + as *areaMemStat[int, string, *mockEvent, any, *mockHandler], + paths ...*pathInfo[int, string, *mockEvent, any, *mockHandler], + ) []string { + sizeToRelease := int64(float64(as.totalPendingSize.Load()) * defaultReleaseMemoryRatio) + releasedSize := int64(0) + res := make([]string, 0) + for _, path := range paths { + if releasedSize >= sizeToRelease || + path.pendingSize.Load() < int64(defaultReleaseMemoryThreshold) || + !path.blocking.Load() { + continue + } + releasedSize += path.pendingSize.Load() + res = append(res, path.path) + } + return res + } + // Create 3 paths with different last handle event timestamps path1 := &pathInfo[int, string, *mockEvent, any, *mockHandler]{ area: area, @@ -310,7 +329,7 @@ func TestReleaseMemory(t *testing.T) { path2.lastHandleEventTs.Store(200) path3.lastHandleEventTs.Store(100) - // Case 1: release path1 + // Case 1: release most recent paths // Add events to each path // Each event has size 100 for i := 0; i < 4; i++ { @@ -355,20 +374,27 @@ func TestReleaseMemory(t *testing.T) { path1.areaMemStat.lastReleaseMemoryTime.Store(time.Now().Add(-2 * time.Second)) path1.areaMemStat.releaseMemory() + expectedPaths := calcExpectedReleasedPaths(path1.areaMemStat, path1, path2, path3) feedbacks := make([]Feedback[int, string, any], 0) - for i := 0; i < 1; i++ { + for i := 0; i < len(expectedPaths); i++ { select { case fb := <-feedbackChan: feedbacks = append(feedbacks, fb) case <-time.After(100 * time.Millisecond): - require.Fail(t, "should receive 1 feedbacks") + require.Fail(t, "should receive feedbacks") } } - require.Equal(t, 1, len(feedbacks)) - require.Equal(t, ReleasePath, feedbacks[0].FeedbackType) - require.Equal(t, area, feedbacks[0].Area) - require.Equal(t, path1.path, feedbacks[0].Path) + require.Len(t, feedbacks, len(expectedPaths)) + gotPaths := make(map[string]bool, len(feedbacks)) + for _, fb := range feedbacks { + require.Equal(t, ReleasePath, fb.FeedbackType) + require.Equal(t, area, fb.Area) + gotPaths[fb.Path] = true + } + for _, path := range expectedPaths { + require.True(t, gotPaths[path]) + } // Case 2: release path1 and path2 // Reset the paths @@ -407,40 +433,39 @@ func TestReleaseMemory(t *testing.T) { path1.areaMemStat.totalPendingSize.Store(900) // Call releaseMemory - // sizeToRelease = 1000 * 0.4 = 360 - // path1 (ts=300): release 300 bytes, sizeToRelease = 360 - 300 = 60 - // path2 (ts=200): release 300 bytes, sizeToRelease = 60 - 300 = -240 + // sizeToRelease = totalPendingSize * defaultReleaseMemoryRatio path1.areaMemStat.lastReleaseMemoryTime.Store(time.Now().Add(-2 * time.Second)) path1.areaMemStat.releaseMemory() // Verify feedback messages - // Should receive 2 ResetPath feedbacks + // Should receive feedbacks to release memory. + expectedPaths = calcExpectedReleasedPaths(path1.areaMemStat, path1, path2, path3) feedbacks = make([]Feedback[int, string, any], 0) timer := time.After(100 * time.Millisecond) - for i := 0; i < 2; i++ { + for i := 0; i < len(expectedPaths); i++ { select { case fb := <-feedbackChan: feedbacks = append(feedbacks, fb) case <-timer: - require.Fail(t, "should receive 2 feedbacks") + require.Fail(t, "should receive feedbacks") } } - require.Equal(t, 2, len(feedbacks)) - // Both should be ResetPath type + require.Len(t, feedbacks, len(expectedPaths)) + // Both should be ReleasePath type for _, fb := range feedbacks { require.Equal(t, ReleasePath, fb.FeedbackType) require.Equal(t, area, fb.Area) } - // Check that we got feedbacks for path1 and path2 - paths := make(map[string]bool) + // Check that we got expected feedbacks. + paths := make(map[string]bool, len(feedbacks)) for _, fb := range feedbacks { paths[fb.Path] = true } - require.True(t, paths["path-1"]) - require.True(t, paths["path-2"]) - require.False(t, paths["path-3"]) + for _, path := range expectedPaths { + require.True(t, paths[path]) + } // Verify no more feedbacks select { From 6c6babf2b7a63dfb8e9b63a146a8bc80f6cb93f4 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 2 Feb 2026 16:54:24 +0800 Subject: [PATCH 28/34] add ddl workload Signed-off-by: dongmen <414110582@qq.com> --- tools/workload/ddl_app.go | 41 +++++ tools/workload/ddl_config.go | 170 ++++++++++++++++++ tools/workload/ddl_config_test.go | 123 +++++++++++++ tools/workload/ddl_executor.go | 282 +++++++++++++++++++++++++++++ tools/workload/ddl_runner.go | 285 ++++++++++++++++++++++++++++++ tools/workload/ddl_types.go | 41 +++++ tools/workload/readme.md | 6 +- 7 files changed, 945 insertions(+), 3 deletions(-) create mode 100644 tools/workload/ddl_app.go create mode 100644 tools/workload/ddl_config.go create mode 100644 tools/workload/ddl_config_test.go create mode 100644 tools/workload/ddl_executor.go create mode 100644 tools/workload/ddl_runner.go create mode 100644 tools/workload/ddl_types.go diff --git a/tools/workload/ddl_app.go b/tools/workload/ddl_app.go new file mode 100644 index 0000000000..f4578b9fea --- /dev/null +++ b/tools/workload/ddl_app.go @@ -0,0 +1,41 @@ +// 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 main + +import ( + "sync" + + "github.com/pingcap/errors" + plog "github.com/pingcap/log" + "go.uber.org/zap" +) + +func (app *WorkloadApp) handleDDLExecution(wg *sync.WaitGroup) error { + cfg, err := LoadDDLConfig(app.Config.DDLConfigPath) + if err != nil { + return err + } + + runner, err := NewDDLRunner(app, cfg) + if err != nil { + return errors.Trace(err) + } + + plog.Info("start ddl workload", + zap.String("mode", cfg.Mode), + zap.Int("ddlWorker", app.Config.DDLWorker), + zap.String("ddlTimeout", app.Config.DDLTimeout.String())) + runner.Start(wg) + return nil +} diff --git a/tools/workload/ddl_config.go b/tools/workload/ddl_config.go new file mode 100644 index 0000000000..cf9f0f9cc7 --- /dev/null +++ b/tools/workload/ddl_config.go @@ -0,0 +1,170 @@ +// 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 main + +import ( + "fmt" + "path/filepath" + "strings" + + "github.com/BurntSushi/toml" + "github.com/pingcap/errors" +) + +const ( + ddlModeFixed = "fixed" + ddlModeRandom = "random" +) + +type DDLConfig struct { + Mode string `toml:"mode"` + RatePerMinute DDLRatePerMinute `toml:"rate_per_minute"` + Tables []string `toml:"tables"` +} + +type DDLRatePerMinute struct { + AddColumn int `toml:"add_column"` + DropColumn int `toml:"drop_column"` + AddIndex int `toml:"add_index"` + DropIndex int `toml:"drop_index"` + TruncateTable int `toml:"truncate_table"` +} + +func LoadDDLConfig(path string) (*DDLConfig, error) { + if strings.TrimSpace(path) == "" { + return nil, errors.New("ddl config path is empty") + } + if filepath.Ext(path) != ".toml" { + return nil, errors.Errorf("ddl config must be a .toml file: %s", path) + } + + var cfg DDLConfig + meta, err := toml.DecodeFile(path, &cfg) + if err != nil { + return nil, errors.Annotate(err, "decode ddl config failed") + } + if undecoded := meta.Undecoded(); len(undecoded) > 0 { + return nil, errors.Errorf("unknown keys in ddl config: %v", undecoded) + } + + cfg.normalize() + if err := cfg.validate(); err != nil { + return nil, err + } + return &cfg, nil +} + +func (c *DDLConfig) normalize() { + c.Mode = strings.ToLower(strings.TrimSpace(c.Mode)) + if c.Mode == "" { + if len(c.Tables) > 0 { + c.Mode = ddlModeFixed + } else { + c.Mode = ddlModeRandom + } + } + + // Trim and drop empty entries. + tables := make([]string, 0, len(c.Tables)) + for _, t := range c.Tables { + t = strings.TrimSpace(t) + if t != "" { + tables = append(tables, t) + } + } + c.Tables = tables +} + +func (c *DDLConfig) validate() error { + if c.Mode != ddlModeFixed && c.Mode != ddlModeRandom { + return errors.Errorf("unsupported ddl mode: %s", c.Mode) + } + if c.Mode == ddlModeFixed && len(c.Tables) == 0 { + return errors.New("ddl mode fixed requires tables") + } + + if err := validateRate("add_column", c.RatePerMinute.AddColumn); err != nil { + return err + } + if err := validateRate("drop_column", c.RatePerMinute.DropColumn); err != nil { + return err + } + if err := validateRate("add_index", c.RatePerMinute.AddIndex); err != nil { + return err + } + if err := validateRate("drop_index", c.RatePerMinute.DropIndex); err != nil { + return err + } + if err := validateRate("truncate_table", c.RatePerMinute.TruncateTable); err != nil { + return err + } + + if c.totalRate() == 0 { + return errors.New("ddl config has no enabled ddl types") + } + return nil +} + +func validateRate(name string, v int) error { + if v < 0 { + return errors.Errorf("ddl rate must be >= 0: %s=%d", name, v) + } + return nil +} + +func (c *DDLConfig) totalRate() int { + return c.RatePerMinute.AddColumn + + c.RatePerMinute.DropColumn + + c.RatePerMinute.AddIndex + + c.RatePerMinute.DropIndex + + c.RatePerMinute.TruncateTable +} + +type TableName struct { + Schema string + Name string +} + +func ParseTableName(raw string, defaultSchema string) (TableName, error) { + raw = strings.TrimSpace(raw) + if raw == "" { + return TableName{}, errors.New("table name is empty") + } + + parts := strings.Split(raw, ".") + switch len(parts) { + case 1: + name := strings.TrimSpace(parts[0]) + if name == "" { + return TableName{}, errors.Errorf("invalid table name: %s", raw) + } + if strings.TrimSpace(defaultSchema) == "" { + return TableName{}, errors.Errorf("table %s missing schema", raw) + } + return TableName{Schema: strings.TrimSpace(defaultSchema), Name: name}, nil + case 2: + schema := strings.TrimSpace(parts[0]) + name := strings.TrimSpace(parts[1]) + if schema == "" || name == "" { + return TableName{}, errors.Errorf("invalid table name: %s", raw) + } + return TableName{Schema: schema, Name: name}, nil + default: + return TableName{}, errors.Errorf("invalid table name: %s", raw) + } +} + +func (t TableName) String() string { + return fmt.Sprintf("%s.%s", t.Schema, t.Name) +} diff --git a/tools/workload/ddl_config_test.go b/tools/workload/ddl_config_test.go new file mode 100644 index 0000000000..23bca253be --- /dev/null +++ b/tools/workload/ddl_config_test.go @@ -0,0 +1,123 @@ +package main + +import ( + "os" + "path/filepath" + "testing" +) + +func TestParseTableName(t *testing.T) { + t.Parallel() + + t.Run("schema qualified", func(t *testing.T) { + table, err := ParseTableName("test.sbtest1", "ignored") + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + if table.Schema != "test" || table.Name != "sbtest1" { + t.Fatalf("unexpected table: %+v", table) + } + }) + + t.Run("default schema", func(t *testing.T) { + table, err := ParseTableName("sbtest1", "test") + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + if table.Schema != "test" || table.Name != "sbtest1" { + t.Fatalf("unexpected table: %+v", table) + } + }) + + t.Run("missing schema", func(t *testing.T) { + _, err := ParseTableName("sbtest1", "") + if err == nil { + t.Fatalf("expected error") + } + }) + + t.Run("invalid", func(t *testing.T) { + _, err := ParseTableName("a.b.c", "test") + if err == nil { + t.Fatalf("expected error") + } + }) +} + +func TestLoadDDLConfig(t *testing.T) { + t.Parallel() + + dir := t.TempDir() + + t.Run("extension check", func(t *testing.T) { + path := filepath.Join(dir, "ddl.txt") + if err := os.WriteFile(path, []byte("mode = \"fixed\""), 0o644); err != nil { + t.Fatalf("write file failed: %v", err) + } + _, err := LoadDDLConfig(path) + if err == nil { + t.Fatalf("expected error") + } + }) + + t.Run("default mode fixed", func(t *testing.T) { + path := filepath.Join(dir, "ddl_fixed.toml") + content := ` +tables = ["test.sbtest1"] + +[rate_per_minute] +add_column = 1 +` + if err := os.WriteFile(path, []byte(content), 0o644); err != nil { + t.Fatalf("write file failed: %v", err) + } + cfg, err := LoadDDLConfig(path) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + if cfg.Mode != ddlModeFixed { + t.Fatalf("expected mode fixed, got %s", cfg.Mode) + } + }) + + t.Run("default mode random", func(t *testing.T) { + path := filepath.Join(dir, "ddl_random.toml") + content := ` +[rate_per_minute] +add_column = 1 +` + if err := os.WriteFile(path, []byte(content), 0o644); err != nil { + t.Fatalf("write file failed: %v", err) + } + cfg, err := LoadDDLConfig(path) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + if cfg.Mode != ddlModeRandom { + t.Fatalf("expected mode random, got %s", cfg.Mode) + } + }) + + t.Run("no enabled ddl types", func(t *testing.T) { + path := filepath.Join(dir, "ddl_empty.toml") + content := ` +mode = "fixed" + +tables = ["test.sbtest1"] + +[rate_per_minute] +add_column = 0 +drop_column = 0 +add_index = 0 +drop_index = 0 +truncate_table = 0 +` + if err := os.WriteFile(path, []byte(content), 0o644); err != nil { + t.Fatalf("write file failed: %v", err) + } + _, err := LoadDDLConfig(path) + if err == nil { + t.Fatalf("expected error") + } + }) +} diff --git a/tools/workload/ddl_executor.go b/tools/workload/ddl_executor.go new file mode 100644 index 0000000000..6b4c7c768c --- /dev/null +++ b/tools/workload/ddl_executor.go @@ -0,0 +1,282 @@ +// 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 main + +import ( + "context" + "database/sql" + "fmt" + "math/rand" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + plog "github.com/pingcap/log" + "go.uber.org/zap" +) + +const ( + ddlColumnPrefix = "ddl_col_" + ddlIndexPrefix = "ddl_idx_" +) + +var ddlNameSeq atomic.Uint64 + +func (r *DDLRunner) startWorkers(wg *sync.WaitGroup) { + workerCount := r.app.Config.DDLWorker + if workerCount <= 0 { + workerCount = 1 + } + + wg.Add(workerCount) + for workerID := range workerCount { + db := r.app.DBManager.GetDB() + go func(workerID int, db *DBWrapper) { + defer func() { + plog.Info("ddl worker exited", zap.Int("worker", workerID)) + wg.Done() + }() + + conn, err := getConnWithTimeout(db.DB, 10*time.Second) + if err != nil { + plog.Info("get connection failed for ddl worker", zap.Error(err)) + time.Sleep(5 * time.Second) + return + } + defer func() { + if conn != nil { + conn.Close() + } + }() + + plog.Info("start ddl worker", zap.Int("worker", workerID), zap.String("db", db.Name)) + + for { + task := <-r.taskCh + if err := r.executeTask(conn, task); err != nil { + if r.app.isConnectionError(err) { + conn.Close() + time.Sleep(2 * time.Second) + newConn, err := getConnWithTimeout(db.DB, 10*time.Second) + if err != nil { + plog.Info("reconnect failed for ddl worker", zap.Error(err)) + time.Sleep(5 * time.Second) + continue + } + conn = newConn + } + } + } + }(workerID, db) + } +} + +func getConnWithTimeout(db *sql.DB, timeout time.Duration) (*sql.Conn, error) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + conn, err := db.Conn(ctx) + if err != nil { + return nil, errors.Trace(err) + } + return conn, nil +} + +func (r *DDLRunner) executeTask(conn *sql.Conn, task DDLTask) error { + ctx, cancel := context.WithTimeout(context.Background(), r.app.Config.DDLTimeout) + defer cancel() + + sqlStr, skipped, reason, err := r.buildDDL(ctx, conn, task) + if err != nil { + r.app.Stats.DDLFailed.Add(1) + r.app.Stats.ErrorCount.Add(1) + plog.Info("build ddl failed", + zap.String("ddlType", task.Type.String()), + zap.String("table", task.Table.String()), + zap.Error(err)) + return err + } + if skipped { + r.app.Stats.DDLSkipped.Add(1) + if reason != "" { + plog.Debug("ddl task skipped", + zap.String("ddlType", task.Type.String()), + zap.String("table", task.Table.String()), + zap.String("reason", reason)) + } + return nil + } + + r.app.Stats.DDLExecuted.Add(1) + r.app.Stats.QueryCount.Add(1) + start := time.Now() + if _, err := conn.ExecContext(ctx, sqlStr); err != nil { + r.app.Stats.DDLFailed.Add(1) + r.app.Stats.ErrorCount.Add(1) + plog.Info("execute ddl failed", + zap.String("ddlType", task.Type.String()), + zap.String("table", task.Table.String()), + zap.Duration("cost", time.Since(start)), + zap.String("sql", getSQLPreview(sqlStr)), + zap.Error(err)) + return err + } + + r.app.Stats.DDLSucceeded.Add(1) + plog.Debug("ddl executed", + zap.String("ddlType", task.Type.String()), + zap.String("table", task.Table.String()), + zap.Duration("cost", time.Since(start))) + return nil +} + +func (r *DDLRunner) buildDDL(ctx context.Context, conn *sql.Conn, task DDLTask) (sqlStr string, skipped bool, reason string, err error) { + switch task.Type { + case ddlAddColumn: + return r.buildAddColumnDDL(task.Table), false, "", nil + case ddlDropColumn: + return r.buildDropColumnDDL(ctx, conn, task.Table) + case ddlAddIndex: + return r.buildAddIndexDDL(ctx, conn, task.Table) + case ddlDropIndex: + return r.buildDropIndexDDL(ctx, conn, task.Table) + case ddlTruncateTable: + return r.buildTruncateTableDDL(task.Table), false, "", nil + default: + return "", false, "", errors.Errorf("unknown ddl type: %d", task.Type) + } +} + +func (r *DDLRunner) buildAddColumnDDL(table TableName) string { + colName := ddlColumnPrefix + ddlNameSuffix() + return fmt.Sprintf("ALTER TABLE %s ADD COLUMN %s BIGINT NOT NULL DEFAULT 0", + quoteTable(table), + quoteIdent(colName)) +} + +func (r *DDLRunner) buildDropColumnDDL(ctx context.Context, conn *sql.Conn, table TableName) (string, bool, string, error) { + col, ok, err := selectOne(ctx, conn, ` +SELECT column_name +FROM information_schema.columns +WHERE table_schema = ? + AND table_name = ? + AND column_name LIKE ? +`, table.Schema, table.Name, ddlColumnPrefix+"%") + if err != nil { + return "", false, "", err + } + if !ok { + return "", true, "no ddl columns", nil + } + return fmt.Sprintf("ALTER TABLE %s DROP COLUMN %s", + quoteTable(table), + quoteIdent(col)), false, "", nil +} + +func (r *DDLRunner) buildAddIndexDDL(ctx context.Context, conn *sql.Conn, table TableName) (string, bool, string, error) { + col, ok, err := selectOne(ctx, conn, ` +SELECT column_name +FROM information_schema.columns +WHERE table_schema = ? + AND table_name = ? + AND column_name LIKE ? +`, table.Schema, table.Name, ddlColumnPrefix+"%") + if err != nil { + return "", false, "", err + } + + if !ok { + col, ok, err = selectOne(ctx, conn, ` +SELECT column_name +FROM information_schema.columns +WHERE table_schema = ? + AND table_name = ? + AND data_type NOT IN ('json','tinyblob','blob','mediumblob','longblob','tinytext','text','mediumtext','longtext') +`, table.Schema, table.Name) + if err != nil { + return "", false, "", err + } + if !ok { + return "", true, "no suitable columns", nil + } + } + + idxName := ddlIndexPrefix + ddlNameSuffix() + return fmt.Sprintf("ALTER TABLE %s ADD INDEX %s (%s)", + quoteTable(table), + quoteIdent(idxName), + quoteIdent(col)), false, "", nil +} + +func (r *DDLRunner) buildDropIndexDDL(ctx context.Context, conn *sql.Conn, table TableName) (string, bool, string, error) { + idx, ok, err := selectOne(ctx, conn, ` +SELECT DISTINCT index_name +FROM information_schema.statistics +WHERE table_schema = ? + AND table_name = ? + AND index_name LIKE ? +`, table.Schema, table.Name, ddlIndexPrefix+"%") + if err != nil { + return "", false, "", err + } + if !ok { + return "", true, "no ddl indexes", nil + } + return fmt.Sprintf("ALTER TABLE %s DROP INDEX %s", + quoteTable(table), + quoteIdent(idx)), false, "", nil +} + +func (r *DDLRunner) buildTruncateTableDDL(table TableName) string { + return fmt.Sprintf("TRUNCATE TABLE %s", quoteTable(table)) +} + +func ddlNameSuffix() string { + seq := ddlNameSeq.Add(1) + return fmt.Sprintf("%d_%d", time.Now().UnixNano(), seq) +} + +func quoteIdent(name string) string { + escaped := strings.ReplaceAll(name, "`", "``") + return "`" + escaped + "`" +} + +func quoteTable(table TableName) string { + return quoteIdent(table.Schema) + "." + quoteIdent(table.Name) +} + +func selectOne(ctx context.Context, conn *sql.Conn, query string, args ...interface{}) (value string, ok bool, err error) { + rows, err := conn.QueryContext(ctx, query, args...) + if err != nil { + return "", false, errors.Trace(err) + } + defer rows.Close() + + var values []string + for rows.Next() { + var v string + if err := rows.Scan(&v); err != nil { + return "", false, errors.Trace(err) + } + values = append(values, v) + } + if err := rows.Err(); err != nil { + return "", false, errors.Trace(err) + } + if len(values) == 0 { + return "", false, nil + } + return values[rand.Intn(len(values))], true, nil +} diff --git a/tools/workload/ddl_runner.go b/tools/workload/ddl_runner.go new file mode 100644 index 0000000000..5cdcf99d2b --- /dev/null +++ b/tools/workload/ddl_runner.go @@ -0,0 +1,285 @@ +// 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 main + +import ( + "context" + "database/sql" + "math/rand" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + plog "github.com/pingcap/log" + "go.uber.org/zap" +) + +const ( + randomModeTableSampleSize = 10 + ddlTaskChanMinBufSize = 1024 +) + +type DDLTask struct { + Type DDLType + Table TableName +} + +type DDLRunner struct { + app *WorkloadApp + cfg *DDLConfig + + taskCh chan DDLTask + selector ddlTableSelector + + randomSchema string +} + +func NewDDLRunner(app *WorkloadApp, cfg *DDLConfig) (*DDLRunner, error) { + if app == nil || app.Config == nil || app.DBManager == nil { + return nil, errors.New("ddl runner requires initialized app") + } + + r := &DDLRunner{ + app: app, + cfg: cfg, + } + + bufSize := cfg.totalRate() * 2 + if bufSize < ddlTaskChanMinBufSize { + bufSize = ddlTaskChanMinBufSize + } + r.taskCh = make(chan DDLTask, bufSize) + + switch cfg.Mode { + case ddlModeFixed: + defaultSchema := app.Config.DBName + if app.Config.DBPrefix != "" && app.Config.DBNum > 0 { + defaultSchema = "" + } + tables, err := parseTableList(cfg.Tables, defaultSchema) + if err != nil { + return nil, err + } + r.selector = newFixedTableSelector(tables) + case ddlModeRandom: + if app.Config.DBPrefix != "" || app.Config.DBNum != 1 { + return nil, errors.New("ddl random mode only supports single database connection") + } + r.randomSchema = app.Config.DBName + randomSelector := newRandomTableSelector() + if err := r.refreshRandomTables(randomSelector); err != nil { + return nil, err + } + r.selector = randomSelector + default: + return nil, errors.Errorf("unsupported ddl mode: %s", cfg.Mode) + } + + return r, nil +} + +func (r *DDLRunner) Start(wg *sync.WaitGroup) { + if r.cfg.Mode == ddlModeRandom { + r.startRandomTableRefresh() + } + + r.startTaskSchedulers() + r.startWorkers(wg) +} + +func (r *DDLRunner) startTaskSchedulers() { + r.startTypeScheduler(ddlAddColumn, r.cfg.RatePerMinute.AddColumn) + r.startTypeScheduler(ddlDropColumn, r.cfg.RatePerMinute.DropColumn) + r.startTypeScheduler(ddlAddIndex, r.cfg.RatePerMinute.AddIndex) + r.startTypeScheduler(ddlDropIndex, r.cfg.RatePerMinute.DropIndex) + r.startTypeScheduler(ddlTruncateTable, r.cfg.RatePerMinute.TruncateTable) +} + +func (r *DDLRunner) startTypeScheduler(ddlType DDLType, perMinute int) { + if perMinute <= 0 { + return + } + + go func() { + ticker := time.NewTicker(time.Minute) + defer ticker.Stop() + + for { + for i := 0; i < perMinute; i++ { + table, ok := r.selector.Next() + if !ok { + r.app.Stats.DDLSkipped.Add(1) + continue + } + r.taskCh <- DDLTask{Type: ddlType, Table: table} + } + <-ticker.C + } + }() +} + +func (r *DDLRunner) startRandomTableRefresh() { + go func() { + ticker := time.NewTicker(time.Minute) + defer ticker.Stop() + + randomSelector := r.selector.(*randomTableSelector) + for range ticker.C { + if err := r.refreshRandomTables(randomSelector); err != nil { + plog.Info("refresh random tables failed", zap.Error(err)) + } + } + }() +} + +func (r *DDLRunner) refreshRandomTables(selector *randomTableSelector) error { + dbs := r.app.DBManager.GetDBs() + if len(dbs) == 0 { + return errors.New("no database connections available") + } + + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + tableNames, err := fetchBaseTables(ctx, dbs[0].DB, r.randomSchema) + if err != nil { + return err + } + + if len(tableNames) == 0 { + selector.Update(nil) + plog.Info("no tables found for ddl random mode", zap.String("schema", r.randomSchema)) + return nil + } + + sampled := sampleStrings(tableNames, randomModeTableSampleSize) + tables := make([]TableName, 0, len(sampled)) + for _, name := range sampled { + tables = append(tables, TableName{Schema: r.randomSchema, Name: name}) + } + selector.Update(tables) + + plog.Info("random tables refreshed", + zap.String("schema", r.randomSchema), + zap.Int("tableCount", len(tables))) + return nil +} + +func fetchBaseTables(ctx context.Context, db *sql.DB, schema string) ([]string, error) { + const query = ` +SELECT table_name +FROM information_schema.tables +WHERE table_schema = ? + AND table_type = 'BASE TABLE'` + rows, err := db.QueryContext(ctx, query, schema) + if err != nil { + return nil, errors.Annotate(err, "query tables from information_schema failed") + } + defer rows.Close() + + var tables []string + for rows.Next() { + var name string + if err := rows.Scan(&name); err != nil { + return nil, errors.Annotate(err, "scan table name failed") + } + tables = append(tables, name) + } + if err := rows.Err(); err != nil { + return nil, errors.Annotate(err, "iterate table names failed") + } + return tables, nil +} + +func sampleStrings(in []string, n int) []string { + if n <= 0 || len(in) == 0 { + return nil + } + if len(in) <= n { + out := make([]string, len(in)) + copy(out, in) + return out + } + + indices := rand.Perm(len(in))[:n] + out := make([]string, 0, n) + for _, idx := range indices { + out = append(out, in[idx]) + } + return out +} + +type ddlTableSelector interface { + Next() (TableName, bool) +} + +type fixedTableSelector struct { + tables []TableName + next atomic.Uint64 +} + +func newFixedTableSelector(tables []TableName) *fixedTableSelector { + return &fixedTableSelector{tables: tables} +} + +func (s *fixedTableSelector) Next() (TableName, bool) { + if len(s.tables) == 0 { + return TableName{}, false + } + i := int(s.next.Add(1)-1) % len(s.tables) + return s.tables[i], true +} + +type randomTableSelector struct { + mu sync.RWMutex + tables []TableName +} + +func newRandomTableSelector() *randomTableSelector { + return &randomTableSelector{} +} + +func (s *randomTableSelector) Update(tables []TableName) { + s.mu.Lock() + s.tables = tables + s.mu.Unlock() +} + +func (s *randomTableSelector) Next() (TableName, bool) { + s.mu.RLock() + defer s.mu.RUnlock() + if len(s.tables) == 0 { + return TableName{}, false + } + return s.tables[rand.Intn(len(s.tables))], true +} + +func parseTableList(rawTables []string, defaultSchema string) ([]TableName, error) { + seen := make(map[string]struct{}, len(rawTables)) + out := make([]TableName, 0, len(rawTables)) + for _, raw := range rawTables { + table, err := ParseTableName(raw, defaultSchema) + if err != nil { + return nil, err + } + key := table.String() + if _, ok := seen[key]; ok { + continue + } + seen[key] = struct{}{} + out = append(out, table) + } + return out, nil +} diff --git a/tools/workload/ddl_types.go b/tools/workload/ddl_types.go new file mode 100644 index 0000000000..f03e2a258a --- /dev/null +++ b/tools/workload/ddl_types.go @@ -0,0 +1,41 @@ +// 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 main + +type DDLType int + +const ( + ddlAddColumn DDLType = iota + ddlDropColumn + ddlAddIndex + ddlDropIndex + ddlTruncateTable +) + +func (t DDLType) String() string { + switch t { + case ddlAddColumn: + return "add_column" + case ddlDropColumn: + return "drop_column" + case ddlAddIndex: + return "add_index" + case ddlDropIndex: + return "drop_index" + case ddlTruncateTable: + return "truncate_table" + default: + return "unknown" + } +} diff --git a/tools/workload/readme.md b/tools/workload/readme.md index 7a49cb3120..188f68f8d4 100644 --- a/tools/workload/readme.md +++ b/tools/workload/readme.md @@ -22,7 +22,7 @@ make Run DDL workload based on a TOML config file: ```bash -./workload -action ddl \ +./bin/workload -action ddl \ -database-host 127.0.0.1 \ -database-port 4000 \ -database-db-name test \ @@ -67,7 +67,7 @@ truncate_table = 0 Insert test data using sysbench-compatible schema: ```bash -./workload -action insert \ +./bin/workload -action insert \ -database-host 127.0.0.1 \ -database-port 4000 \ -database-db-name db1 \ @@ -83,7 +83,7 @@ Insert test data using sysbench-compatible schema: Update existing data with large row operations: ```bash -./workload -action update \ +./bin/workload -action update \ -database-host 127.0.0.1 \ -database-port 4000 \ -database-db-name large \ From 6464520f7b5da6e67e5c64c50164ac7a535048b5 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 2 Feb 2026 17:26:46 +0800 Subject: [PATCH 29/34] remove useless code Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/dispatcher_stat.go | 1 - pkg/eventservice/event_broker.go | 17 +---------------- 2 files changed, 1 insertion(+), 17 deletions(-) diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index eb8f72c265..ba0a1ecdd2 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -71,7 +71,6 @@ type dispatcherStat struct { nextSyncPoint atomic.Uint64 syncPointInterval time.Duration txnAtomicity config.AtomicityLevel - lastSyncPoint atomic.Uint64 // ============================================================================= // ================== below are fields need copied when reset ================== diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 4c4643e16e..289f7ab835 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -51,7 +51,6 @@ const ( maxReadyEventIntervalSeconds = 10 // defaultSendResolvedTsInterval use to control whether to send a resolvedTs event to the dispatcher when its scan is skipped. defaultSendResolvedTsInterval = time.Second * 2 - congestionControlLogInterval = time.Minute ) // eventBroker get event from the eventStore, and send the event to the dispatchers. @@ -433,7 +432,7 @@ func (c *eventBroker) getScanTaskDataRange(task scanTask) (bool, common.DataRang if scanMaxTs > 0 { dataRange.CommitTsEnd = min(dataRange.CommitTsEnd, scanMaxTs) if dataRange.CommitTsEnd < commitTsEndBeforeWindow { - log.Debug("fizz scan window capped", + log.Debug("scan window capped", zap.Stringer("changefeedID", task.changefeedStat.changefeedID), zap.Stringer("dispatcherID", task.id), zap.Uint64("baseTs", task.changefeedStat.minSentTs.Load()), @@ -571,7 +570,6 @@ func (c *eventBroker) emitSyncPointEventIfNeeded(ts uint64, d *dispatcherStat, r syncPointEvent := newWrapSyncPointEvent(remoteID, e) c.getMessageCh(d.messageWorkerIndex, common.IsRedoMode(d.info.GetMode())) <- syncPointEvent - d.lastSyncPoint.Store(commitTs) } } @@ -1303,19 +1301,6 @@ func (c *eventBroker) handleCongestionControl(from node.ID, m *event.CongestionC } return true }) - - nowUnix := now.Unix() - lastLog := c.lastCongestionLogTime.Load() - if nowUnix-lastLog >= int64(congestionControlLogInterval.Seconds()) && - c.lastCongestionLogTime.CompareAndSwap(lastLog, nowUnix) { - log.Info("congestion control received", - zap.Uint16("version", uint16(m.GetVersion())), - zap.Int("changefeedCount", len(availables)), - zap.Int("usageCount", usageCount), - zap.Int("zeroMaxCount", zeroMaxCount), - zap.Int("dispatcherCount", len(dispatcherAvailable)), - ) - } } func (c *eventBroker) sendDispatcherResponse(responseMap map[string]*event.DispatcherHeartbeatResponse) { From 730f37312b303450dcca2b6324132371afde74cc Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 3 Feb 2026 18:09:40 +0800 Subject: [PATCH 30/34] refine syncpoint handle Signed-off-by: dongmen <414110582@qq.com> --- .../dispatcher/basic_dispatcher.go | 9 +- heartbeatpb/heartbeat.pb.go | 301 +++++++++--------- heartbeatpb/heartbeat.proto | 3 +- maintainer/barrier.go | 77 ++++- maintainer/barrier_event.go | 7 +- maintainer/barrier_test.go | 140 ++++++++ maintainer/maintainer.go | 40 +++ pkg/metrics/maintainer.go | 9 + 8 files changed, 426 insertions(+), 160 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 66a94e5e04..303d096d2b 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -629,14 +629,19 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D // 3. clear blockEventStatus(should be the old pending event, but clear the new one) d.blockEventStatus.clear() }) - if action.Action == heartbeatpb.Action_Write { + switch action.Action { + case heartbeatpb.Action_Write: actionCommitTs := action.CommitTs actionIsSyncPoint := action.IsSyncPoint d.sharedInfo.GetBlockEventExecutor().Submit(d, func() { d.ExecuteBlockEventDDL(pendingEvent, actionCommitTs, actionIsSyncPoint) }) return true - } else { + case heartbeatpb.Action_Pass, heartbeatpb.Action_Skip: + failpoint.Inject("BlockOrWaitBeforePass", nil) + d.PassBlockEventToSink(pendingEvent) + failpoint.Inject("BlockAfterPass", nil) + default: failpoint.Inject("BlockOrWaitBeforePass", nil) d.PassBlockEventToSink(pendingEvent) failpoint.Inject("BlockAfterPass", nil) diff --git a/heartbeatpb/heartbeat.pb.go b/heartbeatpb/heartbeat.pb.go index 10f1a63dd5..1916f7bf5a 100644 --- a/heartbeatpb/heartbeat.pb.go +++ b/heartbeatpb/heartbeat.pb.go @@ -28,16 +28,19 @@ type Action int32 const ( Action_Write Action = 0 Action_Pass Action = 1 + Action_Skip Action = 2 ) var Action_name = map[int32]string{ 0: "Write", 1: "Pass", + 2: "Skip", } var Action_value = map[string]int32{ "Write": 0, "Pass": 1, + "Skip": 2, } func (x Action) String() string { @@ -3246,156 +3249,156 @@ func init() { func init() { proto.RegisterFile("heartbeatpb/heartbeat.proto", fileDescriptor_6d584080fdadb670) } var fileDescriptor_6d584080fdadb670 = []byte{ - // 2376 bytes of a gzipped FileDescriptorProto + // 2382 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x1a, 0x4d, 0x6f, 0x1c, 0x49, - 0xd5, 0xdd, 0x3d, 0x33, 0x9e, 0x79, 0xe3, 0xb1, 0x3b, 0x95, 0x8f, 0x75, 0x12, 0xc7, 0xf1, 0x36, - 0x20, 0x19, 0xef, 0x92, 0x90, 0xec, 0x46, 0xc0, 0xb2, 0x6c, 0xb0, 0x67, 0xb2, 0x9b, 0x91, 0x63, - 0xaf, 0x55, 0xe3, 0x55, 0x60, 0x39, 0x0c, 0xed, 0xee, 0xca, 0xb8, 0xe5, 0x99, 0xae, 0x4e, 0x57, - 0x4f, 0x1c, 0x47, 0x5a, 0x10, 0x42, 0xdc, 0x38, 0x80, 0xe0, 0xc2, 0x81, 0x03, 0xe2, 0x0f, 0x20, - 0x21, 0x8e, 0x48, 0xdc, 0x40, 0xe2, 0xb2, 0xa7, 0x15, 0x37, 0x50, 0xf2, 0x07, 0x10, 0x1c, 0xb8, - 0xa2, 0xaa, 0xae, 0xea, 0xaf, 0x69, 0xdb, 0x89, 0x3c, 0xca, 0xad, 0x5f, 0xd5, 0x7b, 0xaf, 0x5e, - 0xbd, 0xef, 0x57, 0x33, 0x70, 0x75, 0x9f, 0xd8, 0x61, 0xb4, 0x47, 0xec, 0x28, 0xd8, 0xbb, 0x99, - 0x7c, 0xdf, 0x08, 0x42, 0x1a, 0x51, 0xd4, 0xcc, 0x6c, 0x5a, 0x47, 0xd0, 0xd8, 0xb5, 0xf7, 0x86, - 0xa4, 0x17, 0xd8, 0x3e, 0x5a, 0x84, 0x59, 0x01, 0x74, 0x3b, 0x8b, 0xda, 0x8a, 0xb6, 0x6a, 0x60, - 0x05, 0xa2, 0x2b, 0x50, 0xef, 0x45, 0x76, 0x18, 0x6d, 0x92, 0xa3, 0x45, 0x7d, 0x45, 0x5b, 0x9d, - 0xc3, 0x09, 0x8c, 0x2e, 0x41, 0xed, 0x9e, 0xef, 0xf2, 0x1d, 0x43, 0xec, 0x48, 0x08, 0x2d, 0x03, - 0x6c, 0x92, 0x23, 0x16, 0xd8, 0x0e, 0x67, 0x58, 0x59, 0xd1, 0x56, 0x5b, 0x38, 0xb3, 0x62, 0x7d, - 0xa1, 0x83, 0x79, 0x9f, 0x8b, 0xb2, 0x41, 0xec, 0x08, 0x93, 0xc7, 0x63, 0xc2, 0x22, 0xf4, 0x1d, - 0x98, 0x73, 0xf6, 0x6d, 0x7f, 0x40, 0x1e, 0x11, 0xe2, 0x4a, 0x39, 0x9a, 0xb7, 0x2f, 0xdf, 0xc8, - 0xc8, 0x7c, 0xa3, 0x9d, 0x41, 0xc0, 0x39, 0x74, 0xf4, 0x2e, 0x34, 0x0e, 0xed, 0x88, 0x84, 0x23, - 0x3b, 0x3c, 0x10, 0x82, 0x36, 0x6f, 0x5f, 0xca, 0xd1, 0x3e, 0x54, 0xbb, 0x38, 0x45, 0x44, 0xef, - 0x43, 0x2b, 0x24, 0x2e, 0x4d, 0xf6, 0xc4, 0x45, 0x8e, 0xa7, 0xcc, 0x23, 0xa3, 0x6f, 0x42, 0x9d, - 0x45, 0x76, 0x34, 0x66, 0x84, 0x2d, 0x56, 0x56, 0x8c, 0xd5, 0xe6, 0xed, 0xa5, 0x1c, 0x61, 0xa2, - 0xdf, 0x9e, 0xc0, 0xc2, 0x09, 0x36, 0x5a, 0x85, 0x05, 0x87, 0x8e, 0x02, 0x32, 0x24, 0x11, 0x89, - 0x37, 0x17, 0xab, 0x2b, 0xda, 0x6a, 0x1d, 0x17, 0x97, 0xd1, 0x5b, 0x60, 0x90, 0x30, 0x5c, 0xac, - 0x95, 0x68, 0x03, 0x8f, 0x7d, 0xdf, 0xf3, 0x07, 0xf7, 0xc2, 0x90, 0x86, 0x98, 0x63, 0x59, 0x3f, - 0xd3, 0xa0, 0x91, 0x8a, 0x67, 0x71, 0x8d, 0x12, 0xe7, 0x20, 0xa0, 0x9e, 0x1f, 0xed, 0x32, 0xa1, - 0xd1, 0x0a, 0xce, 0xad, 0x71, 0x53, 0x85, 0x84, 0xd1, 0xe1, 0x13, 0xe2, 0xee, 0x32, 0xa1, 0xb7, - 0x0a, 0xce, 0xac, 0x20, 0x13, 0x0c, 0x46, 0x1e, 0x0b, 0xb5, 0x54, 0x30, 0xff, 0xe4, 0x5c, 0x87, - 0x36, 0x8b, 0x7a, 0x47, 0xbe, 0x23, 0x68, 0x2a, 0x31, 0xd7, 0xec, 0x9a, 0xf5, 0x19, 0x98, 0x1d, - 0x8f, 0x05, 0x76, 0xe4, 0xec, 0x93, 0x70, 0xdd, 0x89, 0x3c, 0xea, 0xa3, 0xb7, 0xa0, 0x66, 0x8b, - 0x2f, 0x21, 0xc7, 0xfc, 0xed, 0xf3, 0xb9, 0xbb, 0xc4, 0x48, 0x58, 0xa2, 0x70, 0xaf, 0x6b, 0xd3, - 0xd1, 0xc8, 0x8b, 0x12, 0xa1, 0x12, 0x18, 0xad, 0x40, 0xb3, 0xcb, 0xf8, 0x51, 0x3b, 0xfc, 0x0e, - 0x42, 0xb4, 0x3a, 0xce, 0x2e, 0x59, 0x6d, 0x30, 0xd6, 0xdb, 0x9b, 0x39, 0x26, 0xda, 0xc9, 0x4c, - 0xf4, 0x49, 0x26, 0x3f, 0xd5, 0xe1, 0x62, 0xd7, 0x7f, 0x34, 0x1c, 0x13, 0x7e, 0xa9, 0xf4, 0x3a, - 0x0c, 0x7d, 0x17, 0x5a, 0xc9, 0xc6, 0xee, 0x51, 0x40, 0xe4, 0x85, 0xae, 0xe4, 0x2e, 0x94, 0xc3, - 0xc0, 0x79, 0x02, 0x74, 0x17, 0x5a, 0x29, 0xc3, 0x6e, 0x87, 0xdf, 0xd1, 0x98, 0x30, 0x6f, 0x16, - 0x03, 0xe7, 0xf1, 0x45, 0x54, 0x3a, 0xfb, 0x64, 0x64, 0x77, 0x3b, 0x42, 0x01, 0x06, 0x4e, 0x60, - 0xb4, 0x09, 0xe7, 0xc9, 0x53, 0x67, 0x38, 0x76, 0x49, 0x86, 0xc6, 0x15, 0x76, 0x3a, 0xf1, 0x88, - 0x32, 0x2a, 0xeb, 0xaf, 0x5a, 0xd6, 0x94, 0xd2, 0x27, 0xbf, 0x07, 0x17, 0xbd, 0x32, 0xcd, 0xc8, - 0x98, 0xb5, 0xca, 0x15, 0x91, 0xc5, 0xc4, 0xe5, 0x0c, 0xd0, 0x9d, 0xc4, 0x49, 0xe2, 0x10, 0xbe, - 0x76, 0x8c, 0xb8, 0x05, 0x77, 0xb1, 0xc0, 0xb0, 0x1d, 0x15, 0xbc, 0x66, 0xde, 0xb1, 0xda, 0x9b, - 0x98, 0x6f, 0x5a, 0x7f, 0xd2, 0xe0, 0x5c, 0x26, 0xe9, 0xb0, 0x80, 0xfa, 0x8c, 0x9c, 0x35, 0xeb, - 0x6c, 0x01, 0x72, 0x0b, 0xda, 0x21, 0xca, 0x9a, 0xc7, 0xc9, 0x2e, 0x93, 0x41, 0x09, 0x21, 0x42, - 0x50, 0x19, 0x51, 0x97, 0x48, 0x93, 0x8a, 0x6f, 0xeb, 0x29, 0x9c, 0x6f, 0x67, 0x22, 0x76, 0x8b, - 0x30, 0x66, 0x0f, 0xce, 0x2c, 0x78, 0x31, 0x37, 0xe8, 0x93, 0xb9, 0xc1, 0xfa, 0xb5, 0x06, 0x0b, - 0x98, 0xb8, 0x74, 0x8b, 0x44, 0xf6, 0x94, 0x8e, 0x3d, 0x2d, 0xdd, 0x14, 0xc5, 0x32, 0x4a, 0xc4, - 0xfa, 0x11, 0x5c, 0xe3, 0x52, 0xe1, 0x84, 0x6a, 0x27, 0xa4, 0x83, 0x90, 0x30, 0xf6, 0x7a, 0x64, - 0xb4, 0x3e, 0x83, 0xa5, 0xfc, 0xf9, 0x1f, 0xd2, 0xf0, 0xd0, 0x0e, 0xdd, 0xd7, 0x74, 0xfc, 0x7f, - 0x73, 0x11, 0xd9, 0xa6, 0xfe, 0x23, 0x6f, 0x80, 0xd6, 0xa0, 0xc2, 0x02, 0xdb, 0x97, 0x67, 0x5d, - 0x2a, 0xaf, 0x42, 0x58, 0xe0, 0xf0, 0x5a, 0xcf, 0x78, 0x05, 0x4f, 0xb8, 0x2b, 0x90, 0x4b, 0xee, - 0x66, 0x32, 0x82, 0x8c, 0xa7, 0x13, 0x52, 0x46, 0x0e, 0x9d, 0x27, 0x25, 0xa6, 0x92, 0x52, 0x25, - 0x4e, 0x4a, 0x0a, 0x4e, 0x3c, 0xbb, 0x9a, 0x7a, 0x36, 0x5a, 0x03, 0x93, 0x1d, 0x78, 0x41, 0x67, - 0xeb, 0xc1, 0x3a, 0xeb, 0x49, 0x89, 0x6a, 0x22, 0x11, 0x4f, 0xac, 0x5b, 0x5f, 0x68, 0x70, 0x99, - 0x67, 0x38, 0x77, 0x3c, 0xcc, 0x24, 0xa8, 0x29, 0xf5, 0x0e, 0x77, 0xa0, 0xe6, 0x08, 0x3d, 0x9e, - 0x92, 0x75, 0x62, 0x65, 0x63, 0x89, 0x8c, 0xda, 0x30, 0xcf, 0xa4, 0x48, 0x71, 0x3e, 0x12, 0x0a, - 0x9b, 0xbf, 0x7d, 0x35, 0x47, 0xde, 0xcb, 0xa1, 0xe0, 0x02, 0x89, 0xf5, 0x3f, 0x0d, 0x2e, 0x6d, - 0x91, 0x70, 0x30, 0xfd, 0x5b, 0xdd, 0x85, 0x96, 0xfb, 0x8a, 0x45, 0x26, 0x87, 0x8f, 0xba, 0x80, - 0x46, 0x5c, 0x32, 0xb7, 0xf3, 0x4a, 0x4e, 0x51, 0x42, 0x94, 0x98, 0xbf, 0x92, 0x49, 0x6c, 0x3b, - 0x70, 0x7e, 0xcb, 0xf6, 0xfc, 0xc8, 0xf6, 0x7c, 0x12, 0xde, 0x57, 0xdc, 0xd0, 0xb7, 0x32, 0x4d, - 0x95, 0x56, 0x92, 0x48, 0x53, 0x9a, 0x62, 0x57, 0x65, 0xfd, 0x45, 0x07, 0xb3, 0xb8, 0x7d, 0x56, - 0x2d, 0x5e, 0x03, 0xe0, 0x5f, 0x7d, 0x7e, 0x08, 0x11, 0xfe, 0xd1, 0xc0, 0x0d, 0xbe, 0xc2, 0xd9, - 0x13, 0x74, 0x0b, 0xaa, 0xf1, 0x4e, 0x99, 0xe9, 0xdb, 0x74, 0x14, 0x50, 0x9f, 0xf8, 0x91, 0xc0, - 0xc5, 0x31, 0x26, 0xfa, 0x12, 0xb4, 0xd2, 0x7c, 0xd6, 0x8f, 0x92, 0x0e, 0x2a, 0xd7, 0x97, 0xc9, - 0xb6, 0xaf, 0x5a, 0x62, 0xb2, 0x89, 0xb6, 0x0f, 0x7d, 0x05, 0xe6, 0xf7, 0x28, 0x8d, 0x58, 0x14, - 0xda, 0x41, 0xdf, 0xa5, 0x3e, 0x91, 0x61, 0xd4, 0x4a, 0x56, 0x3b, 0xd4, 0x27, 0x13, 0x9d, 0xdb, - 0x6c, 0x49, 0xe7, 0xf6, 0x0d, 0xb8, 0xda, 0xa6, 0x34, 0x74, 0x3d, 0xdf, 0x8e, 0x68, 0xb8, 0xa1, - 0xe8, 0x95, 0x4b, 0x2e, 0xc2, 0xec, 0x13, 0x12, 0x32, 0xd5, 0xc5, 0x19, 0x58, 0x81, 0xd6, 0xf7, - 0x61, 0xa9, 0x9c, 0x50, 0x16, 0xda, 0x33, 0x98, 0xf5, 0xef, 0x1a, 0x5c, 0x58, 0x77, 0xdd, 0x14, - 0x43, 0x49, 0xf3, 0x55, 0xd0, 0x3d, 0xf7, 0x74, 0x83, 0xea, 0x9e, 0xcb, 0x47, 0x95, 0x4c, 0x88, - 0xcf, 0x25, 0x31, 0x3c, 0x61, 0x8c, 0x92, 0x8a, 0x83, 0xd6, 0xe0, 0x9c, 0xc7, 0xfa, 0x3e, 0x39, - 0xec, 0xa7, 0xae, 0x21, 0xac, 0x56, 0xc7, 0x0b, 0x1e, 0xdb, 0x26, 0x87, 0xe9, 0x71, 0xe8, 0x3a, - 0x34, 0x0f, 0xe4, 0xa4, 0xd3, 0xf7, 0x5c, 0x91, 0xef, 0x5a, 0x18, 0xd4, 0x52, 0xd7, 0xb5, 0x7e, - 0xa3, 0xc1, 0x1b, 0x98, 0x8c, 0xe8, 0x13, 0x72, 0xa6, 0x0b, 0x2d, 0xc2, 0xac, 0x63, 0x33, 0xc7, - 0x76, 0x89, 0x6c, 0x5e, 0x15, 0xc8, 0x77, 0x42, 0xc1, 0xdf, 0x95, 0xbd, 0xb1, 0x02, 0x8b, 0xb2, - 0x55, 0x26, 0x64, 0xfb, 0xbd, 0x01, 0x57, 0x52, 0xa9, 0x26, 0xac, 0x7f, 0xc6, 0x50, 0x3a, 0xce, - 0x06, 0x97, 0x85, 0x6b, 0x84, 0x19, 0xf5, 0x27, 0x15, 0xc9, 0x81, 0x37, 0x23, 0x5e, 0xbe, 0xfa, - 0x51, 0xe8, 0x0d, 0x06, 0x24, 0xec, 0x93, 0x27, 0xc4, 0x8f, 0xfa, 0x69, 0x9e, 0x52, 0xf7, 0x38, - 0x31, 0x23, 0x5d, 0x13, 0x3c, 0x76, 0x63, 0x16, 0xf7, 0x38, 0x87, 0x6c, 0x8f, 0x5b, 0x6e, 0xde, - 0x6a, 0xb9, 0x79, 0x6d, 0x58, 0xc9, 0x0b, 0xc4, 0x27, 0xc2, 0x82, 0x3c, 0xb5, 0xd3, 0xe4, 0x59, - 0xca, 0xca, 0xc3, 0x7b, 0x89, 0x9c, 0x38, 0x05, 0x2b, 0xcd, 0x4e, 0x58, 0xe9, 0x57, 0x3a, 0x5c, - 0x2d, 0xb5, 0xd2, 0x74, 0x7a, 0xda, 0x3b, 0x50, 0xe5, 0x7d, 0x82, 0xaa, 0x17, 0xd7, 0x73, 0x74, - 0xc9, 0x69, 0x69, 0x57, 0x11, 0x63, 0xab, 0x8c, 0x65, 0xbc, 0xcc, 0xa0, 0xfa, 0x72, 0x39, 0xf0, - 0x6d, 0x40, 0x42, 0xbb, 0x79, 0xcc, 0xaa, 0xc0, 0x34, 0xf9, 0x4e, 0xb6, 0x2f, 0xb6, 0xfe, 0xa8, - 0xc3, 0x72, 0xaa, 0x95, 0x1d, 0xca, 0xa2, 0x69, 0xfb, 0xef, 0x4b, 0x39, 0xa3, 0x7e, 0x46, 0x67, - 0xbc, 0x05, 0xb3, 0x71, 0xd3, 0xc4, 0x63, 0x81, 0xeb, 0xff, 0x8d, 0x89, 0x6e, 0x62, 0x64, 0x77, - 0xfd, 0x47, 0x14, 0x2b, 0x3c, 0xf4, 0x1e, 0xcc, 0x09, 0x3d, 0x29, 0xba, 0xca, 0xc9, 0x74, 0x4d, - 0x8e, 0x1c, 0xc3, 0xcc, 0xfa, 0x8f, 0x06, 0xd7, 0x8f, 0xd5, 0xda, 0x74, 0xfc, 0xe9, 0xb5, 0xa8, - 0xed, 0x55, 0xbc, 0xcf, 0x7a, 0x0a, 0x90, 0xea, 0x23, 0x37, 0x4b, 0x6b, 0x85, 0x59, 0x7a, 0x59, - 0x61, 0x6e, 0xdb, 0x23, 0x55, 0xfd, 0x33, 0x2b, 0xe8, 0x06, 0xd4, 0x44, 0x20, 0x28, 0x63, 0x95, - 0x74, 0xde, 0x42, 0xe7, 0x12, 0xcb, 0x6a, 0xcb, 0x47, 0x37, 0x71, 0xf0, 0xf1, 0x8f, 0x6e, 0x4b, - 0x12, 0x2d, 0x73, 0x6a, 0xba, 0x60, 0xfd, 0x59, 0x07, 0x34, 0x19, 0x87, 0xbc, 0x78, 0x1c, 0x63, - 0x9c, 0x9c, 0x22, 0x75, 0xf9, 0xa8, 0xa7, 0xae, 0xac, 0x17, 0xae, 0xac, 0x46, 0x09, 0xe3, 0x25, - 0x46, 0x89, 0x0f, 0xc1, 0x74, 0x54, 0x8f, 0xd3, 0x8f, 0xeb, 0xb5, 0x88, 0xe4, 0x53, 0x1a, 0xa1, - 0x05, 0x27, 0x0b, 0x8f, 0xd9, 0x64, 0x3a, 0xa8, 0x96, 0xa4, 0x83, 0x77, 0xa0, 0xb9, 0x37, 0xa4, - 0xce, 0x81, 0x6c, 0xc5, 0xe2, 0x2c, 0x8b, 0xf2, 0x5e, 0x2e, 0xd8, 0x83, 0x40, 0x8b, 0xfb, 0x33, - 0xd5, 0x78, 0xce, 0x66, 0x1a, 0xcf, 0xc7, 0x70, 0x29, 0x75, 0xf9, 0xf6, 0x90, 0x32, 0x32, 0xa5, - 0x04, 0x91, 0xa9, 0xbc, 0x7a, 0xae, 0xf2, 0x5a, 0x21, 0xbc, 0x31, 0x71, 0xe4, 0x74, 0xa2, 0x8b, - 0x4f, 0x73, 0x63, 0xc7, 0x21, 0x8c, 0xa9, 0x33, 0x25, 0x68, 0xfd, 0x5c, 0x03, 0x33, 0x7d, 0x7c, - 0x89, 0x1d, 0x70, 0x0a, 0x6f, 0x57, 0x57, 0xa0, 0x2e, 0xdd, 0x34, 0xae, 0x10, 0x06, 0x4e, 0xe0, - 0x93, 0x9e, 0xa5, 0xac, 0x1f, 0x40, 0x55, 0xe0, 0x9d, 0xf2, 0xd6, 0x7c, 0x9c, 0x5b, 0x2e, 0x41, - 0xa3, 0x17, 0x0c, 0x3d, 0x91, 0x05, 0x64, 0x5f, 0x93, 0x2e, 0x58, 0x3e, 0xcc, 0x2b, 0xcc, 0x58, - 0x57, 0x27, 0x9c, 0xb2, 0x02, 0xcd, 0x8f, 0x87, 0x6e, 0xe1, 0xa0, 0xec, 0x12, 0xc7, 0xd8, 0x26, - 0x87, 0x85, 0x9b, 0x64, 0x97, 0xac, 0xdf, 0x19, 0x50, 0x8d, 0x1d, 0x6c, 0x09, 0x1a, 0x5d, 0xb6, - 0xc1, 0x1d, 0x8e, 0xc4, 0x9d, 0x5b, 0x1d, 0xa7, 0x0b, 0x5c, 0x0a, 0xf1, 0x99, 0xce, 0xda, 0x12, - 0x44, 0x77, 0xa1, 0x19, 0x7f, 0xaa, 0xf4, 0x31, 0x39, 0x78, 0x16, 0x8d, 0x87, 0xb3, 0x14, 0x68, - 0x13, 0xce, 0x6d, 0x13, 0xe2, 0x76, 0x42, 0x1a, 0x04, 0x0a, 0x43, 0xb6, 0x42, 0xa7, 0xb0, 0x99, - 0xa4, 0x43, 0xef, 0xc3, 0x02, 0x5f, 0x5c, 0x77, 0xdd, 0x84, 0x55, 0x3c, 0x7a, 0xa0, 0xc9, 0xf8, - 0xc7, 0x45, 0x54, 0x3e, 0x08, 0x7f, 0x12, 0xb8, 0x76, 0x44, 0xa4, 0x0a, 0xf9, 0x18, 0xcf, 0x89, - 0xaf, 0x96, 0x95, 0x20, 0x69, 0x20, 0x5c, 0x20, 0x29, 0xbe, 0xc8, 0xce, 0x4e, 0xbc, 0xc8, 0xa2, - 0xaf, 0x89, 0x59, 0x6b, 0x40, 0x16, 0xeb, 0xc2, 0x67, 0xf3, 0x05, 0x6e, 0x43, 0xc6, 0xfc, 0x20, - 0x9e, 0xb3, 0x06, 0xc4, 0xfa, 0x31, 0x5c, 0x48, 0xf2, 0x95, 0xda, 0xe5, 0xc9, 0xe6, 0x15, 0xf2, - 0xe4, 0xaa, 0x9a, 0xee, 0xf4, 0x63, 0x93, 0x8d, 0x1c, 0xea, 0xca, 0x5e, 0xee, 0xfe, 0xad, 0xc1, - 0x42, 0xe1, 0x27, 0x80, 0x57, 0x39, 0xbc, 0x2c, 0xb9, 0xea, 0xd3, 0x48, 0xae, 0x65, 0x23, 0xce, - 0x2d, 0xb8, 0x18, 0x97, 0x65, 0xe6, 0x3d, 0x23, 0xfd, 0x80, 0x84, 0x7d, 0x46, 0x1c, 0xea, 0xc7, - 0xcd, 0xb5, 0x8e, 0x91, 0xd8, 0xec, 0x79, 0xcf, 0xc8, 0x0e, 0x09, 0x7b, 0x62, 0xa7, 0xec, 0x49, - 0xc7, 0xfa, 0x83, 0x06, 0x28, 0xa3, 0xeb, 0x29, 0xe5, 0xd5, 0x8f, 0xa0, 0xb5, 0x97, 0x32, 0x4d, - 0x1e, 0x58, 0xdf, 0x2c, 0xaf, 0x4d, 0xd9, 0xf3, 0xf3, 0x74, 0xa5, 0x56, 0x72, 0x61, 0x2e, 0xdb, - 0x21, 0x70, 0x9c, 0xc8, 0x1b, 0xc5, 0x89, 0xb1, 0x81, 0xc5, 0x37, 0x5f, 0xf3, 0xa9, 0xab, 0x4a, - 0xb1, 0xf8, 0xe6, 0x6b, 0x8e, 0xe2, 0xd5, 0xc0, 0xe2, 0x9b, 0x87, 0xfb, 0x28, 0x7e, 0x05, 0x14, - 0x7a, 0x6b, 0x60, 0x05, 0x5a, 0xef, 0xc2, 0x5c, 0xf1, 0x41, 0x64, 0xdf, 0x1b, 0xec, 0xcb, 0xdf, - 0x25, 0xc4, 0x37, 0x32, 0xc1, 0x18, 0xd2, 0x43, 0x99, 0x28, 0xf8, 0x27, 0x97, 0x2d, 0xab, 0x96, - 0x97, 0xa3, 0x12, 0xd2, 0xf2, 0xc6, 0x41, 0x4a, 0xc6, 0xbf, 0x79, 0x6a, 0x55, 0x13, 0x84, 0x14, - 0x2d, 0x81, 0xad, 0x1f, 0xc2, 0xf5, 0x07, 0x74, 0x90, 0x99, 0xde, 0xd3, 0xa7, 0xcd, 0xe9, 0x18, - 0xd0, 0xfa, 0x89, 0x06, 0x2b, 0xc7, 0x1f, 0x31, 0x9d, 0x42, 0x78, 0xda, 0xbb, 0xe9, 0x90, 0xeb, - 0x92, 0x38, 0x07, 0x6c, 0x3c, 0xda, 0x22, 0x91, 0x8d, 0xbe, 0xae, 0x62, 0xbb, 0xac, 0x02, 0x2a, - 0xcc, 0x5c, 0x8c, 0xaf, 0x81, 0xe9, 0x64, 0xd7, 0x7b, 0xe4, 0xb1, 0x3c, 0x67, 0x62, 0xdd, 0xfa, - 0xa5, 0x06, 0x17, 0x33, 0x4f, 0xfe, 0x24, 0x52, 0x1c, 0xd1, 0x05, 0xa8, 0x3a, 0x74, 0xec, 0x47, - 0xd2, 0x88, 0x31, 0xc0, 0x3d, 0xe7, 0x29, 0x0d, 0xef, 0x73, 0xe3, 0xca, 0x42, 0x21, 0x41, 0x3e, - 0x35, 0x3f, 0xa5, 0xe1, 0x03, 0x7a, 0x28, 0xe3, 0x56, 0x42, 0x71, 0xe1, 0x1f, 0x09, 0x8a, 0x8a, - 0x1c, 0x9a, 0x63, 0x90, 0x53, 0xb0, 0xf1, 0x88, 0x53, 0xc4, 0x6d, 0x94, 0x84, 0xac, 0xdf, 0x6a, - 0xb0, 0x52, 0x2a, 0xd3, 0xba, 0x73, 0x30, 0x2d, 0x2b, 0x5c, 0x80, 0x2a, 0x09, 0xa8, 0xa3, 0x6e, - 0x11, 0x03, 0x65, 0x71, 0xa7, 0x7e, 0x59, 0xac, 0x24, 0xbf, 0x2c, 0x5a, 0xff, 0xd4, 0xc0, 0x2a, - 0x95, 0x2f, 0xae, 0x14, 0x53, 0x4a, 0x26, 0x67, 0x90, 0x10, 0x7d, 0x00, 0x75, 0x65, 0x69, 0xa1, - 0xdb, 0xe2, 0x6f, 0x5d, 0xa5, 0xd2, 0xe3, 0x84, 0x66, 0xed, 0x1a, 0xd4, 0xe4, 0xaf, 0xa1, 0x0d, - 0xa8, 0x3e, 0x0c, 0xbd, 0x88, 0x98, 0x33, 0xa8, 0x0e, 0x95, 0x1d, 0x9b, 0x31, 0x53, 0x5b, 0x5b, - 0x8d, 0xbb, 0x98, 0xf4, 0x75, 0x18, 0x01, 0xd4, 0xda, 0x21, 0xb1, 0x05, 0x1e, 0x40, 0x2d, 0x7e, - 0x37, 0x32, 0xb5, 0xb5, 0xf7, 0x00, 0xd2, 0x82, 0xc7, 0x39, 0x6c, 0x7f, 0xbc, 0x7d, 0xcf, 0x9c, - 0x41, 0x4d, 0x98, 0x7d, 0xb8, 0xde, 0xdd, 0xed, 0x6e, 0x7f, 0x64, 0x6a, 0x02, 0xc0, 0x31, 0xa0, - 0x73, 0x9c, 0x0e, 0xc7, 0x31, 0xd6, 0xde, 0x2e, 0xb4, 0x80, 0x68, 0x16, 0x8c, 0xf5, 0xe1, 0xd0, - 0x9c, 0x41, 0x35, 0xd0, 0x3b, 0x1b, 0xa6, 0xc6, 0x4f, 0xda, 0xa6, 0xe1, 0xc8, 0x1e, 0x9a, 0xfa, - 0xda, 0x33, 0x98, 0xcf, 0x17, 0x18, 0xc1, 0x96, 0x86, 0x07, 0x9e, 0x3f, 0x88, 0x0f, 0xec, 0x45, - 0xa2, 0x93, 0x88, 0x0f, 0x8c, 0x25, 0x74, 0x4d, 0x1d, 0x99, 0x30, 0xd7, 0xf5, 0xbd, 0xc8, 0xb3, - 0x87, 0xde, 0x33, 0x8e, 0x6b, 0xa0, 0x16, 0x34, 0x76, 0x42, 0x12, 0xd8, 0x21, 0x07, 0x2b, 0x68, - 0x1e, 0x40, 0x3c, 0x7c, 0x63, 0x62, 0xbb, 0x47, 0x66, 0x95, 0x13, 0x3c, 0xb4, 0xbd, 0xc8, 0xf3, - 0x07, 0x62, 0xd9, 0xac, 0xad, 0x7d, 0x1b, 0x5a, 0xb9, 0x40, 0x44, 0xe7, 0xa0, 0xf5, 0xc9, 0x76, - 0x77, 0xbb, 0xbb, 0xdb, 0x5d, 0x7f, 0xd0, 0xfd, 0xf4, 0x5e, 0xc7, 0x9c, 0x41, 0x73, 0x50, 0xdf, - 0xea, 0xf6, 0xb6, 0xd6, 0x77, 0xdb, 0xf7, 0x4d, 0x8d, 0xab, 0x35, 0xfe, 0xd4, 0x37, 0x3e, 0xf8, - 0xdb, 0xf3, 0x65, 0xed, 0xf3, 0xe7, 0xcb, 0xda, 0xbf, 0x9e, 0x2f, 0x6b, 0xbf, 0x78, 0xb1, 0x3c, - 0xf3, 0xf9, 0x8b, 0xe5, 0x99, 0x7f, 0xbc, 0x58, 0x9e, 0xf9, 0xf4, 0xcb, 0x03, 0x2f, 0xda, 0x1f, - 0xef, 0xdd, 0x70, 0xe8, 0xe8, 0x66, 0xe0, 0xf9, 0x03, 0xc7, 0x0e, 0x6e, 0x46, 0x9e, 0xe3, 0x3a, - 0x37, 0x33, 0xb6, 0xdc, 0xab, 0x89, 0xff, 0x4c, 0xbc, 0xf3, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, - 0x48, 0x01, 0xd7, 0x2e, 0x52, 0x21, 0x00, 0x00, + 0xd5, 0xdd, 0xf3, 0xe1, 0x99, 0x37, 0x1e, 0xbb, 0x53, 0xf9, 0x72, 0x12, 0xc7, 0xf1, 0x36, 0x20, + 0x8c, 0x77, 0x49, 0x48, 0x76, 0x23, 0x60, 0x59, 0x36, 0xd8, 0x33, 0xd9, 0xcd, 0xc8, 0xb1, 0xd7, + 0xaa, 0xf1, 0x2a, 0xb0, 0x1c, 0x86, 0x76, 0x77, 0x65, 0xdc, 0xf2, 0x4c, 0x57, 0xa7, 0xab, 0x27, + 0x8e, 0x23, 0x2d, 0x08, 0x21, 0x6e, 0x1c, 0x40, 0x70, 0xe1, 0xc0, 0x01, 0xf1, 0x07, 0x90, 0x10, + 0x47, 0x24, 0x6e, 0x20, 0x71, 0xd9, 0xd3, 0x8a, 0x1b, 0x28, 0xf9, 0x03, 0x08, 0x0e, 0x5c, 0x57, + 0x55, 0x5d, 0xd5, 0x5f, 0xd3, 0x63, 0xc7, 0xf2, 0x28, 0xb7, 0x7e, 0x55, 0xef, 0xbd, 0x7a, 0xf5, + 0xbe, 0x5f, 0xcd, 0xc0, 0xb5, 0x7d, 0x62, 0x05, 0xe1, 0x1e, 0xb1, 0x42, 0x7f, 0xef, 0x56, 0xfc, + 0x7d, 0xd3, 0x0f, 0x68, 0x48, 0x51, 0x23, 0xb5, 0x69, 0x1e, 0x41, 0x7d, 0xd7, 0xda, 0x1b, 0x90, + 0xae, 0x6f, 0x79, 0x68, 0x11, 0x66, 0x05, 0xd0, 0x69, 0x2f, 0x6a, 0x2b, 0xda, 0x6a, 0x09, 0x2b, + 0x10, 0x5d, 0x85, 0x5a, 0x37, 0xb4, 0x82, 0x70, 0x93, 0x1c, 0x2d, 0xea, 0x2b, 0xda, 0xea, 0x1c, + 0x8e, 0x61, 0x74, 0x09, 0xaa, 0xf7, 0x3d, 0x87, 0xef, 0x94, 0xc4, 0x8e, 0x84, 0xd0, 0x32, 0xc0, + 0x26, 0x39, 0x62, 0xbe, 0x65, 0x73, 0x86, 0xe5, 0x15, 0x6d, 0xb5, 0x89, 0x53, 0x2b, 0xe6, 0xe7, + 0x3a, 0x18, 0x0f, 0xb8, 0x28, 0x1b, 0xc4, 0x0a, 0x31, 0x79, 0x32, 0x22, 0x2c, 0x44, 0xdf, 0x85, + 0x39, 0x7b, 0xdf, 0xf2, 0xfa, 0xe4, 0x31, 0x21, 0x8e, 0x94, 0xa3, 0x71, 0xe7, 0xca, 0xcd, 0x94, + 0xcc, 0x37, 0x5b, 0x29, 0x04, 0x9c, 0x41, 0x47, 0xef, 0x40, 0xfd, 0xd0, 0x0a, 0x49, 0x30, 0xb4, + 0x82, 0x03, 0x21, 0x68, 0xe3, 0xce, 0xa5, 0x0c, 0xed, 0x23, 0xb5, 0x8b, 0x13, 0x44, 0xf4, 0x1e, + 0x34, 0x03, 0xe2, 0xd0, 0x78, 0x4f, 0x5c, 0x64, 0x32, 0x65, 0x16, 0x19, 0x7d, 0x0b, 0x6a, 0x2c, + 0xb4, 0xc2, 0x11, 0x23, 0x6c, 0xb1, 0xbc, 0x52, 0x5a, 0x6d, 0xdc, 0x59, 0xca, 0x10, 0xc6, 0xfa, + 0xed, 0x0a, 0x2c, 0x1c, 0x63, 0xa3, 0x55, 0x58, 0xb0, 0xe9, 0xd0, 0x27, 0x03, 0x12, 0x92, 0x68, + 0x73, 0xb1, 0xb2, 0xa2, 0xad, 0xd6, 0x70, 0x7e, 0x19, 0xbd, 0x09, 0x25, 0x12, 0x04, 0x8b, 0xd5, + 0x02, 0x6d, 0xe0, 0x91, 0xe7, 0xb9, 0x5e, 0xff, 0x7e, 0x10, 0xd0, 0x00, 0x73, 0x2c, 0xf3, 0xe7, + 0x1a, 0xd4, 0x13, 0xf1, 0x4c, 0xae, 0x51, 0x62, 0x1f, 0xf8, 0xd4, 0xf5, 0xc2, 0x5d, 0x26, 0x34, + 0x5a, 0xc6, 0x99, 0x35, 0x6e, 0xaa, 0x80, 0x30, 0x3a, 0x78, 0x4a, 0x9c, 0x5d, 0x26, 0xf4, 0x56, + 0xc6, 0xa9, 0x15, 0x64, 0x40, 0x89, 0x91, 0x27, 0x42, 0x2d, 0x65, 0xcc, 0x3f, 0x39, 0xd7, 0x81, + 0xc5, 0xc2, 0xee, 0x91, 0x67, 0x0b, 0x9a, 0x72, 0xc4, 0x35, 0xbd, 0x66, 0x7e, 0x0a, 0x46, 0xdb, + 0x65, 0xbe, 0x15, 0xda, 0xfb, 0x24, 0x58, 0xb7, 0x43, 0x97, 0x7a, 0xe8, 0x4d, 0xa8, 0x5a, 0xe2, + 0x4b, 0xc8, 0x31, 0x7f, 0xe7, 0x7c, 0xe6, 0x2e, 0x11, 0x12, 0x96, 0x28, 0xdc, 0xeb, 0x5a, 0x74, + 0x38, 0x74, 0xc3, 0x58, 0xa8, 0x18, 0x46, 0x2b, 0xd0, 0xe8, 0x30, 0x7e, 0xd4, 0x0e, 0xbf, 0x83, + 0x10, 0xad, 0x86, 0xd3, 0x4b, 0x66, 0x0b, 0x4a, 0xeb, 0xad, 0xcd, 0x0c, 0x13, 0xed, 0x78, 0x26, + 0xfa, 0x38, 0x93, 0x9f, 0xe9, 0x70, 0xb1, 0xe3, 0x3d, 0x1e, 0x8c, 0x08, 0xbf, 0x54, 0x72, 0x1d, + 0x86, 0xbe, 0x07, 0xcd, 0x78, 0x63, 0xf7, 0xc8, 0x27, 0xf2, 0x42, 0x57, 0x33, 0x17, 0xca, 0x60, + 0xe0, 0x2c, 0x01, 0xba, 0x07, 0xcd, 0x84, 0x61, 0xa7, 0xcd, 0xef, 0x58, 0x1a, 0x33, 0x6f, 0x1a, + 0x03, 0x67, 0xf1, 0x45, 0x54, 0xda, 0xfb, 0x64, 0x68, 0x75, 0xda, 0x42, 0x01, 0x25, 0x1c, 0xc3, + 0x68, 0x13, 0xce, 0x93, 0x67, 0xf6, 0x60, 0xe4, 0x90, 0x14, 0x8d, 0x23, 0xec, 0x74, 0xec, 0x11, + 0x45, 0x54, 0xe6, 0xdf, 0xb4, 0xb4, 0x29, 0xa5, 0x4f, 0x7e, 0x1f, 0x2e, 0xba, 0x45, 0x9a, 0x91, + 0x31, 0x6b, 0x16, 0x2b, 0x22, 0x8d, 0x89, 0x8b, 0x19, 0xa0, 0xbb, 0xb1, 0x93, 0x44, 0x21, 0x7c, + 0x7d, 0x82, 0xb8, 0x39, 0x77, 0x31, 0xa1, 0x64, 0xd9, 0x2a, 0x78, 0x8d, 0xac, 0x63, 0xb5, 0x36, + 0x31, 0xdf, 0x34, 0xff, 0xac, 0xc1, 0xb9, 0x54, 0xd2, 0x61, 0x3e, 0xf5, 0x18, 0x39, 0x6b, 0xd6, + 0xd9, 0x02, 0xe4, 0xe4, 0xb4, 0x43, 0x94, 0x35, 0x27, 0xc9, 0x2e, 0x93, 0x41, 0x01, 0x21, 0x42, + 0x50, 0x1e, 0x52, 0x87, 0x48, 0x93, 0x8a, 0x6f, 0xf3, 0x19, 0x9c, 0x6f, 0xa5, 0x22, 0x76, 0x8b, + 0x30, 0x66, 0xf5, 0xcf, 0x2c, 0x78, 0x3e, 0x37, 0xe8, 0xe3, 0xb9, 0xc1, 0xfc, 0x8d, 0x06, 0x0b, + 0x98, 0x38, 0x74, 0x8b, 0x84, 0xd6, 0x94, 0x8e, 0x3d, 0x29, 0xdd, 0xe4, 0xc5, 0x2a, 0x15, 0x88, + 0xf5, 0x63, 0xb8, 0xce, 0xa5, 0xc2, 0x31, 0xd5, 0x4e, 0x40, 0xfb, 0x01, 0x61, 0xec, 0xf5, 0xc8, + 0x68, 0x7e, 0x0a, 0x4b, 0xd9, 0xf3, 0x3f, 0xa0, 0xc1, 0xa1, 0x15, 0x38, 0xaf, 0xe9, 0xf8, 0xff, + 0x65, 0x22, 0xb2, 0x45, 0xbd, 0xc7, 0x6e, 0x1f, 0xad, 0x41, 0x99, 0xf9, 0x96, 0x27, 0xcf, 0xba, + 0x54, 0x5c, 0x85, 0xb0, 0xc0, 0xe1, 0xb5, 0x9e, 0xf1, 0x0a, 0x1e, 0x73, 0x57, 0x20, 0x97, 0xdc, + 0x49, 0x65, 0x04, 0x19, 0x4f, 0xc7, 0xa4, 0x8c, 0x0c, 0x3a, 0x4f, 0x4a, 0x4c, 0x25, 0xa5, 0x72, + 0x94, 0x94, 0x14, 0x1c, 0x7b, 0x76, 0x25, 0xf1, 0x6c, 0xb4, 0x06, 0x06, 0x3b, 0x70, 0xfd, 0xf6, + 0xd6, 0xc3, 0x75, 0xd6, 0x95, 0x12, 0x55, 0x45, 0x22, 0x1e, 0x5b, 0x37, 0x3f, 0xd7, 0xe0, 0x0a, + 0xcf, 0x70, 0xce, 0x68, 0x90, 0x4a, 0x50, 0x53, 0xea, 0x1d, 0xee, 0x42, 0xd5, 0x16, 0x7a, 0x3c, + 0x21, 0xeb, 0x44, 0xca, 0xc6, 0x12, 0x19, 0xb5, 0x60, 0x9e, 0x49, 0x91, 0xa2, 0x7c, 0x24, 0x14, + 0x36, 0x7f, 0xe7, 0x5a, 0x86, 0xbc, 0x9b, 0x41, 0xc1, 0x39, 0x12, 0xf3, 0xff, 0x1a, 0x5c, 0xda, + 0x22, 0x41, 0x7f, 0xfa, 0xb7, 0xba, 0x07, 0x4d, 0xe7, 0x94, 0x45, 0x26, 0x83, 0x8f, 0x3a, 0x80, + 0x86, 0x5c, 0x32, 0xa7, 0x7d, 0x2a, 0xa7, 0x28, 0x20, 0x8a, 0xcd, 0x5f, 0x4e, 0x25, 0xb6, 0x1d, + 0x38, 0xbf, 0x65, 0xb9, 0x5e, 0x68, 0xb9, 0x1e, 0x09, 0x1e, 0x28, 0x6e, 0xe8, 0xdb, 0xa9, 0xa6, + 0x4a, 0x2b, 0x48, 0xa4, 0x09, 0x4d, 0xbe, 0xab, 0x32, 0xff, 0xaa, 0x83, 0x91, 0xdf, 0x3e, 0xab, + 0x16, 0xaf, 0x03, 0xf0, 0xaf, 0x1e, 0x3f, 0x84, 0x08, 0xff, 0xa8, 0xe3, 0x3a, 0x5f, 0xe1, 0xec, + 0x09, 0xba, 0x0d, 0x95, 0x68, 0xa7, 0xc8, 0xf4, 0x2d, 0x3a, 0xf4, 0xa9, 0x47, 0xbc, 0x50, 0xe0, + 0xe2, 0x08, 0x13, 0x7d, 0x09, 0x9a, 0x49, 0x3e, 0xeb, 0x85, 0x71, 0x07, 0x95, 0xe9, 0xcb, 0x64, + 0xdb, 0x57, 0x29, 0x30, 0xd9, 0x58, 0xdb, 0x87, 0xbe, 0x02, 0xf3, 0x7b, 0x94, 0x86, 0x2c, 0x0c, + 0x2c, 0xbf, 0xe7, 0x50, 0x8f, 0xc8, 0x30, 0x6a, 0xc6, 0xab, 0x6d, 0xea, 0x91, 0xb1, 0xce, 0x6d, + 0xb6, 0xa0, 0x73, 0xfb, 0x26, 0x5c, 0x6b, 0x51, 0x1a, 0x38, 0xae, 0x67, 0x85, 0x34, 0xd8, 0x50, + 0xf4, 0xca, 0x25, 0x17, 0x61, 0xf6, 0x29, 0x09, 0x98, 0xea, 0xe2, 0x4a, 0x58, 0x81, 0xe6, 0x0f, + 0x60, 0xa9, 0x98, 0x50, 0x16, 0xda, 0x33, 0x98, 0xf5, 0x1f, 0x1a, 0x5c, 0x58, 0x77, 0x9c, 0x04, + 0x43, 0x49, 0xf3, 0x35, 0xd0, 0x5d, 0xe7, 0x64, 0x83, 0xea, 0xae, 0xc3, 0x47, 0x95, 0x54, 0x88, + 0xcf, 0xc5, 0x31, 0x3c, 0x66, 0x8c, 0x82, 0x8a, 0x83, 0xd6, 0xe0, 0x9c, 0xcb, 0x7a, 0x1e, 0x39, + 0xec, 0x25, 0xae, 0x21, 0xac, 0x56, 0xc3, 0x0b, 0x2e, 0xdb, 0x26, 0x87, 0xc9, 0x71, 0xe8, 0x06, + 0x34, 0x0e, 0xe4, 0xa4, 0xd3, 0x73, 0x1d, 0x91, 0xef, 0x9a, 0x18, 0xd4, 0x52, 0xc7, 0x31, 0x7f, + 0xab, 0xc1, 0x65, 0x4c, 0x86, 0xf4, 0x29, 0x39, 0xd3, 0x85, 0x16, 0x61, 0xd6, 0xb6, 0x98, 0x6d, + 0x39, 0x44, 0x36, 0xaf, 0x0a, 0xe4, 0x3b, 0x81, 0xe0, 0xef, 0xc8, 0xde, 0x58, 0x81, 0x79, 0xd9, + 0xca, 0x63, 0xb2, 0xfd, 0xa1, 0x04, 0x57, 0x13, 0xa9, 0xc6, 0xac, 0x7f, 0xc6, 0x50, 0x9a, 0x64, + 0x83, 0x2b, 0xc2, 0x35, 0x82, 0x94, 0xfa, 0xe3, 0x8a, 0x64, 0xc3, 0x1b, 0x21, 0x2f, 0x5f, 0xbd, + 0x30, 0x70, 0xfb, 0x7d, 0x12, 0xf4, 0xc8, 0x53, 0xe2, 0x85, 0xbd, 0x24, 0x4f, 0xa9, 0x7b, 0x1c, + 0x9b, 0x91, 0xae, 0x0b, 0x1e, 0xbb, 0x11, 0x8b, 0xfb, 0x9c, 0x43, 0xba, 0xc7, 0x2d, 0x36, 0x6f, + 0xa5, 0xd8, 0xbc, 0x16, 0xac, 0x64, 0x05, 0xe2, 0x13, 0x61, 0x4e, 0x9e, 0xea, 0x49, 0xf2, 0x2c, + 0xa5, 0xe5, 0xe1, 0xbd, 0x44, 0x46, 0x9c, 0x9c, 0x95, 0x66, 0xc7, 0xac, 0xf4, 0x6b, 0x1d, 0xae, + 0x15, 0x5a, 0x69, 0x3a, 0x3d, 0xed, 0x5d, 0xa8, 0xf0, 0x3e, 0x41, 0xd5, 0x8b, 0x1b, 0x19, 0xba, + 0xf8, 0xb4, 0xa4, 0xab, 0x88, 0xb0, 0x55, 0xc6, 0x2a, 0xbd, 0xca, 0xa0, 0xfa, 0x6a, 0x39, 0xf0, + 0x2d, 0x40, 0x42, 0xbb, 0x59, 0xcc, 0x8a, 0xc0, 0x34, 0xf8, 0x4e, 0xba, 0x2f, 0x36, 0xff, 0xa4, + 0xc3, 0x72, 0xa2, 0x95, 0x1d, 0xca, 0xc2, 0x69, 0xfb, 0xef, 0x2b, 0x39, 0xa3, 0x7e, 0x46, 0x67, + 0xbc, 0x0d, 0xb3, 0x51, 0xd3, 0xc4, 0x63, 0x81, 0xeb, 0xff, 0xf2, 0x58, 0x37, 0x31, 0xb4, 0x3a, + 0xde, 0x63, 0x8a, 0x15, 0x1e, 0x7a, 0x17, 0xe6, 0x84, 0x9e, 0x14, 0x5d, 0xf9, 0x78, 0xba, 0x06, + 0x47, 0x8e, 0x60, 0x66, 0xfe, 0x57, 0x83, 0x1b, 0x13, 0xb5, 0x36, 0x1d, 0x7f, 0x7a, 0x2d, 0x6a, + 0x3b, 0x8d, 0xf7, 0x99, 0xcf, 0x00, 0x12, 0x7d, 0x64, 0x66, 0x69, 0x2d, 0x37, 0x4b, 0x2f, 0x2b, + 0xcc, 0x6d, 0x6b, 0xa8, 0xaa, 0x7f, 0x6a, 0x05, 0xdd, 0x84, 0xaa, 0x08, 0x04, 0x65, 0xac, 0x82, + 0xce, 0x5b, 0xe8, 0x5c, 0x62, 0x99, 0x2d, 0xf9, 0xe8, 0x26, 0x0e, 0x9e, 0xfc, 0xe8, 0xb6, 0x24, + 0xd1, 0x52, 0xa7, 0x26, 0x0b, 0xe6, 0x5f, 0x74, 0x40, 0xe3, 0x71, 0xc8, 0x8b, 0xc7, 0x04, 0xe3, + 0x64, 0x14, 0xa9, 0xcb, 0x47, 0x3d, 0x75, 0x65, 0x3d, 0x77, 0x65, 0x35, 0x4a, 0x94, 0x5e, 0x61, + 0x94, 0xf8, 0x00, 0x0c, 0x5b, 0xf5, 0x38, 0xbd, 0xa8, 0x5e, 0x8b, 0x48, 0x3e, 0xa1, 0x11, 0x5a, + 0xb0, 0xd3, 0xf0, 0x88, 0x8d, 0xa7, 0x83, 0x4a, 0x41, 0x3a, 0x78, 0x1b, 0x1a, 0x7b, 0x03, 0x6a, + 0x1f, 0xc8, 0x56, 0x2c, 0xca, 0xb2, 0x28, 0xeb, 0xe5, 0x82, 0x3d, 0x08, 0xb4, 0xa8, 0x3f, 0x53, + 0x8d, 0xe7, 0x6c, 0xaa, 0xf1, 0x7c, 0x02, 0x97, 0x12, 0x97, 0x6f, 0x0d, 0x28, 0x23, 0x53, 0x4a, + 0x10, 0xa9, 0xca, 0xab, 0x67, 0x2a, 0xaf, 0x19, 0xc0, 0xe5, 0xb1, 0x23, 0xa7, 0x13, 0x5d, 0x7c, + 0x9a, 0x1b, 0xd9, 0x36, 0x61, 0x4c, 0x9d, 0x29, 0x41, 0xf3, 0x17, 0x1a, 0x18, 0xc9, 0xe3, 0x4b, + 0xe4, 0x80, 0x53, 0x78, 0xbb, 0xba, 0x0a, 0x35, 0xe9, 0xa6, 0x51, 0x85, 0x28, 0xe1, 0x18, 0x3e, + 0xee, 0x59, 0xca, 0xfc, 0x21, 0x54, 0x04, 0xde, 0x09, 0x6f, 0xcd, 0x93, 0xdc, 0x72, 0x09, 0xea, + 0x5d, 0x7f, 0xe0, 0x8a, 0x2c, 0x20, 0xfb, 0x9a, 0x64, 0xc1, 0xf4, 0x60, 0x5e, 0x61, 0x46, 0xba, + 0x3a, 0xe6, 0x94, 0x15, 0x68, 0x7c, 0x34, 0x70, 0x72, 0x07, 0xa5, 0x97, 0x38, 0xc6, 0x36, 0x39, + 0xcc, 0xdd, 0x24, 0xbd, 0x64, 0xfe, 0xbe, 0x04, 0x95, 0xc8, 0xc1, 0x96, 0xa0, 0xde, 0x61, 0x1b, + 0xdc, 0xe1, 0x48, 0xd4, 0xb9, 0xd5, 0x70, 0xb2, 0xc0, 0xa5, 0x10, 0x9f, 0xc9, 0xac, 0x2d, 0x41, + 0x74, 0x0f, 0x1a, 0xd1, 0xa7, 0x4a, 0x1f, 0xe3, 0x83, 0x67, 0xde, 0x78, 0x38, 0x4d, 0x81, 0x36, + 0xe1, 0xdc, 0x36, 0x21, 0x4e, 0x3b, 0xa0, 0xbe, 0xaf, 0x30, 0x64, 0x2b, 0x74, 0x02, 0x9b, 0x71, + 0x3a, 0xf4, 0x1e, 0x2c, 0xf0, 0xc5, 0x75, 0xc7, 0x89, 0x59, 0x45, 0xa3, 0x07, 0x1a, 0x8f, 0x7f, + 0x9c, 0x47, 0xe5, 0x83, 0xf0, 0xc7, 0xbe, 0x63, 0x85, 0x44, 0xaa, 0x90, 0x8f, 0xf1, 0x9c, 0xf8, + 0x5a, 0x51, 0x09, 0x92, 0x06, 0xc2, 0x39, 0x92, 0xfc, 0x8b, 0xec, 0xec, 0xd8, 0x8b, 0x2c, 0xfa, + 0xba, 0x98, 0xb5, 0xfa, 0x64, 0xb1, 0x26, 0x7c, 0x36, 0x5b, 0xe0, 0x36, 0x64, 0xcc, 0xf7, 0xa3, + 0x39, 0xab, 0x4f, 0xcc, 0x9f, 0xc0, 0x85, 0x38, 0x5f, 0xa9, 0x5d, 0x9e, 0x6c, 0x4e, 0x91, 0x27, + 0x57, 0xd5, 0x74, 0xa7, 0x4f, 0x4c, 0x36, 0x72, 0xa8, 0x2b, 0x7a, 0xb9, 0xfb, 0x8f, 0x06, 0x0b, + 0xb9, 0x9f, 0x00, 0x4e, 0x73, 0x78, 0x51, 0x72, 0xd5, 0xa7, 0x91, 0x5c, 0x8b, 0x46, 0x9c, 0xdb, + 0x70, 0x31, 0x2a, 0xcb, 0xcc, 0x7d, 0x4e, 0x7a, 0x3e, 0x09, 0x7a, 0x8c, 0xd8, 0xd4, 0x8b, 0x9a, + 0x6b, 0x1d, 0x23, 0xb1, 0xd9, 0x75, 0x9f, 0x93, 0x1d, 0x12, 0x74, 0xc5, 0x4e, 0xd1, 0x93, 0x8e, + 0xf9, 0x47, 0x0d, 0x50, 0x4a, 0xd7, 0x53, 0xca, 0xab, 0x1f, 0x42, 0x73, 0x2f, 0x61, 0x1a, 0x3f, + 0xb0, 0xbe, 0x51, 0x5c, 0x9b, 0xd2, 0xe7, 0x67, 0xe9, 0x0a, 0xad, 0xe4, 0xc0, 0x5c, 0xba, 0x43, + 0xe0, 0x38, 0xa1, 0x3b, 0x8c, 0x12, 0x63, 0x1d, 0x8b, 0x6f, 0xbe, 0xe6, 0x51, 0x47, 0x95, 0x62, + 0xf1, 0xcd, 0xd7, 0x6c, 0xc5, 0xab, 0x8e, 0xc5, 0x37, 0x0f, 0xf7, 0x61, 0xf4, 0x0a, 0x28, 0xf4, + 0x56, 0xc7, 0x0a, 0x34, 0xdf, 0x81, 0xb9, 0xfc, 0x83, 0xc8, 0xbe, 0xdb, 0xdf, 0x97, 0xbf, 0x4b, + 0x88, 0x6f, 0x64, 0x40, 0x69, 0x40, 0x0f, 0x65, 0xa2, 0xe0, 0x9f, 0x5c, 0xb6, 0xb4, 0x5a, 0x5e, + 0x8d, 0x4a, 0x48, 0xcb, 0x1b, 0x07, 0x29, 0x19, 0xff, 0xe6, 0xa9, 0x55, 0x4d, 0x10, 0x52, 0xb4, + 0x18, 0x36, 0x7f, 0x04, 0x37, 0x1e, 0xd2, 0x7e, 0x6a, 0x7a, 0x4f, 0x9e, 0x36, 0xa7, 0x63, 0x40, + 0xf3, 0xa7, 0x1a, 0xac, 0x4c, 0x3e, 0x62, 0x3a, 0x85, 0xf0, 0xa4, 0x77, 0xd3, 0x01, 0xd7, 0x25, + 0xb1, 0x0f, 0xd8, 0x68, 0xb8, 0x45, 0x42, 0x0b, 0x7d, 0x43, 0xc5, 0x76, 0x51, 0x05, 0x54, 0x98, + 0x99, 0x18, 0x5f, 0x03, 0xc3, 0x4e, 0xaf, 0x77, 0xc9, 0x13, 0x79, 0xce, 0xd8, 0xba, 0xf9, 0x2b, + 0x0d, 0x2e, 0xa6, 0x9e, 0xfc, 0x49, 0xa8, 0x38, 0xa2, 0x0b, 0x50, 0xb1, 0xe9, 0xc8, 0x0b, 0xa5, + 0x11, 0x23, 0x80, 0x7b, 0xce, 0x33, 0x1a, 0x3c, 0xe0, 0xc6, 0x95, 0x85, 0x42, 0x82, 0x7c, 0x6a, + 0x7e, 0x46, 0x83, 0x87, 0xf4, 0x50, 0xc6, 0xad, 0x84, 0xa2, 0xc2, 0x3f, 0x14, 0x14, 0x65, 0x39, + 0x34, 0x47, 0x20, 0xa7, 0x60, 0xa3, 0x21, 0xa7, 0x88, 0xda, 0x28, 0x09, 0x99, 0xbf, 0xd3, 0x60, + 0xa5, 0x50, 0xa6, 0x75, 0xfb, 0x60, 0x5a, 0x56, 0xb8, 0x00, 0x15, 0xe2, 0x53, 0x5b, 0xdd, 0x22, + 0x02, 0x8a, 0xe2, 0x4e, 0xfd, 0xb2, 0x58, 0x8e, 0x7f, 0x59, 0x34, 0xff, 0xa5, 0x81, 0x59, 0x28, + 0x5f, 0x54, 0x29, 0xa6, 0x94, 0x4c, 0xce, 0x20, 0x21, 0x7a, 0x1f, 0x6a, 0xca, 0xd2, 0x42, 0xb7, + 0xf9, 0xdf, 0xba, 0x0a, 0xa5, 0xc7, 0x31, 0xcd, 0xda, 0x57, 0xa1, 0x2a, 0x7f, 0x0d, 0xad, 0x43, + 0xe5, 0x51, 0xe0, 0x86, 0xc4, 0x98, 0x41, 0x35, 0x28, 0xef, 0x58, 0x8c, 0x19, 0x1a, 0xff, 0xea, + 0x1e, 0xb8, 0xbe, 0xa1, 0xaf, 0xad, 0x46, 0xfd, 0x4c, 0xf2, 0x4e, 0x8c, 0x00, 0xaa, 0xad, 0x80, + 0x58, 0x82, 0x02, 0xa0, 0x1a, 0xbd, 0x20, 0x19, 0xda, 0xda, 0xbb, 0x00, 0x49, 0xe9, 0xe3, 0x1c, + 0xb6, 0x3f, 0xda, 0xbe, 0x6f, 0xcc, 0xa0, 0x06, 0xcc, 0x3e, 0x5a, 0xef, 0xec, 0x76, 0xb6, 0x3f, + 0x34, 0x34, 0x01, 0xe0, 0x08, 0xd0, 0x39, 0x4e, 0x9b, 0xe3, 0x94, 0xd6, 0xde, 0xca, 0x35, 0x83, + 0x68, 0x16, 0x4a, 0xeb, 0x83, 0x81, 0x31, 0x83, 0xaa, 0xa0, 0xb7, 0x37, 0x0c, 0x8d, 0x9f, 0xb4, + 0x4d, 0x83, 0xa1, 0x35, 0x30, 0xf4, 0xb5, 0xe7, 0x30, 0x9f, 0x2d, 0x35, 0x82, 0x2d, 0x0d, 0x0e, + 0x5c, 0xaf, 0x1f, 0x1d, 0xd8, 0x0d, 0x45, 0x4f, 0x11, 0x1d, 0x18, 0x49, 0xe8, 0x18, 0x3a, 0x32, + 0x60, 0xae, 0xe3, 0xb9, 0xa1, 0x6b, 0x0d, 0xdc, 0xe7, 0x1c, 0xb7, 0x84, 0x9a, 0x50, 0xdf, 0x09, + 0x88, 0x6f, 0x05, 0x1c, 0x2c, 0xa3, 0x79, 0x00, 0xf1, 0x04, 0x8e, 0x89, 0xe5, 0x1c, 0x19, 0x15, + 0x4e, 0xf0, 0xc8, 0x72, 0x43, 0xd7, 0xeb, 0x8b, 0x65, 0xa3, 0xba, 0xf6, 0x1d, 0x68, 0x66, 0x42, + 0x12, 0x9d, 0x83, 0xe6, 0xc7, 0xdb, 0x9d, 0xed, 0xce, 0x6e, 0x67, 0xfd, 0x61, 0xe7, 0x93, 0xfb, + 0x6d, 0x63, 0x06, 0xcd, 0x41, 0x6d, 0xab, 0xd3, 0xdd, 0x5a, 0xdf, 0x6d, 0x3d, 0x30, 0x34, 0xae, + 0xe0, 0xe8, 0x53, 0xdf, 0x78, 0xff, 0xef, 0x2f, 0x96, 0xb5, 0xcf, 0x5e, 0x2c, 0x6b, 0xff, 0x7e, + 0xb1, 0xac, 0xfd, 0xf2, 0xe5, 0xf2, 0xcc, 0x67, 0x2f, 0x97, 0x67, 0xfe, 0xf9, 0x72, 0x79, 0xe6, + 0x93, 0x2f, 0xf7, 0xdd, 0x70, 0x7f, 0xb4, 0x77, 0xd3, 0xa6, 0xc3, 0x5b, 0xbe, 0xeb, 0xf5, 0x6d, + 0xcb, 0xbf, 0x15, 0xba, 0xb6, 0x63, 0xdf, 0x4a, 0x59, 0x75, 0xaf, 0x2a, 0xfe, 0x3d, 0xf1, 0xf6, + 0x17, 0x01, 0x00, 0x00, 0xff, 0xff, 0x4d, 0x9a, 0x25, 0xc9, 0x5c, 0x21, 0x00, 0x00, } func (m *TableSpan) Marshal() (dAtA []byte, err error) { diff --git a/heartbeatpb/heartbeat.proto b/heartbeatpb/heartbeat.proto index 0e3fba2cd8..b3192402c2 100644 --- a/heartbeatpb/heartbeat.proto +++ b/heartbeatpb/heartbeat.proto @@ -29,6 +29,7 @@ message Watermark { enum Action { Write = 0; Pass = 1; + Skip = 2; } message DispatcherAction { @@ -376,4 +377,4 @@ message DispatcherSetChecksumUpdateRequest { int64 mode = 3; uint64 seq = 4; DispatcherSetChecksum checksum = 5; -} \ No newline at end of file +} diff --git a/maintainer/barrier.go b/maintainer/barrier.go index 841f273aa3..a3899734c4 100644 --- a/maintainer/barrier.go +++ b/maintainer/barrier.go @@ -22,7 +22,9 @@ import ( "github.com/pingcap/ticdc/maintainer/span" "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/messaging" + "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -43,6 +45,10 @@ type Barrier struct { operatorController *operator.Controller splitTableEnabled bool mode int64 + + syncPointSkipEnabled atomic.Bool + syncPointSkipCheckpointLag atomic.Int64 // nanoseconds + syncPointInterval atomic.Int64 // nanoseconds } // NewBarrier create a new barrier for the changefeed @@ -64,6 +70,44 @@ func NewBarrier(spanController *span.Controller, return &barrier } +func (b *Barrier) SetSyncPointSkipState(enabled bool, checkpointLag time.Duration, syncPointInterval time.Duration) { + b.syncPointSkipEnabled.Store(enabled) + b.syncPointSkipCheckpointLag.Store(int64(checkpointLag)) + b.syncPointInterval.Store(int64(syncPointInterval)) +} + +func (b *Barrier) maybeSkipSyncPointEvent(event *BarrierEvent) { + if event == nil || !event.isSyncPoint { + return + } + if event.selected.Load() || event.skipSyncPoint { + return + } + if !b.syncPointSkipEnabled.Load() { + return + } + + event.skipSyncPoint = true + event.selected.Store(true) + event.writerDispatcherAdvanced = true + // Ensure the next resend can deliver the skip actions immediately. + event.lastResendTime = time.Now().Add(-20 * time.Second) + + checkpointLag := time.Duration(b.syncPointSkipCheckpointLag.Load()) + syncPointInterval := time.Duration(b.syncPointInterval.Load()) + threshold := 2 * syncPointInterval + metrics.MaintainerSkipSyncPointCount.WithLabelValues(event.cfID.Keyspace(), event.cfID.Name()).Inc() + log.Info("skip syncpoint", + zap.String("changefeed", event.cfID.Name()), + zap.String("keyspace", event.cfID.Keyspace()), + zap.Uint64("commitTs", event.commitTs), + zap.Duration("checkpointLag", checkpointLag), + zap.Duration("syncPointInterval", syncPointInterval), + zap.Duration("threshold", threshold), + zap.Int64("mode", b.mode), + ) +} + // HandleStatus handle the block status from dispatcher manager func (b *Barrier) HandleStatus(from node.ID, request *heartbeatpb.BlockStatusRequest, @@ -218,6 +262,7 @@ func (b *Barrier) Resend() []*messaging.TargetMessage { eventList := make([]*BarrierEvent, 0) b.blockedEvents.Range(func(key eventKey, barrierEvent *BarrierEvent) bool { // todo: we can limit the number of messages to send in one round here + b.maybeSkipSyncPointEvent(barrierEvent) msgs = append(msgs, barrierEvent.resend(b.mode)...) eventList = append(eventList, barrierEvent) @@ -342,6 +387,20 @@ func (b *Barrier) handleBlockState(changefeedID common.ChangeFeedID, zap.Uint64("commitTs", blockState.BlockTs)) event.tableTriggerDispatcherRelated = true } + + // Skip syncpoint alignment when maintainer observes large checkpoint lag. + // It marks the syncpoint barrier as skipped and relies on the resend loop + // to broadcast Action_Skip to all dispatcher managers. + if blockState.IsSyncPoint { + b.maybeSkipSyncPointEvent(event) + if event.skipSyncPoint { + event.markDispatcherEventDone(dispatcherID) + // No need to return a per-dispatcher action here: resend will broadcast + // the skip action to all nodes. + return event, nil, "", true + } + } + if event.selected.Load() { // the event already in the selected state, ignore the block event just sent ack log.Debug("the block event already selected, ignore the block event", @@ -369,13 +428,17 @@ func (b *Barrier) handleBlockState(changefeedID common.ChangeFeedID, blockState.BlockTables != nil && (blockState.BlockTables.InfluenceType != heartbeatpb.InfluenceType_Normal) { if pending := b.pendingEvents.Len(); pending > 0 { - log.Debug("discard db/all block event from ddl dispatcher due to pending schedule events, wait next resend", - zap.String("changefeed", changefeedID.Name()), - zap.String("dispatcher", dispatcherID.String()), - zap.Uint64("commitTs", blockState.BlockTs), - zap.Bool("isSyncPoint", blockState.IsSyncPoint), - zap.Int("pendingScheduleEvents", pending)) - return event, nil, "", false + // When syncpoint skip is enabled, we will skip this syncpoint anyway and do + // not need a precise task snapshot to build the range checker. + if !blockState.IsSyncPoint || !b.syncPointSkipEnabled.Load() { + log.Debug("discard db/all block event from ddl dispatcher due to pending schedule events, wait next resend", + zap.String("changefeed", changefeedID.Name()), + zap.String("dispatcher", dispatcherID.String()), + zap.Uint64("commitTs", blockState.BlockTs), + zap.Bool("isSyncPoint", blockState.IsSyncPoint), + zap.Int("pendingScheduleEvents", pending)) + return event, nil, "", false + } } } diff --git a/maintainer/barrier_event.go b/maintainer/barrier_event.go index e5bf55464a..e62f5ae97a 100644 --- a/maintainer/barrier_event.go +++ b/maintainer/barrier_event.go @@ -52,6 +52,7 @@ type BarrierEvent struct { newTables []*heartbeatpb.Table schemaIDChange []*heartbeatpb.SchemaIDChange isSyncPoint bool + skipSyncPoint bool needSchedule bool // if the split table is enable for this changefeed, if not we can use tableID to check coverage dynamicSplitEnabled bool @@ -673,6 +674,10 @@ func (be *BarrierEvent) newWriterActionMessage(capture node.ID, mode int64) *mes } func (be *BarrierEvent) newPassActionMessage(capture node.ID, mode int64) *messaging.TargetMessage { + action := heartbeatpb.Action_Pass + if be.skipSyncPoint { + action = heartbeatpb.Action_Skip + } influenced := &heartbeatpb.InfluencedDispatchers{ InfluenceType: be.blockedDispatchers.InfluenceType, SchemaID: be.blockedDispatchers.SchemaID, @@ -688,7 +693,7 @@ func (be *BarrierEvent) newPassActionMessage(capture node.ID, mode int64) *messa ChangefeedID: be.cfID.ToPB(), DispatcherStatuses: []*heartbeatpb.DispatcherStatus{ { - Action: be.action(heartbeatpb.Action_Pass), + Action: be.action(action), InfluencedDispatchers: influenced, }, }, diff --git a/maintainer/barrier_test.go b/maintainer/barrier_test.go index bac9fcf538..b0b6edff26 100644 --- a/maintainer/barrier_test.go +++ b/maintainer/barrier_test.go @@ -826,6 +826,146 @@ func TestSyncPointBlock(t *testing.T) { require.Len(t, barrier.blockedEvents.m, 0) } +func TestSyncPointSkip(t *testing.T) { + testutil.SetUpTestServices() + nm := appcontext.GetService[*watcher.NodeManager](watcher.NodeManagerName) + nmap := nm.GetAliveNodes() + for key := range nmap { + delete(nmap, key) + } + nmap["node1"] = &node.Info{ID: "node1"} + nmap["node2"] = &node.Info{ID: "node2"} + + tableTriggerEventDispatcherID := common.NewDispatcherID() + cfID := common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceName) + ddlSpan := replica.NewWorkingSpanReplication(cfID, tableTriggerEventDispatcherID, + common.DDLSpanSchemaID, + common.KeyspaceDDLSpan(common.DefaultKeyspaceID), &heartbeatpb.TableSpanStatus{ + ID: tableTriggerEventDispatcherID.ToPB(), + ComponentStatus: heartbeatpb.ComponentState_Working, + CheckpointTs: 1, + }, "node1", false) + spanController := span.NewController(cfID, ddlSpan, nil, nil, nil, common.DefaultKeyspaceID, common.DefaultMode) + operatorController := operator.NewOperatorController(cfID, spanController, 1000, common.DefaultMode) + spanController.AddNewTable(commonEvent.Table{SchemaID: 1, TableID: 1}, 1) + spanController.AddNewTable(commonEvent.Table{SchemaID: 1, TableID: 2}, 1) + spanController.AddNewTable(commonEvent.Table{SchemaID: 2, TableID: 3}, 1) + + var dispatcherIDs []*heartbeatpb.DispatcherID + absents := spanController.GetAbsentForTest(10000) + for _, stm := range absents { + dispatcherIDs = append(dispatcherIDs, stm.ID.ToPB()) + spanController.BindSpanToNode("", "node1", stm) + spanController.MarkSpanReplicating(stm) + } + + selectDispatcherID := common.NewDispatcherIDFromPB(dispatcherIDs[2]) + selectedRep := spanController.GetTaskByID(selectDispatcherID) + spanController.BindSpanToNode("node1", "node2", selectedRep) + spanController.MarkSpanReplicating(selectedRep) + + barrier := NewBarrier(spanController, operatorController, true, nil, common.DefaultMode) + msgs := barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ + { + ID: spanController.GetDDLDispatcherID().ToPB(), + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, + IsSyncPoint: true, + }, + }, + { + ID: dispatcherIDs[0], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, + IsSyncPoint: true, + }, + }, + { + ID: dispatcherIDs[1], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, + IsSyncPoint: true, + }, + }, + }, + }) + require.NotNil(t, msgs) + require.NotEmpty(t, msgs) + + barrier.SetSyncPointSkipState(true, 5*time.Minute, 2*time.Minute) + resendMsgs := barrier.Resend() + require.Len(t, resendMsgs, 2) + for _, msg := range resendMsgs { + resp := msg.Message[0].(*heartbeatpb.HeartBeatResponse) + require.Len(t, resp.DispatcherStatuses, 1) + require.Equal(t, heartbeatpb.Action_Skip, resp.DispatcherStatuses[0].Action.Action) + require.Equal(t, uint64(10), resp.DispatcherStatuses[0].Action.CommitTs) + require.True(t, resp.DispatcherStatuses[0].Action.IsSyncPoint) + } + + _ = barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ + { + ID: spanController.GetDDLDispatcherID().ToPB(), + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + Stage: heartbeatpb.BlockStage_DONE, + IsSyncPoint: true, + }, + }, + { + ID: dispatcherIDs[0], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + Stage: heartbeatpb.BlockStage_DONE, + IsSyncPoint: true, + }, + }, + { + ID: dispatcherIDs[1], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + Stage: heartbeatpb.BlockStage_DONE, + IsSyncPoint: true, + }, + }, + }, + }) + _ = barrier.HandleStatus("node2", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ + { + ID: dispatcherIDs[2], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + Stage: heartbeatpb.BlockStage_DONE, + IsSyncPoint: true, + }, + }, + }, + }) + require.Len(t, barrier.blockedEvents.m, 0) +} + func TestNonBlocked(t *testing.T) { testutil.SetUpTestServices() tableTriggerEventDispatcherID := common.NewDispatcherID() diff --git a/maintainer/maintainer.go b/maintainer/maintainer.go index 5ed31da75f..c8f285021b 100644 --- a/maintainer/maintainer.go +++ b/maintainer/maintainer.go @@ -738,6 +738,43 @@ func (m *Maintainer) updateMetrics() { m.resolvedTsLagGauge.Set(lag) } +func (m *Maintainer) calculateSyncPointSkipState() (enabled bool, checkpointLag time.Duration, syncPointInterval time.Duration) { + if m.info == nil || m.info.Config == nil { + return false, 0, 0 + } + if !util.GetOrZero(m.info.Config.EnableSyncPoint) { + return false, 0, util.GetOrZero(m.info.Config.SyncPointInterval) + } + syncPointInterval = util.GetOrZero(m.info.Config.SyncPointInterval) + if syncPointInterval <= 0 { + return false, 0, syncPointInterval + } + + watermark := m.getWatermark() + pdPhysicalTime := oracle.GetPhysical(m.pdClock.CurrentTime()) + phyCkpTs := oracle.ExtractPhysical(watermark.CheckpointTs) + lagMs := pdPhysicalTime - phyCkpTs + if lagMs < 0 { + lagMs = 0 + } + checkpointLag = time.Duration(lagMs) * time.Millisecond + enabled = checkpointLag > 2*syncPointInterval + return enabled, checkpointLag, syncPointInterval +} + +func (m *Maintainer) refreshBarrierSyncPointSkipState() { + if m.controller == nil { + return + } + enabled, checkpointLag, syncPointInterval := m.calculateSyncPointSkipState() + if m.controller.barrier != nil { + m.controller.barrier.SetSyncPointSkipState(enabled, checkpointLag, syncPointInterval) + } + if m.controller.redoBarrier != nil { + m.controller.redoBarrier.SetSyncPointSkipState(enabled, checkpointLag, syncPointInterval) + } +} + // send message to other components func (m *Maintainer) sendMessages(msgs []*messaging.TargetMessage) { for _, msg := range msgs { @@ -820,6 +857,8 @@ func (m *Maintainer) onBlockStateRequest(msg *messaging.TargetMessage) { } req := msg.Message[0].(*heartbeatpb.BlockStatusRequest) + m.refreshBarrierSyncPointSkipState() + var ackMsg []*messaging.TargetMessage if common.IsDefaultMode(req.Mode) { ackMsg = m.controller.barrier.HandleStatus(msg.From, req) @@ -952,6 +991,7 @@ func (m *Maintainer) handleResendMessage() { m.trySendMaintainerCloseRequestToAllNode() return } + m.refreshBarrierSyncPointSkipState() // resend bootstrap message m.sendMessages(m.bootstrapper.ResendBootstrapMessage()) m.sendPostBootstrapRequest() diff --git a/pkg/metrics/maintainer.go b/pkg/metrics/maintainer.go index b77706e4f4..a8dc6d7fc6 100644 --- a/pkg/metrics/maintainer.go +++ b/pkg/metrics/maintainer.go @@ -16,6 +16,14 @@ package metrics import "github.com/prometheus/client_golang/prometheus" var ( + MaintainerSkipSyncPointCount = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "maintainer", + Name: "skip_syncpoint_count", + Help: "The number of skipped syncpoint events", + }, []string{getKeyspaceLabel(), "changefeed"}) + MaintainerHandleEventDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "ticdc", @@ -60,6 +68,7 @@ var ( ) func initMaintainerMetrics(registry *prometheus.Registry) { + registry.MustRegister(MaintainerSkipSyncPointCount) registry.MustRegister(MaintainerHandleEventDuration) registry.MustRegister(MaintainerEventChLenGauge) registry.MustRegister(OperatorCount) From dcb47a89a91715b5f3898a4614dad2910e228f6f Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 3 Feb 2026 19:14:45 +0800 Subject: [PATCH 31/34] fix bug Signed-off-by: dongmen <414110582@qq.com> --- maintainer/barrier.go | 146 +++++++++++++++++++++++++----------- maintainer/barrier_event.go | 7 +- maintainer/barrier_test.go | 58 +++++++------- 3 files changed, 135 insertions(+), 76 deletions(-) diff --git a/maintainer/barrier.go b/maintainer/barrier.go index a3899734c4..7a1d5e8b80 100644 --- a/maintainer/barrier.go +++ b/maintainer/barrier.go @@ -49,6 +49,7 @@ type Barrier struct { syncPointSkipEnabled atomic.Bool syncPointSkipCheckpointLag atomic.Int64 // nanoseconds syncPointInterval atomic.Int64 // nanoseconds + maxSkippedSyncPointTs atomic.Uint64 } // NewBarrier create a new barrier for the changefeed @@ -71,36 +72,52 @@ func NewBarrier(spanController *span.Controller, } func (b *Barrier) SetSyncPointSkipState(enabled bool, checkpointLag time.Duration, syncPointInterval time.Duration) { - b.syncPointSkipEnabled.Store(enabled) + wasEnabled := b.syncPointSkipEnabled.Swap(enabled) b.syncPointSkipCheckpointLag.Store(int64(checkpointLag)) b.syncPointInterval.Store(int64(syncPointInterval)) -} -func (b *Barrier) maybeSkipSyncPointEvent(event *BarrierEvent) { - if event == nil || !event.isSyncPoint { - return + // When enabling skip, drop any pending syncpoint events in barrier state. + // Dispatchers will resend their WAITING states and receive Action_Skip directly. + if enabled && !wasEnabled { + var keys []eventKey + b.blockedEvents.Range(func(key eventKey, barrierEvent *BarrierEvent) bool { + if barrierEvent != nil && barrierEvent.isSyncPoint { + keys = append(keys, key) + } + return true + }) + for _, key := range keys { + b.blockedEvents.Delete(key) + } } - if event.selected.Load() || event.skipSyncPoint { +} + +func (b *Barrier) isSkippedSyncPointCommitTs(commitTs uint64) bool { + return commitTs > 0 && commitTs <= b.maxSkippedSyncPointTs.Load() +} + +func (b *Barrier) recordSkippedSyncPoint(cfID common.ChangeFeedID, commitTs uint64) { + if commitTs == 0 { return } - if !b.syncPointSkipEnabled.Load() { - return + for { + old := b.maxSkippedSyncPointTs.Load() + if commitTs <= old { + return + } + if b.maxSkippedSyncPointTs.CompareAndSwap(old, commitTs) { + break + } } - event.skipSyncPoint = true - event.selected.Store(true) - event.writerDispatcherAdvanced = true - // Ensure the next resend can deliver the skip actions immediately. - event.lastResendTime = time.Now().Add(-20 * time.Second) - checkpointLag := time.Duration(b.syncPointSkipCheckpointLag.Load()) syncPointInterval := time.Duration(b.syncPointInterval.Load()) threshold := 2 * syncPointInterval - metrics.MaintainerSkipSyncPointCount.WithLabelValues(event.cfID.Keyspace(), event.cfID.Name()).Inc() + metrics.MaintainerSkipSyncPointCount.WithLabelValues(cfID.Keyspace(), cfID.Name()).Inc() log.Info("skip syncpoint", - zap.String("changefeed", event.cfID.Name()), - zap.String("keyspace", event.cfID.Keyspace()), - zap.Uint64("commitTs", event.commitTs), + zap.String("changefeed", cfID.Name()), + zap.String("keyspace", cfID.Keyspace()), + zap.Uint64("commitTs", commitTs), zap.Duration("checkpointLag", checkpointLag), zap.Duration("syncPointInterval", syncPointInterval), zap.Duration("threshold", threshold), @@ -108,6 +125,21 @@ func (b *Barrier) maybeSkipSyncPointEvent(event *BarrierEvent) { ) } +func (b *Barrier) buildSkipSyncPointStatus(dispatcherID *heartbeatpb.DispatcherID, commitTs uint64) *heartbeatpb.DispatcherStatus { + return &heartbeatpb.DispatcherStatus{ + InfluencedDispatchers: &heartbeatpb.InfluencedDispatchers{ + InfluenceType: heartbeatpb.InfluenceType_Normal, + DispatcherIDs: []*heartbeatpb.DispatcherID{dispatcherID}, + }, + Action: &heartbeatpb.DispatcherAction{ + Action: heartbeatpb.Action_Skip, + CommitTs: commitTs, + IsSyncPoint: true, + }, + Ack: ackEvent(commitTs, true), + } +} + // HandleStatus handle the block status from dispatcher manager func (b *Barrier) HandleStatus(from node.ID, request *heartbeatpb.BlockStatusRequest, @@ -119,10 +151,14 @@ func (b *Barrier) HandleStatus(from node.ID, actions := map[node.ID][]*heartbeatpb.DispatcherStatus{} var dispatcherStatus []*heartbeatpb.DispatcherStatus for _, status := range request.BlockStatuses { + if status.State == nil { + continue + } + // only receive block status from the replicating dispatcher dispatcherID := common.NewDispatcherIDFromPB(status.ID) + task := b.spanController.GetTaskByID(dispatcherID) if dispatcherID != b.spanController.GetDDLDispatcherID() { - task := b.spanController.GetTaskByID(dispatcherID) if task == nil { log.Info("Get block status from unexisted dispatcher, ignore it", zap.String("changefeed", request.ChangefeedID.GetName()), zap.String("dispatcher", dispatcherID.String()), zap.Uint64("commitTs", status.State.BlockTs)) continue @@ -134,6 +170,46 @@ func (b *Barrier) HandleStatus(from node.ID, } } + if status.State.IsSyncPoint && + status.State.Stage == heartbeatpb.BlockStage_DONE && + (b.syncPointSkipEnabled.Load() || b.isSkippedSyncPointCommitTs(status.State.BlockTs)) { + // Ignore DONE statuses for skipped syncpoints. They are not tracked in blockedEvents. + if task != nil { + task.UpdateStatus(&heartbeatpb.TableSpanStatus{ + ID: status.ID, + CheckpointTs: status.State.BlockTs - 1, + ComponentStatus: heartbeatpb.ComponentState_Working, + Mode: status.Mode, + }) + task.UpdateBlockState(*status.State) + } + continue + } + + shouldSkipSyncPoint := status.State.IsSyncPoint && + status.State.IsBlocked && + status.State.BlockTs > 0 && + (status.State.Stage == heartbeatpb.BlockStage_NONE || + status.State.Stage == heartbeatpb.BlockStage_WAITING || + status.State.Stage == heartbeatpb.BlockStage_WRITING) && + (b.syncPointSkipEnabled.Load() || b.isSkippedSyncPointCommitTs(status.State.BlockTs)) + if shouldSkipSyncPoint { + // Track progress in span controller (same as handleOneStatus would do). + if task != nil { + task.UpdateStatus(&heartbeatpb.TableSpanStatus{ + ID: status.ID, + CheckpointTs: status.State.BlockTs - 1, + ComponentStatus: heartbeatpb.ComponentState_Working, + Mode: status.Mode, + }) + task.UpdateBlockState(*status.State) + } + + actions[from] = append(actions[from], b.buildSkipSyncPointStatus(status.ID, status.State.BlockTs)) + b.recordSkippedSyncPoint(common.NewChangefeedIDFromPB(request.ChangefeedID), status.State.BlockTs) + continue + } + // deal with block status, and check whether need to return action. // we need to deal with the block status in order, otherwise scheduler may have problem // e.g. TODO(truncate + create table) @@ -262,7 +338,6 @@ func (b *Barrier) Resend() []*messaging.TargetMessage { eventList := make([]*BarrierEvent, 0) b.blockedEvents.Range(func(key eventKey, barrierEvent *BarrierEvent) bool { // todo: we can limit the number of messages to send in one round here - b.maybeSkipSyncPointEvent(barrierEvent) msgs = append(msgs, barrierEvent.resend(b.mode)...) eventList = append(eventList, barrierEvent) @@ -388,19 +463,6 @@ func (b *Barrier) handleBlockState(changefeedID common.ChangeFeedID, event.tableTriggerDispatcherRelated = true } - // Skip syncpoint alignment when maintainer observes large checkpoint lag. - // It marks the syncpoint barrier as skipped and relies on the resend loop - // to broadcast Action_Skip to all dispatcher managers. - if blockState.IsSyncPoint { - b.maybeSkipSyncPointEvent(event) - if event.skipSyncPoint { - event.markDispatcherEventDone(dispatcherID) - // No need to return a per-dispatcher action here: resend will broadcast - // the skip action to all nodes. - return event, nil, "", true - } - } - if event.selected.Load() { // the event already in the selected state, ignore the block event just sent ack log.Debug("the block event already selected, ignore the block event", @@ -428,17 +490,13 @@ func (b *Barrier) handleBlockState(changefeedID common.ChangeFeedID, blockState.BlockTables != nil && (blockState.BlockTables.InfluenceType != heartbeatpb.InfluenceType_Normal) { if pending := b.pendingEvents.Len(); pending > 0 { - // When syncpoint skip is enabled, we will skip this syncpoint anyway and do - // not need a precise task snapshot to build the range checker. - if !blockState.IsSyncPoint || !b.syncPointSkipEnabled.Load() { - log.Debug("discard db/all block event from ddl dispatcher due to pending schedule events, wait next resend", - zap.String("changefeed", changefeedID.Name()), - zap.String("dispatcher", dispatcherID.String()), - zap.Uint64("commitTs", blockState.BlockTs), - zap.Bool("isSyncPoint", blockState.IsSyncPoint), - zap.Int("pendingScheduleEvents", pending)) - return event, nil, "", false - } + log.Debug("discard db/all block event from ddl dispatcher due to pending schedule events, wait next resend", + zap.String("changefeed", changefeedID.Name()), + zap.String("dispatcher", dispatcherID.String()), + zap.Uint64("commitTs", blockState.BlockTs), + zap.Bool("isSyncPoint", blockState.IsSyncPoint), + zap.Int("pendingScheduleEvents", pending)) + return event, nil, "", false } } diff --git a/maintainer/barrier_event.go b/maintainer/barrier_event.go index e62f5ae97a..e5bf55464a 100644 --- a/maintainer/barrier_event.go +++ b/maintainer/barrier_event.go @@ -52,7 +52,6 @@ type BarrierEvent struct { newTables []*heartbeatpb.Table schemaIDChange []*heartbeatpb.SchemaIDChange isSyncPoint bool - skipSyncPoint bool needSchedule bool // if the split table is enable for this changefeed, if not we can use tableID to check coverage dynamicSplitEnabled bool @@ -674,10 +673,6 @@ func (be *BarrierEvent) newWriterActionMessage(capture node.ID, mode int64) *mes } func (be *BarrierEvent) newPassActionMessage(capture node.ID, mode int64) *messaging.TargetMessage { - action := heartbeatpb.Action_Pass - if be.skipSyncPoint { - action = heartbeatpb.Action_Skip - } influenced := &heartbeatpb.InfluencedDispatchers{ InfluenceType: be.blockedDispatchers.InfluenceType, SchemaID: be.blockedDispatchers.SchemaID, @@ -693,7 +688,7 @@ func (be *BarrierEvent) newPassActionMessage(capture node.ID, mode int64) *messa ChangefeedID: be.cfID.ToPB(), DispatcherStatuses: []*heartbeatpb.DispatcherStatus{ { - Action: be.action(action), + Action: be.action(heartbeatpb.Action_Pass), InfluencedDispatchers: influenced, }, }, diff --git a/maintainer/barrier_test.go b/maintainer/barrier_test.go index b0b6edff26..023227447b 100644 --- a/maintainer/barrier_test.go +++ b/maintainer/barrier_test.go @@ -865,6 +865,8 @@ func TestSyncPointSkip(t *testing.T) { spanController.MarkSpanReplicating(selectedRep) barrier := NewBarrier(spanController, operatorController, true, nil, common.DefaultMode) + barrier.SetSyncPointSkipState(true, 5*time.Minute, 2*time.Minute) + msgs := barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ ChangefeedID: cfID.ToPB(), BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ @@ -904,42 +906,46 @@ func TestSyncPointSkip(t *testing.T) { }, }) require.NotNil(t, msgs) - require.NotEmpty(t, msgs) - - barrier.SetSyncPointSkipState(true, 5*time.Minute, 2*time.Minute) - resendMsgs := barrier.Resend() - require.Len(t, resendMsgs, 2) - for _, msg := range resendMsgs { - resp := msg.Message[0].(*heartbeatpb.HeartBeatResponse) - require.Len(t, resp.DispatcherStatuses, 1) - require.Equal(t, heartbeatpb.Action_Skip, resp.DispatcherStatuses[0].Action.Action) - require.Equal(t, uint64(10), resp.DispatcherStatuses[0].Action.CommitTs) - require.True(t, resp.DispatcherStatuses[0].Action.IsSyncPoint) + require.Len(t, msgs, 1) + resp := msgs[0].Message[0].(*heartbeatpb.HeartBeatResponse) + require.Len(t, resp.DispatcherStatuses, 3) + for _, st := range resp.DispatcherStatuses { + require.NotNil(t, st.Action) + require.Equal(t, heartbeatpb.Action_Skip, st.Action.Action) + require.Equal(t, uint64(10), st.Action.CommitTs) + require.True(t, st.Action.IsSyncPoint) + require.NotNil(t, st.Ack) + require.Equal(t, uint64(10), st.Ack.CommitTs) + require.True(t, st.Ack.IsSyncPoint) } - _ = barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ + msgs = barrier.HandleStatus("node2", &heartbeatpb.BlockStatusRequest{ ChangefeedID: cfID.ToPB(), BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ { - ID: spanController.GetDDLDispatcherID().ToPB(), - State: &heartbeatpb.State{ - IsBlocked: true, - BlockTs: 10, - Stage: heartbeatpb.BlockStage_DONE, - IsSyncPoint: true, - }, - }, - { - ID: dispatcherIDs[0], + ID: dispatcherIDs[2], State: &heartbeatpb.State{ - IsBlocked: true, - BlockTs: 10, - Stage: heartbeatpb.BlockStage_DONE, + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, IsSyncPoint: true, }, }, + }, + }) + require.Len(t, msgs, 1) + resp = msgs[0].Message[0].(*heartbeatpb.HeartBeatResponse) + require.Len(t, resp.DispatcherStatuses, 1) + require.NotNil(t, resp.DispatcherStatuses[0].Action) + require.Equal(t, heartbeatpb.Action_Skip, resp.DispatcherStatuses[0].Action.Action) + + _ = barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ { - ID: dispatcherIDs[1], + ID: spanController.GetDDLDispatcherID().ToPB(), State: &heartbeatpb.State{ IsBlocked: true, BlockTs: 10, From d41d18040e5e0b25ed2c3dad6ff6fff2b55ce1ad Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 3 Feb 2026 20:30:21 +0800 Subject: [PATCH 32/34] add debug log Signed-off-by: dongmen <414110582@qq.com> --- pkg/sink/mysql/mysql_writer_for_syncpoint.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/sink/mysql/mysql_writer_for_syncpoint.go b/pkg/sink/mysql/mysql_writer_for_syncpoint.go index a931dd4e15..4e975aa389 100644 --- a/pkg/sink/mysql/mysql_writer_for_syncpoint.go +++ b/pkg/sink/mysql/mysql_writer_for_syncpoint.go @@ -45,6 +45,8 @@ func (w *Writer) createSyncTable() error { } func (w *Writer) SendSyncPointEvent(event *commonEvent.SyncPointEvent) error { + log.Info("fizz send syncpoint event", zap.Stringer("changefeedID", w.ChangefeedID), zap.Uint64("commitTs", event.GetCommitTs())) + tx, err := w.db.BeginTx(w.ctx, nil) if err != nil { return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;")) From aeeda690f38ebae420c24debb048c48ed83d9c77 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 4 Feb 2026 09:36:06 +0800 Subject: [PATCH 33/34] fix Signed-off-by: dongmen <414110582@qq.com> --- .../dispatcher/basic_dispatcher.go | 8 ++ .../dispatcher/event_dispatcher_test.go | 64 +++++++++++ downstreamadapter/dispatcher/helper.go | 7 +- maintainer/barrier.go | 74 +++++++++--- maintainer/barrier_test.go | 105 ++++++++++++++++++ maintainer/span/span_controller.go | 4 + 6 files changed, 245 insertions(+), 17 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 303d096d2b..7b38661e46 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -655,6 +655,14 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D zap.Stringer("dispatcher", d.id)) return false } + if ok && action.CommitTs == ts { + log.Debug("action does not match pending event, ignore it", + zap.Uint64("pendingEventCommitTs", ts), + zap.Uint64("actionCommitTs", action.CommitTs), + zap.Bool("actionIsSyncPoint", action.IsSyncPoint), + zap.Stringer("dispatcher", d.id)) + return false + } } // Step3: whether the outdate message or not, we need to return message show we have finished the event. diff --git a/downstreamadapter/dispatcher/event_dispatcher_test.go b/downstreamadapter/dispatcher/event_dispatcher_test.go index f6e93e0665..854f9d2c85 100644 --- a/downstreamadapter/dispatcher/event_dispatcher_test.go +++ b/downstreamadapter/dispatcher/event_dispatcher_test.go @@ -402,6 +402,70 @@ func TestDispatcherHandleEvents(t *testing.T) { require.Equal(t, uint64(7), checkpointTs) } +func TestDispatcherIgnoreActionWithSameCommitTsDifferentType(t *testing.T) { + sink := sink.NewMockSink(common.MysqlSinkType) + tableSpan, err := getCompleteTableSpan(getTestingKeyspaceID()) + require.NoError(t, err) + dispatcher := newDispatcherForTest(sink, tableSpan) + + nodeID := node.NewID() + + ddlEvent := &commonEvent.DDLEvent{ + FinishedTs: 10, + BlockedTables: &commonEvent.InfluencedTables{ + InfluenceType: commonEvent.InfluenceTypeNormal, + TableIDs: []int64{0, 1}, + }, + } + block := dispatcher.HandleEvents([]DispatcherEvent{NewDispatcherEvent(&nodeID, ddlEvent)}, callback) + require.True(t, block) + + // Pending DDL event should be in WAITING stage. + pending, stage := dispatcher.blockEventStatus.getEventAndStage() + require.NotNil(t, pending) + require.Equal(t, heartbeatpb.BlockStage_WAITING, stage) + + // ACK the DDL so the resend task is cancelled. + dispatcher.HandleDispatcherStatus(&heartbeatpb.DispatcherStatus{ + Ack: &heartbeatpb.ACK{ + CommitTs: ddlEvent.FinishedTs, + IsSyncPoint: false, + }, + }) + require.Equal(t, 0, dispatcher.resendTaskMap.Len()) + + // Drain any previously sent block status messages. +drain: + for { + select { + case <-dispatcher.sharedInfo.blockStatusesChan: + default: + break drain + } + } + + // Send an action for a syncpoint with the same commitTs. It should be ignored. + dispatcher.HandleDispatcherStatus(&heartbeatpb.DispatcherStatus{ + Action: &heartbeatpb.DispatcherAction{ + Action: heartbeatpb.Action_Skip, + CommitTs: ddlEvent.FinishedTs, + IsSyncPoint: true, + }, + }) + + // Still waiting on the DDL. + pending, stage = dispatcher.blockEventStatus.getEventAndStage() + require.NotNil(t, pending) + require.Equal(t, heartbeatpb.BlockStage_WAITING, stage) + + // Should not emit a DONE block status for the mismatched action. + select { + case msg := <-dispatcher.sharedInfo.blockStatusesChan: + require.FailNow(t, "unexpected block status emitted", "msg=%v", msg) + default: + } +} + // test uncompelete table span can correctly handle the ddl events func TestUncompeleteTableSpanDispatcherHandleEvents(t *testing.T) { count.Swap(0) diff --git a/downstreamadapter/dispatcher/helper.go b/downstreamadapter/dispatcher/helper.go index 29373edc14..5871adc654 100644 --- a/downstreamadapter/dispatcher/helper.go +++ b/downstreamadapter/dispatcher/helper.go @@ -139,7 +139,12 @@ func (b *BlockEventStatus) actionMatchs(action *heartbeatpb.DispatcherAction) bo return false } - return b.blockCommitTs == action.CommitTs + if b.blockCommitTs != action.CommitTs { + return false + } + + isSyncPointPending := b.blockPendingEvent.GetType() == commonEvent.TypeSyncPointEvent + return isSyncPointPending == action.IsSyncPoint } func (b *BlockEventStatus) getEventCommitTs() (uint64, bool) { diff --git a/maintainer/barrier.go b/maintainer/barrier.go index 7a1d5e8b80..98fbde5043 100644 --- a/maintainer/barrier.go +++ b/maintainer/barrier.go @@ -72,24 +72,9 @@ func NewBarrier(spanController *span.Controller, } func (b *Barrier) SetSyncPointSkipState(enabled bool, checkpointLag time.Duration, syncPointInterval time.Duration) { - wasEnabled := b.syncPointSkipEnabled.Swap(enabled) + b.syncPointSkipEnabled.Store(enabled) b.syncPointSkipCheckpointLag.Store(int64(checkpointLag)) b.syncPointInterval.Store(int64(syncPointInterval)) - - // When enabling skip, drop any pending syncpoint events in barrier state. - // Dispatchers will resend their WAITING states and receive Action_Skip directly. - if enabled && !wasEnabled { - var keys []eventKey - b.blockedEvents.Range(func(key eventKey, barrierEvent *BarrierEvent) bool { - if barrierEvent != nil && barrierEvent.isSyncPoint { - keys = append(keys, key) - } - return true - }) - for _, key := range keys { - b.blockedEvents.Delete(key) - } - } } func (b *Barrier) isSkippedSyncPointCommitTs(commitTs uint64) bool { @@ -140,6 +125,53 @@ func (b *Barrier) buildSkipSyncPointStatus(dispatcherID *heartbeatpb.DispatcherI } } +func (b *Barrier) resendSkipSyncPointActions() []*messaging.TargetMessage { + if !b.syncPointSkipEnabled.Load() && b.maxSkippedSyncPointTs.Load() == 0 { + return nil + } + + cfID := b.spanController.GetChangefeedID() + var ( + msgs []*messaging.TargetMessage + statusMap = make(map[node.ID][]*heartbeatpb.DispatcherStatus) + ) + for _, task := range b.spanController.GetAllTasks() { + if task == nil { + continue + } + state := task.GetBlockState() + if state == nil || !state.IsBlocked || !state.IsSyncPoint || state.BlockTs == 0 { + continue + } + if state.Stage != heartbeatpb.BlockStage_WAITING { + continue + } + if !b.syncPointSkipEnabled.Load() && !b.isSkippedSyncPointCommitTs(state.BlockTs) { + continue + } + + nodeID := task.GetNodeID() + if nodeID == "" { + continue + } + + statusMap[nodeID] = append(statusMap[nodeID], b.buildSkipSyncPointStatus(task.ID.ToPB(), state.BlockTs)) + b.recordSkippedSyncPoint(cfID, state.BlockTs) + } + + for nodeID, statuses := range statusMap { + msg := messaging.NewSingleTargetMessage(nodeID, + messaging.HeartbeatCollectorTopic, + &heartbeatpb.HeartBeatResponse{ + ChangefeedID: cfID.ToPB(), + DispatcherStatuses: statuses, + Mode: b.mode, + }) + msgs = append(msgs, msg) + } + return msgs +} + // HandleStatus handle the block status from dispatcher manager func (b *Barrier) HandleStatus(from node.ID, request *heartbeatpb.BlockStatusRequest, @@ -335,14 +367,24 @@ func (b *Barrier) handleBootstrapResponse(bootstrapRespMap map[node.ID]*heartbea func (b *Barrier) Resend() []*messaging.TargetMessage { var msgs []*messaging.TargetMessage + msgs = append(msgs, b.resendSkipSyncPointActions()...) + eventList := make([]*BarrierEvent, 0) + var skipKeys []eventKey b.blockedEvents.Range(func(key eventKey, barrierEvent *BarrierEvent) bool { // todo: we can limit the number of messages to send in one round here + if barrierEvent != nil && barrierEvent.isSyncPoint && (b.syncPointSkipEnabled.Load() || b.isSkippedSyncPointCommitTs(barrierEvent.commitTs)) { + skipKeys = append(skipKeys, key) + return true + } msgs = append(msgs, barrierEvent.resend(b.mode)...) eventList = append(eventList, barrierEvent) return true }) + for _, key := range skipKeys { + b.blockedEvents.Delete(key) + } for _, event := range eventList { if event != nil { diff --git a/maintainer/barrier_test.go b/maintainer/barrier_test.go index 023227447b..853c4accb7 100644 --- a/maintainer/barrier_test.go +++ b/maintainer/barrier_test.go @@ -972,6 +972,111 @@ func TestSyncPointSkip(t *testing.T) { require.Len(t, barrier.blockedEvents.m, 0) } +func TestSyncPointSkipEnableUnblocksInflight(t *testing.T) { + testutil.SetUpTestServices() + nm := appcontext.GetService[*watcher.NodeManager](watcher.NodeManagerName) + nmap := nm.GetAliveNodes() + for key := range nmap { + delete(nmap, key) + } + nmap["node1"] = &node.Info{ID: "node1"} + nmap["node2"] = &node.Info{ID: "node2"} + + tableTriggerEventDispatcherID := common.NewDispatcherID() + cfID := common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceName) + ddlSpan := replica.NewWorkingSpanReplication(cfID, tableTriggerEventDispatcherID, + common.DDLSpanSchemaID, + common.KeyspaceDDLSpan(common.DefaultKeyspaceID), &heartbeatpb.TableSpanStatus{ + ID: tableTriggerEventDispatcherID.ToPB(), + ComponentStatus: heartbeatpb.ComponentState_Working, + CheckpointTs: 1, + }, "node1", false) + spanController := span.NewController(cfID, ddlSpan, nil, nil, nil, common.DefaultKeyspaceID, common.DefaultMode) + operatorController := operator.NewOperatorController(cfID, spanController, 1000, common.DefaultMode) + spanController.AddNewTable(commonEvent.Table{SchemaID: 1, TableID: 1}, 1) + spanController.AddNewTable(commonEvent.Table{SchemaID: 1, TableID: 2}, 1) + + var tableDispatcherIDs []*heartbeatpb.DispatcherID + absents := spanController.GetAbsentForTest(10000) + for _, stm := range absents { + tableDispatcherIDs = append(tableDispatcherIDs, stm.ID.ToPB()) + spanController.BindSpanToNode("", "node1", stm) + spanController.MarkSpanReplicating(stm) + } + + // Move one table dispatcher to another node, so skip actions should be sent to two nodes. + movedDispatcherID := common.NewDispatcherIDFromPB(tableDispatcherIDs[1]) + movedRep := spanController.GetTaskByID(movedDispatcherID) + spanController.BindSpanToNode("node1", "node2", movedRep) + spanController.MarkSpanReplicating(movedRep) + + barrier := NewBarrier(spanController, operatorController, true, nil, common.DefaultMode) + + // Report a syncpoint barrier from one table dispatcher first (rangeChecker is not created yet). + _ = barrier.HandleStatus("node2", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ + { + ID: tableDispatcherIDs[1], + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, + IsSyncPoint: true, + Stage: heartbeatpb.BlockStage_WAITING, + }, + }, + }, + }) + + // Report from ddl dispatcher, which creates the rangeChecker but still not fully covered. + _ = barrier.HandleStatus("node1", &heartbeatpb.BlockStatusRequest{ + ChangefeedID: cfID.ToPB(), + BlockStatuses: []*heartbeatpb.TableSpanBlockStatus{ + { + ID: spanController.GetDDLDispatcherID().ToPB(), + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + BlockTables: &heartbeatpb.InfluencedTables{ + InfluenceType: heartbeatpb.InfluenceType_All, + }, + IsSyncPoint: true, + Stage: heartbeatpb.BlockStage_WAITING, + }, + }, + }, + }) + + barrier.SetSyncPointSkipState(true, 5*time.Minute, 2*time.Minute) + + resendMsgs := barrier.Resend() + require.Len(t, resendMsgs, 2) + + expected := map[string]struct{}{ + tableDispatcherIDs[1].String(): {}, + spanController.GetDDLDispatcherID().ToPB().String(): {}, + } + for _, msg := range resendMsgs { + resp := msg.Message[0].(*heartbeatpb.HeartBeatResponse) + for _, st := range resp.DispatcherStatuses { + require.NotNil(t, st.Action) + require.Equal(t, heartbeatpb.Action_Skip, st.Action.Action) + require.Equal(t, uint64(10), st.Action.CommitTs) + require.True(t, st.Action.IsSyncPoint) + require.NotNil(t, st.Ack) + require.Equal(t, uint64(10), st.Ack.CommitTs) + require.True(t, st.Ack.IsSyncPoint) + require.Equal(t, heartbeatpb.InfluenceType_Normal, st.InfluencedDispatchers.InfluenceType) + require.Len(t, st.InfluencedDispatchers.DispatcherIDs, 1) + delete(expected, st.InfluencedDispatchers.DispatcherIDs[0].String()) + } + } + require.Len(t, expected, 0) +} + func TestNonBlocked(t *testing.T) { testutil.SetUpTestServices() tableTriggerEventDispatcherID := common.NewDispatcherID() diff --git a/maintainer/span/span_controller.go b/maintainer/span/span_controller.go index df95d76424..c0d020f128 100644 --- a/maintainer/span/span_controller.go +++ b/maintainer/span/span_controller.go @@ -251,6 +251,10 @@ func (c *Controller) GetAllTasks() []*replica.SpanReplication { return tasks } +func (c *Controller) GetChangefeedID() common.ChangeFeedID { + return c.changefeedID +} + // GetTaskSizeBySchemaID returns the size of the task by the schema id func (c *Controller) GetTaskSizeBySchemaID(schemaID int64) int { c.mu.RLock() From a24e5b31fa6c5bfb118dd021d9e1407748ccccce Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 4 Feb 2026 10:49:21 +0800 Subject: [PATCH 34/34] add log Signed-off-by: dongmen <414110582@qq.com> --- pkg/sink/mysql/mysql_writer.go | 1 + pkg/sink/mysql/mysql_writer_for_syncpoint.go | 2 -- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/sink/mysql/mysql_writer.go b/pkg/sink/mysql/mysql_writer.go index a7ecb632d6..b8f98c3749 100644 --- a/pkg/sink/mysql/mysql_writer.go +++ b/pkg/sink/mysql/mysql_writer.go @@ -143,6 +143,7 @@ func (w *Writer) FlushDDLEvent(event *commonEvent.DDLEvent) error { func (w *Writer) FlushSyncPointEvent(event *commonEvent.SyncPointEvent) error { if w.cfg.DryRun { + log.Info("dry-run mode, skip send syncpoint event", zap.Stringer("changefeedID", w.ChangefeedID), zap.Uint64("commitTs", event.GetCommitTs())) return nil } diff --git a/pkg/sink/mysql/mysql_writer_for_syncpoint.go b/pkg/sink/mysql/mysql_writer_for_syncpoint.go index 4e975aa389..a931dd4e15 100644 --- a/pkg/sink/mysql/mysql_writer_for_syncpoint.go +++ b/pkg/sink/mysql/mysql_writer_for_syncpoint.go @@ -45,8 +45,6 @@ func (w *Writer) createSyncTable() error { } func (w *Writer) SendSyncPointEvent(event *commonEvent.SyncPointEvent) error { - log.Info("fizz send syncpoint event", zap.Stringer("changefeedID", w.ChangefeedID), zap.Uint64("commitTs", event.GetCommitTs())) - tx, err := w.db.BeginTx(w.ctx, nil) if err != nil { return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;"))