From 1cab7d35b2857203db09fdab2e04e1186db238df Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 12 Aug 2024 14:14:57 +0800 Subject: [PATCH 01/29] add debug log for perf testing Signed-off-by: dongmen <414110582@qq.com> --- .../dispatcher/table_event_dispatcher.go | 11 +++++++---- go.mod | 1 - pkg/eventservice/event_broker.go | 13 +++++++++++-- pkg/eventservice/event_service.go | 8 ++++---- pkg/eventservice/event_service_test.go | 5 ++--- pkg/messaging/message.go | 2 ++ 6 files changed, 26 insertions(+), 14 deletions(-) diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 8400fba99..97c0c9d31 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -138,7 +138,7 @@ func NewTableEventDispatcher(tableSpan *common.TableSpan, sink sink.Sink, startT tableEventDispatcher.wg.Add(1) go tableEventDispatcher.DispatcherEvents(ctx) - log.Info("table event dispatcher created", zap.Any("DispatcherID", tableEventDispatcher.id)) + log.Info("table event dispatcher created", zap.String("DispatcherID", tableEventDispatcher.id), zap.Stringer("tableSpan", tableSpan)) return tableEventDispatcher } @@ -156,8 +156,8 @@ func (d *TableEventDispatcher) DispatcherEvents(ctx context.Context) { if event.IsDMLEvent() { sink.AddDMLEvent(tableSpan, event) } else { - // resolvedTs - d.resolvedTs.Set(event.ResolvedTs) + // TODO: consider the ddl event + continue } } } @@ -200,7 +200,10 @@ func (d *TableEventDispatcher) GetCheckpointTs() uint64 { } func (d *TableEventDispatcher) UpdateResolvedTs(ts uint64) { - d.GetEventChan() <- &common.TxnEvent{ResolvedTs: ts} + if d.tableSpan.TableID == uint64(217) { + log.Info("fizz update resolvedTs", zap.Uint64("tableID", d.tableSpan.TableID), zap.Uint64("resolvedTs", ts)) + } + d.resolvedTs.Set(ts) } func (d *TableEventDispatcher) GetId() string { diff --git a/go.mod b/go.mod index a3bb4aed4..a78b8a669 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,6 @@ require ( github.com/go-sql-driver/mysql v1.7.1 github.com/gogo/protobuf v1.3.2 github.com/google/btree v1.1.2 - github.com/google/martian v2.1.0+incompatible github.com/google/uuid v1.6.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 14f98869d..6ddd79ab1 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -146,6 +146,12 @@ func (c *eventBroker) runScanWorker(ctx context.Context) { case <-ctx.Done(): return case task := <-c.taskPool.popTask(chIndex): + if task.dispatcherStat.spanSubscription.span.TableID == uint64(217) { + log.Info("fizz on scan task", + zap.Any("tableID", task.dispatcherStat.spanSubscription.span.TableID), + zap.Any("watermark", task.dataRange.EndTs), zap.Any("eventCount", task.eventCount)) + } + needScan := task.checkAndAdjustScanTask() if !needScan { continue @@ -245,7 +251,6 @@ func (c *eventBroker) runSendMessageWorker(ctx context.Context) { case <-ctx.Done(): return case m := <-c.messageCh: - // Send the message to messageCenter. Retry if the send failed. for { select { @@ -261,7 +266,6 @@ func (c *eventBroker) runSendMessageWorker(ctx context.Context) { log.Debug("send message failed", zap.Error(err)) continue } - metricEventServiceSendEventDuration.Observe(time.Since(start).Seconds()) break } @@ -392,6 +396,11 @@ func (a *dispatcherStat) onSubscriptionWatermark(watermark uint64) { dispatcherInfo: a.info, eventCount: a.spanSubscription.newEventCount.Swap(0), } + // fizz: remove it after test + if a.spanSubscription.span.TableID == uint64(217) { + log.Info("fizz onSubscriptionWatermark", zap.Any("tableID", a.spanSubscription.span.TableID), zap.Any("watermark", watermark), zap.Any("eventCount", sub.eventCount)) + } + select { case a.notify <- sub: default: diff --git a/pkg/eventservice/event_service.go b/pkg/eventservice/event_service.go index 42845363f..9c7457c85 100644 --- a/pkg/eventservice/event_service.go +++ b/pkg/eventservice/event_service.go @@ -77,7 +77,7 @@ func (s *eventService) Run(ctx context.Context) error { if info.IsRegister() { s.registerDispatcher(ctx, info) } else { - s.deregisterAcceptor(info) + s.deregisterDispatcher(info) } } } @@ -96,7 +96,7 @@ func (s *eventService) handleMessage(ctx context.Context, msg *messaging.TargetM select { case <-ctx.Done(): return ctx.Err() - case s.acceptorInfoCh <- msgToAcceptorInfo(msg): + case s.acceptorInfoCh <- msgToDispatcherInfo(msg): } return nil } @@ -127,7 +127,7 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn log.Info("register acceptor", zap.Uint64("clusterID", clusterID), zap.String("acceptorID", info.GetID()), zap.Uint64("tableID", span.TableID), zap.Uint64("startTs", startTs)) } -func (s *eventService) deregisterAcceptor(dispatcherInfo DispatcherInfo) { +func (s *eventService) deregisterDispatcher(dispatcherInfo DispatcherInfo) { clusterID := dispatcherInfo.GetClusterID() c, ok := s.brokers[clusterID] if !ok { @@ -139,6 +139,6 @@ func (s *eventService) deregisterAcceptor(dispatcherInfo DispatcherInfo) { } // TODO: implement the following functions -func msgToAcceptorInfo(msg *messaging.TargetMessage) DispatcherInfo { +func msgToDispatcherInfo(msg *messaging.TargetMessage) DispatcherInfo { return msg.Message.(messaging.RegisterDispatcherRequest) } diff --git a/pkg/eventservice/event_service_test.go b/pkg/eventservice/event_service_test.go index d1814bc28..c49cdfca2 100644 --- a/pkg/eventservice/event_service_test.go +++ b/pkg/eventservice/event_service_test.go @@ -17,7 +17,6 @@ import ( appcontext "github.com/flowbehappy/tigate/pkg/common/context" "github.com/flowbehappy/tigate/pkg/config" "github.com/flowbehappy/tigate/pkg/messaging" - "github.com/flowbehappy/tigate/server/watcher" "github.com/google/uuid" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -391,7 +390,7 @@ func TestDispatcherCommunicateWithEventService(t *testing.T) { defer cancel() serverId := messaging.NewServerId() - appcontext.SetService(appcontext.MessageCenter, messaging.NewMessageCenter(ctx, serverId, watcher.TempEpoch, config.NewDefaultMessageCenterConfig())) + appcontext.SetService(appcontext.MessageCenter, messaging.NewMessageCenter(ctx, serverId, 1, config.NewDefaultMessageCenterConfig())) appcontext.SetService(appcontext.EventCollector, eventcollector.NewEventCollector(100*1024*1024*1024, serverId)) // 100GB for demo mockStore := newMockEventStore() @@ -412,7 +411,7 @@ func TestDispatcherCommunicateWithEventService(t *testing.T) { startTs := uint64(1) tableEventDispatcher := dispatcher.NewTableEventDispatcher(tableSpan, mysqlSink, startTs, nil) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs) + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).RegisterDispatcher(tableEventDispatcher, startTs, nil) time.Sleep(1 * time.Second) // add events to logpuller diff --git a/pkg/messaging/message.go b/pkg/messaging/message.go index 5dd03032f..a94274126 100644 --- a/pkg/messaging/message.go +++ b/pkg/messaging/message.go @@ -18,7 +18,9 @@ type IOType int32 const ( TypeInvalid IOType = iota TypeBytes + // LogService related TypeTxnEvent + TypeHeartBeatRequest TypeHeartBeatResponse TypeScheduleDispatcherRequest From 754fffb52dd7c11c0e373975fd468cd95f970f30 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 13 Aug 2024 11:26:23 +0800 Subject: [PATCH 02/29] add more log Signed-off-by: dongmen <414110582@qq.com> --- .../event_dispatcher_manager.go | 7 ++++++- maintainer/maintainer.go | 3 +++ pkg/eventservice/event_broker.go | 3 +++ pkg/metrics/eventService.go | 16 ++++++++++++++++ 4 files changed, 28 insertions(+), 1 deletion(-) diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index d22f1bb2a..e1d2ac936 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -386,6 +386,10 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * // TODO: we need to consider how to deal with the checkpointTs of the removed dispatcher if the message will be discarded. dispatcher.CollectDispatcherHeartBeatInfo(tableEventDispatcher, dispatcherHeartBeatInfo) + if dispatcherHeartBeatInfo.TableSpan.TableID == uint64(217) { + log.Info("fizz on CollectHeartbeatInfo", zap.Uint64("tableID", dispatcherHeartBeatInfo.TableSpan.TableID), zap.Uint64("checkpointTs", dispatcherHeartBeatInfo.Watermark.CheckpointTs), zap.Uint64("resolvedTs", dispatcherHeartBeatInfo.Watermark.ResolvedTs)) + } + componentStatus := dispatcherHeartBeatInfo.ComponentStatus if componentStatus == heartbeatpb.ComponentState_Stopping { watermark, ok := tableEventDispatcher.TryClose() @@ -407,6 +411,7 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * message.Statuses = append(message.Statuses, &heartbeatpb.TableSpanStatus{ Span: dispatcherHeartBeatInfo.TableSpan.TableSpan, ComponentStatus: dispatcherHeartBeatInfo.ComponentStatus, + CheckpointTs: dispatcherHeartBeatInfo.Watermark.CheckpointTs, }) } } @@ -414,7 +419,7 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * for _, tableSpan := range toReomveTableSpans { e.cleanTableEventDispatcher(tableSpan) } - + log.Info("fizz on CollectHeartbeatInfo", zap.Uint64("checkpointTs", message.Watermark.CheckpointTs), zap.Uint64("resolvedTs", message.Watermark.ResolvedTs)) e.metricCheckpointTs.Set(float64(oracle.ExtractPhysical(message.Watermark.CheckpointTs))) e.metricResolveTs.Set(float64(oracle.ExtractPhysical(message.Watermark.ResolvedTs))) return &message diff --git a/maintainer/maintainer.go b/maintainer/maintainer.go index 2399c06ed..ddff7351d 100644 --- a/maintainer/maintainer.go +++ b/maintainer/maintainer.go @@ -368,6 +368,9 @@ func (m *Maintainer) onHeartBeatRequest(msg *messaging.TargetMessage) error { var status []scheduler.InferiorStatus for _, info := range req.Statuses { + if info.Span.TableID == uint64(217) { + log.Info("fizz on heartbeat request", zap.Uint64("tableID", info.Span.TableID), zap.Uint64("checkpointTs", info.CheckpointTs)) + } status = append(status, &ReplicaSetStatus{ ID: &common.TableSpan{ TableSpan: info.Span, diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 6ddd79ab1..837dfb0fe 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -52,6 +52,9 @@ type eventBroker struct { wg *sync.WaitGroup // cancel is used to cancel the goroutines spawned by the eventBroker. cancel context.CancelFunc + + metricEventServiceResolvedTs prometheus.HistogramVec + metricEventServiceResolvedTsLag prometheus.HistogramVec } func newEventBroker( diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index fe0f2243e..bcadc30cc 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -42,6 +42,20 @@ var ( Help: "The duration of sending events by the event service", Buckets: prometheus.DefBuckets, }, []string{"type"}) + EventServiceResolvedTsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "event_service", + Name: "resolved_ts", + Help: "resolved ts of changefeeds", + }, []string{"namespace", "changefeed"}) + EventServiceResolvedTsLagGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "event_service", + Name: "resolved_ts_lag", + Help: "resolved ts lag of changefeeds in seconds", + }, []string{"namespace", "changefeed"}) ) // InitMetrics registers all metrics in this file. @@ -49,4 +63,6 @@ func InitEventServiceMetrics(registry *prometheus.Registry) { registry.MustRegister(SorterOutputEventCount) registry.MustRegister(EventServiceSendEventCount) registry.MustRegister(EventServiceSendEventDuration) + registry.MustRegister(EventServiceResolvedTsGauge) + registry.MustRegister(EventServiceResolvedTsLagGauge) } From 21ba685a69916bd9118c0ad992c70325b4dd95ed Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 13 Aug 2024 15:16:17 +0800 Subject: [PATCH 03/29] add resolved Ts metrics in eventService Signed-off-by: dongmen <414110582@qq.com> --- maintainer/maintainer.go | 5 ++--- pkg/eventservice/event_broker.go | 37 ++++++++++++++++++++++++++++++-- pkg/metrics/eventService.go | 4 ++-- 3 files changed, 39 insertions(+), 7 deletions(-) diff --git a/maintainer/maintainer.go b/maintainer/maintainer.go index ddff7351d..538ca59b1 100644 --- a/maintainer/maintainer.go +++ b/maintainer/maintainer.go @@ -299,6 +299,8 @@ func (m *Maintainer) calCheckpointTs() { } func (m *Maintainer) updateMetrics() { + log.Info("fizz update metrics", zap.String("id", m.id.String()), zap.Uint64("checkpointTs", m.watermark.CheckpointTs), zap.Uint64("resolvedTs", m.watermark.ResolvedTs)) + phyCkpTs := oracle.ExtractPhysical(m.watermark.CheckpointTs) m.changefeedCheckpointTsGauge.Set(float64(phyCkpTs)) lag := (oracle.GetPhysical(time.Now()) - phyCkpTs) / 1e3 @@ -368,9 +370,6 @@ func (m *Maintainer) onHeartBeatRequest(msg *messaging.TargetMessage) error { var status []scheduler.InferiorStatus for _, info := range req.Statuses { - if info.Span.TableID == uint64(217) { - log.Info("fizz on heartbeat request", zap.Uint64("tableID", info.Span.TableID), zap.Uint64("checkpointTs", info.CheckpointTs)) - } status = append(status, &ReplicaSetStatus{ ID: &common.TableSpan{ TableSpan: info.Span, diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 837dfb0fe..3c0984611 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -14,6 +14,7 @@ import ( "github.com/flowbehappy/tigate/pkg/metrics" "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) @@ -53,8 +54,8 @@ type eventBroker struct { // cancel is used to cancel the goroutines spawned by the eventBroker. cancel context.CancelFunc - metricEventServiceResolvedTs prometheus.HistogramVec - metricEventServiceResolvedTsLag prometheus.HistogramVec + metricEventServiceResolvedTs prometheus.Gauge + metricEventServiceResolvedTsLag prometheus.Gauge } func newEventBroker( @@ -79,6 +80,9 @@ func newEventBroker( messageCh: make(chan *wrapMessage, defaultChannelSize), cancel: cancel, wg: wg, + + metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge.WithLabelValues("all"), + metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("all"), } c.runGenerateScanTask(ctx) c.runScanWorker(ctx) @@ -316,6 +320,35 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { }() } +func (c *eventBroker) updateMetrics(ctx context.Context) { + ticker := time.NewTicker(time.Second * 5) + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + var minResolvedTs uint64 + c.dispatchers.mu.RLock() + for _, dispatcher := range c.dispatchers.m { + resolvedTs := dispatcher.spanSubscription.watermark.Load() + if minResolvedTs == 0 || resolvedTs < minResolvedTs { + minResolvedTs = resolvedTs + } + } + c.dispatchers.mu.RUnlock() + if minResolvedTs == 0 { + continue + } + + phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) + lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 + + c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) + c.metricEventServiceResolvedTsLag.Set(float64(lag)) + } + } +} + func (c *eventBroker) close() { c.cancel() c.wg.Wait() diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index bcadc30cc..8f9a49b8a 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -48,14 +48,14 @@ var ( Subsystem: "event_service", Name: "resolved_ts", Help: "resolved ts of changefeeds", - }, []string{"namespace", "changefeed"}) + }, []string{"changefeed"}) EventServiceResolvedTsLagGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts_lag", Help: "resolved ts lag of changefeeds in seconds", - }, []string{"namespace", "changefeed"}) + }, []string{"changefeed"}) ) // InitMetrics registers all metrics in this file. From 8d10db9707593fc4308064b25e5a908dee8b836d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 13 Aug 2024 15:21:19 +0800 Subject: [PATCH 04/29] add resolved Ts metrics in eventService 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 3c0984611..8a1be7e6d 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -342,7 +342,7 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 - + log.Info("fizz on update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Any("lag", lag)) c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) c.metricEventServiceResolvedTsLag.Set(float64(lag)) } From cbcc86a8f147d19740560c098ad7447f92d9ee1b Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 13 Aug 2024 16:24:38 +0800 Subject: [PATCH 05/29] add resolved Ts metrics in eventService 3 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 60 +++++++++++++-------------- pkg/eventservice/event_broker_test.go | 16 +++++++ 2 files changed, 46 insertions(+), 30 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 8a1be7e6d..5f041ca25 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -73,14 +73,13 @@ func newEventBroker( mu sync.RWMutex m map[string]*dispatcherStat }{m: make(map[string]*dispatcherStat)}, - msgSender: mc, - changedCh: make(chan *subscriptionChange, defaultChannelSize), - taskPool: newScanTaskPool(), - scanWorkerCount: defaultWorkerCount, - messageCh: make(chan *wrapMessage, defaultChannelSize), - cancel: cancel, - wg: wg, - + msgSender: mc, + changedCh: make(chan *subscriptionChange, defaultChannelSize), + taskPool: newScanTaskPool(), + scanWorkerCount: defaultWorkerCount, + messageCh: make(chan *wrapMessage, defaultChannelSize), + cancel: cancel, + wg: wg, metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge.WithLabelValues("all"), metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("all"), } @@ -322,31 +321,32 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { func (c *eventBroker) updateMetrics(ctx context.Context) { ticker := time.NewTicker(time.Second * 5) - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - var minResolvedTs uint64 - c.dispatchers.mu.RLock() - for _, dispatcher := range c.dispatchers.m { - resolvedTs := dispatcher.spanSubscription.watermark.Load() - if minResolvedTs == 0 || resolvedTs < minResolvedTs { - minResolvedTs = resolvedTs + go func() { + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + var minResolvedTs uint64 + c.dispatchers.mu.RLock() + for _, dispatcher := range c.dispatchers.m { + resolvedTs := dispatcher.spanSubscription.watermark.Load() + if minResolvedTs == 0 || resolvedTs < minResolvedTs { + minResolvedTs = resolvedTs + } } + c.dispatchers.mu.RUnlock() + if minResolvedTs == 0 { + continue + } + phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) + lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 + log.Info("fizz on update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Any("lag", lag)) + c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) + c.metricEventServiceResolvedTsLag.Set(float64(lag)) } - c.dispatchers.mu.RUnlock() - if minResolvedTs == 0 { - continue - } - - phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) - lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 - log.Info("fizz on update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Any("lag", lag)) - c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) - c.metricEventServiceResolvedTsLag.Set(float64(lag)) } - } + }() } func (c *eventBroker) close() { diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index fad8329b5..bca8fe98a 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -165,3 +165,19 @@ func newTableSpan(tableID uint64, start, end string) *common.TableSpan { } return res } + +type did struct { + cid uint64 + tid uint64 + sid uint64 +} + +func TestDIDAsMapKey(t *testing.T) { + k1 := did{1, 2, 3} + k2 := did{1, 2, 4} + + m := make(map[did]string) + m[k1] = "a" + m[k2] = "b" + require.Equal(t, "a", m[k1]) +} From e2b7c01d758f7b4d2322c78db1ec94eaa2741b1d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 13 Aug 2024 17:18:07 +0800 Subject: [PATCH 06/29] use send command to send heartbeat Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/dispatchermanager/heartbeat_collector.go | 2 +- pkg/eventservice/event_broker.go | 1 + pkg/metrics/eventService.go | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/downstreamadapter/dispatchermanager/heartbeat_collector.go b/downstreamadapter/dispatchermanager/heartbeat_collector.go index 17fc654b3..0222ffba4 100644 --- a/downstreamadapter/dispatchermanager/heartbeat_collector.go +++ b/downstreamadapter/dispatchermanager/heartbeat_collector.go @@ -98,7 +98,7 @@ func (c *HeartBeatCollector) RegisterEventDispatcherManager(m *EventDispatcherMa func (c *HeartBeatCollector) SendHeartBeatMessages() { for { heartBeatRequestWithTargetID := c.requestQueue.Dequeue() - err := appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).SendEvent(&messaging.TargetMessage{ + err := appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).SendCommand(&messaging.TargetMessage{ To: heartBeatRequestWithTargetID.TargetID, Topic: messaging.MaintainerManagerTopic, Type: messaging.TypeHeartBeatRequest, diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 5f041ca25..98daac926 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -87,6 +87,7 @@ func newEventBroker( c.runScanWorker(ctx) c.runSendMessageWorker(ctx) c.logSlowDispatchers(ctx) + c.updateMetrics(ctx) return c } diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index 8f9a49b8a..1e76c9942 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -47,14 +47,14 @@ var ( Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts", - Help: "resolved ts of changefeeds", + Help: "resolved ts of eventService", }, []string{"changefeed"}) EventServiceResolvedTsLagGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts_lag", - Help: "resolved ts lag of changefeeds in seconds", + Help: "resolved ts lag of eventService in seconds", }, []string{"changefeed"}) ) From 42ae90da18092b4f9d185d52f284ab7614a1bd9a Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 11:03:29 +0800 Subject: [PATCH 07/29] add checkpointTsLag and resolvedTsLag metrics to dispatcherManager Signed-off-by: dongmen <414110582@qq.com> --- .../dispatcher/table_event_dispatcher.go | 3 --- .../event_dispatcher_manager.go | 20 +++++++++------- maintainer/maintainer.go | 2 -- pkg/eventservice/event_broker.go | 12 ---------- pkg/metrics/dispatcher.go | 24 +++++++++++++++---- 5 files changed, 32 insertions(+), 29 deletions(-) diff --git a/downstreamadapter/dispatcher/table_event_dispatcher.go b/downstreamadapter/dispatcher/table_event_dispatcher.go index 97c0c9d31..9b29c86ba 100644 --- a/downstreamadapter/dispatcher/table_event_dispatcher.go +++ b/downstreamadapter/dispatcher/table_event_dispatcher.go @@ -200,9 +200,6 @@ func (d *TableEventDispatcher) GetCheckpointTs() uint64 { } func (d *TableEventDispatcher) UpdateResolvedTs(ts uint64) { - if d.tableSpan.TableID == uint64(217) { - log.Info("fizz update resolvedTs", zap.Uint64("tableID", d.tableSpan.TableID), zap.Uint64("resolvedTs", ts)) - } d.resolvedTs.Set(ts) } diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index e1d2ac936..e40a4ed3b 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -77,7 +77,9 @@ type EventDispatcherManager struct { filter filter.Filter metricCreateDispatcherDuration prometheus.Observer metricCheckpointTs prometheus.Gauge + metricCheckpointTsLag prometheus.Gauge metricResolveTs prometheus.Gauge + metricResolvedTsLag prometheus.Gauge closing bool closed atomic.Bool @@ -97,7 +99,9 @@ func NewEventDispatcherManager(changefeedID model.ChangeFeedID, changefeedConfig tableEventDispatcherCount: metrics.TableEventDispatcherGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricCreateDispatcherDuration: metrics.CreateDispatcherDuration.WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricCheckpointTs: metrics.EventDispatcherManagerCheckpointTsGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID), + metricCheckpointTsLag: metrics.EventDispatcherManagerCheckpointTsLagGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricResolveTs: metrics.EventDispatcherManagerResolvedTsGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID), + metricResolvedTsLag: metrics.EventDispatcherManagerResolvedTsLagGauge.WithLabelValues(changefeedID.Namespace, changefeedID.ID), } // TODO: 最后去更新一下 filter 的内部 NewFilter 函数,现在是在套壳适配 @@ -385,11 +389,6 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * // If it's closed successfully, we could clean it up. // TODO: we need to consider how to deal with the checkpointTs of the removed dispatcher if the message will be discarded. dispatcher.CollectDispatcherHeartBeatInfo(tableEventDispatcher, dispatcherHeartBeatInfo) - - if dispatcherHeartBeatInfo.TableSpan.TableID == uint64(217) { - log.Info("fizz on CollectHeartbeatInfo", zap.Uint64("tableID", dispatcherHeartBeatInfo.TableSpan.TableID), zap.Uint64("checkpointTs", dispatcherHeartBeatInfo.Watermark.CheckpointTs), zap.Uint64("resolvedTs", dispatcherHeartBeatInfo.Watermark.ResolvedTs)) - } - componentStatus := dispatcherHeartBeatInfo.ComponentStatus if componentStatus == heartbeatpb.ComponentState_Stopping { watermark, ok := tableEventDispatcher.TryClose() @@ -419,9 +418,14 @@ func (e *EventDispatcherManager) CollectHeartbeatInfo(needCompleteStatus bool) * for _, tableSpan := range toReomveTableSpans { e.cleanTableEventDispatcher(tableSpan) } - log.Info("fizz on CollectHeartbeatInfo", zap.Uint64("checkpointTs", message.Watermark.CheckpointTs), zap.Uint64("resolvedTs", message.Watermark.ResolvedTs)) - e.metricCheckpointTs.Set(float64(oracle.ExtractPhysical(message.Watermark.CheckpointTs))) - e.metricResolveTs.Set(float64(oracle.ExtractPhysical(message.Watermark.ResolvedTs))) + ckptTs := oracle.ExtractPhysical(message.Watermark.CheckpointTs) + e.metricCheckpointTs.Set(float64(ckptTs)) + lag := (oracle.GetPhysical(time.Now()) - ckptTs) / 1e3 + e.metricCheckpointTsLag.Set(float64(lag)) + resolvedTs := oracle.ExtractPhysical(message.Watermark.ResolvedTs) + e.metricResolveTs.Set(float64(resolvedTs)) + lag = (oracle.GetPhysical(time.Now()) - resolvedTs) / 1e3 + e.metricResolvedTsLag.Set(float64(lag)) return &message } diff --git a/maintainer/maintainer.go b/maintainer/maintainer.go index 538ca59b1..2399c06ed 100644 --- a/maintainer/maintainer.go +++ b/maintainer/maintainer.go @@ -299,8 +299,6 @@ func (m *Maintainer) calCheckpointTs() { } func (m *Maintainer) updateMetrics() { - log.Info("fizz update metrics", zap.String("id", m.id.String()), zap.Uint64("checkpointTs", m.watermark.CheckpointTs), zap.Uint64("resolvedTs", m.watermark.ResolvedTs)) - phyCkpTs := oracle.ExtractPhysical(m.watermark.CheckpointTs) m.changefeedCheckpointTsGauge.Set(float64(phyCkpTs)) lag := (oracle.GetPhysical(time.Now()) - phyCkpTs) / 1e3 diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 98daac926..b299fac27 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -153,12 +153,6 @@ func (c *eventBroker) runScanWorker(ctx context.Context) { case <-ctx.Done(): return case task := <-c.taskPool.popTask(chIndex): - if task.dispatcherStat.spanSubscription.span.TableID == uint64(217) { - log.Info("fizz on scan task", - zap.Any("tableID", task.dispatcherStat.spanSubscription.span.TableID), - zap.Any("watermark", task.dataRange.EndTs), zap.Any("eventCount", task.eventCount)) - } - needScan := task.checkAndAdjustScanTask() if !needScan { continue @@ -342,7 +336,6 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { } phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 - log.Info("fizz on update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Any("lag", lag)) c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) c.metricEventServiceResolvedTsLag.Set(float64(lag)) } @@ -433,11 +426,6 @@ func (a *dispatcherStat) onSubscriptionWatermark(watermark uint64) { dispatcherInfo: a.info, eventCount: a.spanSubscription.newEventCount.Swap(0), } - // fizz: remove it after test - if a.spanSubscription.span.TableID == uint64(217) { - log.Info("fizz onSubscriptionWatermark", zap.Any("tableID", a.spanSubscription.span.TableID), zap.Any("watermark", watermark), zap.Any("eventCount", sub.eventCount)) - } - select { case a.notify <- sub: default: diff --git a/pkg/metrics/dispatcher.go b/pkg/metrics/dispatcher.go index b8441f095..089729120 100644 --- a/pkg/metrics/dispatcher.go +++ b/pkg/metrics/dispatcher.go @@ -46,19 +46,35 @@ var ( EventDispatcherManagerResolvedTsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "sink", - Name: "event_dispatcher_manager_resolved_ts", + Subsystem: "dispatcher", + Name: "resolved_ts", Help: "Resolved ts of event dispatcher manager(changefeed)", }, []string{"namespace", "changefeed"}) + EventDispatcherManagerResolvedTsLagGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "dispatcher", + Name: "resolved_ts_lag", + Help: "Resolved ts lag of event dispatcher manager(changefeed) in seconds", + }, []string{"namespace", "changefeed"}) + EventDispatcherManagerCheckpointTsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "sink", - Name: "event_dispatcher_manager_checkpoint_ts", + Subsystem: "dispatcher", + Name: "checkpoint_ts", Help: "Checkpoint ts of event dispatcher manager(changefeed)", }, []string{"namespace", "changefeed"}) + EventDispatcherManagerCheckpointTsLagGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "dispatcher", + Name: "checkpoint_ts_lag", + Help: "Checkpoint ts lag of event dispatcher manager(changefeed) in seconds", + }, []string{"namespace", "changefeed"}) + HandleDispatcherRequsetCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "ticdc", From 20440e57e457cec4113921f3d4d967ae86c9d75c Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 11:11:42 +0800 Subject: [PATCH 08/29] remove useless code Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker_test.go | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index bca8fe98a..fad8329b5 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -165,19 +165,3 @@ func newTableSpan(tableID uint64, start, end string) *common.TableSpan { } return res } - -type did struct { - cid uint64 - tid uint64 - sid uint64 -} - -func TestDIDAsMapKey(t *testing.T) { - k1 := did{1, 2, 3} - k2 := did{1, 2, 4} - - m := make(map[did]string) - m[k1] = "a" - m[k2] = "b" - require.Equal(t, "a", m[k1]) -} From 4d2879b77caa44b1841fb7dcad5ad4b9cfcf4760 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 11:49:56 +0800 Subject: [PATCH 09/29] add scan task inqueue duration metric Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 7 +++++-- pkg/metrics/dispatcher.go | 2 ++ pkg/metrics/eventService.go | 9 +++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index b299fac27..b6eda8bfb 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -56,6 +56,7 @@ type eventBroker struct { metricEventServiceResolvedTs prometheus.Gauge metricEventServiceResolvedTsLag prometheus.Gauge + metricTaskInQueueDuration prometheus.Observer } func newEventBroker( @@ -82,6 +83,7 @@ func newEventBroker( wg: wg, metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge.WithLabelValues("all"), metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("all"), + metricTaskInQueueDuration: metrics.EventServiceScanTaskInQueueDuration.WithLabelValues("all"), } c.runGenerateScanTask(ctx) c.runScanWorker(ctx) @@ -135,6 +137,7 @@ func (c *eventBroker) runGenerateScanTask(ctx context.Context) { dispatcherStat: dispatcher, dataRange: dataRange, eventCount: change.eventCount, + createTime: time.Now(), } c.taskPool.pushTask(task) } @@ -157,7 +160,7 @@ func (c *eventBroker) runScanWorker(ctx context.Context) { if !needScan { continue } - + c.metricTaskInQueueDuration.Observe(time.Since(task.createTime).Seconds()) remoteID := messaging.ServerId(task.dispatcherStat.info.GetServerID()) dispatcherID := task.dispatcherStat.info.GetID() topic := task.dispatcherStat.info.GetTopic() @@ -169,7 +172,6 @@ func (c *eventBroker) runScanWorker(ctx context.Context) { task.dispatcherStat.lastSent.Store(time.Now()) continue } - //2. Get events iterator from eventStore. iter, err := c.eventStore.GetIterator(task.dataRange) if err != nil { @@ -463,6 +465,7 @@ type scanTask struct { dispatcherStat *dispatcherStat dataRange *common.DataRange eventCount uint64 + createTime time.Time } func (t *scanTask) checkAndAdjustScanTask() bool { diff --git a/pkg/metrics/dispatcher.go b/pkg/metrics/dispatcher.go index 089729120..38c48ab1d 100644 --- a/pkg/metrics/dispatcher.go +++ b/pkg/metrics/dispatcher.go @@ -103,7 +103,9 @@ func InitDisaptcherMetrics(registry *prometheus.Registry) { registry.MustRegister(TableEventDispatcherGauge) registry.MustRegister(CreateDispatcherDuration) registry.MustRegister(EventDispatcherManagerResolvedTsGauge) + registry.MustRegister(EventDispatcherManagerResolvedTsLagGauge) registry.MustRegister(EventDispatcherManagerCheckpointTsGauge) + registry.MustRegister(EventDispatcherManagerCheckpointTsLagGauge) registry.MustRegister(HandleDispatcherRequsetCounter) registry.MustRegister(DispatcherReceivedEventCount) registry.MustRegister(EventCollectorRegisteredDispatcherCount) diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index 1e76c9942..5c745d3e1 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -56,6 +56,14 @@ var ( Name: "resolved_ts_lag", Help: "resolved ts lag of eventService in seconds", }, []string{"changefeed"}) + EventServiceScanTaskInQueueDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "event_service", + Name: "scan_task_in_queue_duration", + Help: "The duration of scanning task in queue", + Buckets: prometheus.DefBuckets, + }, []string{"changefeed"}) ) // InitMetrics registers all metrics in this file. @@ -65,4 +73,5 @@ func InitEventServiceMetrics(registry *prometheus.Registry) { registry.MustRegister(EventServiceSendEventDuration) registry.MustRegister(EventServiceResolvedTsGauge) registry.MustRegister(EventServiceResolvedTsLagGauge) + registry.MustRegister(EventServiceScanTaskInQueueDuration) } From b234d221992134aefbb358a0476521a5abebaddf Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 12:25:17 +0800 Subject: [PATCH 10/29] add event lag duration for event collector Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 7 +++++++ pkg/messaging/message.go | 3 +++ pkg/metrics/dispatcher.go | 10 ++++++++++ 3 files changed, 20 insertions(+) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index d7ff37e27..201ad57bd 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -81,6 +81,8 @@ type EventCollector struct { registerMessageChan *chann.DrainableChann[*RegisterInfo] // for temp metricDispatcherReceivedKVEventCount prometheus.Counter metricDispatcherReceivedResolvedTsEventCount prometheus.Counter + metricReceiveEventLagDuration prometheus.Observer + metricReceiveResolvedTsEventLagDuration prometheus.Observer } func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *EventCollector { @@ -91,6 +93,8 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev registerMessageChan: chann.NewAutoDrainChann[*RegisterInfo](), metricDispatcherReceivedKVEventCount: metrics.DispatcherReceivedEventCount.WithLabelValues("KVEvent"), metricDispatcherReceivedResolvedTsEventCount: metrics.DispatcherReceivedEventCount.WithLabelValues("ResolvedTs"), + metricReceiveEventLagDuration: metrics.EventCollectorReceivedEventLagDuration.WithLabelValues("KVEvent"), + metricReceiveResolvedTsEventLagDuration: metrics.EventCollectorReceivedEventLagDuration.WithLabelValues("ResolvedTs"), } appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).RegisterHandler(messaging.EventCollectorTopic, eventCollector.RecvEventsMessage) @@ -194,6 +198,7 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T log.Error("invalid event feed message", zap.Any("msg", msg)) return apperror.AppError{Type: apperror.ErrorTypeInvalidMessage, Reason: fmt.Sprintf("invalid heartbeat response message")} } + inflightDuration := time.Since(time.Unix(0, msg.CrateAt)).Milliseconds() dispatcherID := txnEvent.DispatcherID // log.Info("Recv TxnEvent", zap.Any("dispatcherID", dispatcherID), zap.Any("event is dml event", txnEvent.IsDMLEvent())) @@ -232,9 +237,11 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T if txnEvent.IsDMLEvent() { dispatcherItem.PushTxnEvent(txnEvent) c.metricDispatcherReceivedKVEventCount.Inc() + c.metricReceiveEventLagDuration.Observe(float64(inflightDuration)) } else { dispatcherItem.UpdateResolvedTs(txnEvent.ResolvedTs) c.metricDispatcherReceivedResolvedTsEventCount.Inc() + c.metricReceiveResolvedTsEventLagDuration.Observe(float64(inflightDuration)) } // dispatcherItem.UpdateResolvedTs(eventFeeds.ResolvedTs) diff --git a/pkg/messaging/message.go b/pkg/messaging/message.go index a94274126..db4749939 100644 --- a/pkg/messaging/message.go +++ b/pkg/messaging/message.go @@ -2,6 +2,7 @@ package messaging import ( "fmt" + "time" "github.com/flowbehappy/tigate/eventpb" "github.com/flowbehappy/tigate/heartbeatpb" @@ -190,6 +191,7 @@ type TargetMessage struct { Topic string Type IOType Message IOTypeT + CrateAt int64 } // NewTargetMessage creates a new TargetMessage to be sent to a target server. @@ -233,6 +235,7 @@ func NewTargetMessage(To ServerId, Topic string, Message IOTypeT) *TargetMessage Type: ioType, Topic: Topic, Message: Message, + CrateAt: time.Now().UnixMilli(), } } diff --git a/pkg/metrics/dispatcher.go b/pkg/metrics/dispatcher.go index 38c48ab1d..2264954fd 100644 --- a/pkg/metrics/dispatcher.go +++ b/pkg/metrics/dispatcher.go @@ -96,6 +96,15 @@ var ( Name: "event_collector_registered_dispatcher_count", Help: "The number of registered dispatchers in the event collector", }) + + EventCollectorReceivedEventLagDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "dispatcher", + Name: "received_event_lag_duration", + Help: "The duration of lag between the event collector received event and the event's ts", + Buckets: prometheus.DefBuckets, + }, []string{"type"}) ) func InitDisaptcherMetrics(registry *prometheus.Registry) { @@ -109,4 +118,5 @@ func InitDisaptcherMetrics(registry *prometheus.Registry) { registry.MustRegister(HandleDispatcherRequsetCounter) registry.MustRegister(DispatcherReceivedEventCount) registry.MustRegister(EventCollectorRegisteredDispatcherCount) + registry.MustRegister(EventCollectorReceivedEventLagDuration) } From 6ae0bd058d6779f1f51172e5ca529c716f898897 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 14:28:05 +0800 Subject: [PATCH 11/29] add resolvedTs lag for event collector Signed-off-by: dongmen <414110582@qq.com> --- .../eventcollector/event_collector.go | 47 +++++++++++++++---- pkg/eventservice/event_broker.go | 13 +++-- pkg/eventservice/event_service.go | 8 +++- pkg/metrics/dispatcher.go | 10 ++++ 4 files changed, 62 insertions(+), 16 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 201ad57bd..cdca18be3 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -16,6 +16,7 @@ package eventcollector import ( "context" "fmt" + "math" "sync" "time" @@ -29,12 +30,13 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/chann" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) type DispatcherMap struct { - mutex sync.Mutex - m map[string]dispatcher.Dispatcher // dispatcher_id --> dispatcher + sync.RWMutex + m map[string]dispatcher.Dispatcher // dispatcher_id --> dispatcher } func newDispatcherMap() *DispatcherMap { @@ -44,21 +46,21 @@ func newDispatcherMap() *DispatcherMap { } func (m *DispatcherMap) Get(dispatcherId string) (dispatcher.Dispatcher, bool) { - m.mutex.Lock() - defer m.mutex.Unlock() + m.RLock() + defer m.RUnlock() d, ok := m.m[dispatcherId] return d, ok } func (m *DispatcherMap) Set(dispatcherId string, d dispatcher.Dispatcher) { - m.mutex.Lock() - defer m.mutex.Unlock() + m.Lock() + defer m.Unlock() m.m[dispatcherId] = d } func (m *DispatcherMap) Delete(dispatcherId string) { - m.mutex.Lock() - defer m.mutex.Unlock() + m.Lock() + defer m.Unlock() delete(m.m, dispatcherId) } @@ -83,6 +85,7 @@ type EventCollector struct { metricDispatcherReceivedResolvedTsEventCount prometheus.Counter metricReceiveEventLagDuration prometheus.Observer metricReceiveResolvedTsEventLagDuration prometheus.Observer + metricResolvedTsLag prometheus.Gauge } func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *EventCollector { @@ -95,6 +98,7 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev metricDispatcherReceivedResolvedTsEventCount: metrics.DispatcherReceivedEventCount.WithLabelValues("ResolvedTs"), metricReceiveEventLagDuration: metrics.EventCollectorReceivedEventLagDuration.WithLabelValues("KVEvent"), metricReceiveResolvedTsEventLagDuration: metrics.EventCollectorReceivedEventLagDuration.WithLabelValues("ResolvedTs"), + metricResolvedTsLag: metrics.EventCollectorResolvedTsLagGauge, } appcontext.GetService[messaging.MessageCenter](appcontext.MessageCenter).RegisterHandler(messaging.EventCollectorTopic, eventCollector.RecvEventsMessage) @@ -129,6 +133,7 @@ func NewEventCollector(globalMemoryQuota int64, serverId messaging.ServerId) *Ev metrics.EventCollectorRegisteredDispatcherCount.Inc() } }(&eventCollector) + eventCollector.updateMetrics(context.Background()) return &eventCollector } @@ -264,3 +269,29 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T } return nil } + +func (c *EventCollector) updateMetrics(ctx context.Context) error { + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + minResolvedTs := uint64(math.MaxUint64) + go func() { + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + c.dispatcherMap.RLock() + for _, d := range c.dispatcherMap.m { + if d.GetResolvedTs() < minResolvedTs { + minResolvedTs = d.GetResolvedTs() + } + } + c.dispatcherMap.RUnlock() + phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) + lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 + c.metricResolvedTsLag.Set(float64(lag)) + } + } + }() + return nil +} diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index b6eda8bfb..cbf23ae7a 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -2,6 +2,7 @@ package eventservice import ( "context" + "math" "sync" "sync/atomic" "time" @@ -88,7 +89,7 @@ func newEventBroker( c.runGenerateScanTask(ctx) c.runScanWorker(ctx) c.runSendMessageWorker(ctx) - c.logSlowDispatchers(ctx) + //c.logSlowDispatchers(ctx) c.updateMetrics(ctx) return c } @@ -317,25 +318,23 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { } func (c *eventBroker) updateMetrics(ctx context.Context) { - ticker := time.NewTicker(time.Second * 5) + ticker := time.NewTicker(time.Second * 10) + defer ticker.Stop() + minResolvedTs := uint64(math.MaxUint64) go func() { for { select { case <-ctx.Done(): return case <-ticker.C: - var minResolvedTs uint64 c.dispatchers.mu.RLock() for _, dispatcher := range c.dispatchers.m { resolvedTs := dispatcher.spanSubscription.watermark.Load() - if minResolvedTs == 0 || resolvedTs < minResolvedTs { + if resolvedTs < minResolvedTs { minResolvedTs = resolvedTs } } c.dispatchers.mu.RUnlock() - if minResolvedTs == 0 { - continue - } phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) diff --git a/pkg/eventservice/event_service.go b/pkg/eventservice/event_service.go index 9c7457c85..07cd4d816 100644 --- a/pkg/eventservice/event_service.go +++ b/pkg/eventservice/event_service.go @@ -2,6 +2,7 @@ package eventservice import ( "context" + "time" "github.com/flowbehappy/tigate/logservice/eventstore" "github.com/flowbehappy/tigate/pkg/common" @@ -106,6 +107,7 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn startTs := info.GetStartTs() span := info.GetTableSpan() + start := time.Now() c, ok := s.brokers[clusterID] if !ok { c = newEventBroker(ctx, clusterID, s.eventStore, s.mc) @@ -117,6 +119,8 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn c.dispatchers.m[info.GetID()] = dispatcher c.dispatchers.mu.Unlock() + firstDuration := time.Since(start) + start = time.Now() c.eventStore.RegisterDispatcher( info.GetID(), span, @@ -124,7 +128,9 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn dispatcher.onNewEvent, dispatcher.onSubscriptionWatermark, ) - log.Info("register acceptor", zap.Uint64("clusterID", clusterID), zap.String("acceptorID", info.GetID()), zap.Uint64("tableID", span.TableID), zap.Uint64("startTs", startTs)) + + secondDuration := time.Since(start) + log.Info("register acceptor", zap.Uint64("clusterID", clusterID), zap.String("acceptorID", info.GetID()), zap.Uint64("tableID", span.TableID), zap.Uint64("startTs", startTs), zap.Duration("firstDuration", firstDuration), zap.Duration("secondDuration", secondDuration)) } func (s *eventService) deregisterDispatcher(dispatcherInfo DispatcherInfo) { diff --git a/pkg/metrics/dispatcher.go b/pkg/metrics/dispatcher.go index 2264954fd..0a9ee868f 100644 --- a/pkg/metrics/dispatcher.go +++ b/pkg/metrics/dispatcher.go @@ -105,6 +105,14 @@ var ( Help: "The duration of lag between the event collector received event and the event's ts", Buckets: prometheus.DefBuckets, }, []string{"type"}) + + EventCollectorResolvedTsLagGauge = prometheus.NewGauge( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "event_collector", + Name: "resolved_ts_lag", + Help: "Resolved ts lag of event collector in seconds", + }) ) func InitDisaptcherMetrics(registry *prometheus.Registry) { @@ -119,4 +127,6 @@ func InitDisaptcherMetrics(registry *prometheus.Registry) { registry.MustRegister(DispatcherReceivedEventCount) registry.MustRegister(EventCollectorRegisteredDispatcherCount) registry.MustRegister(EventCollectorReceivedEventLagDuration) + registry.MustRegister(EventCollectorResolvedTsLagGauge) + } From ef0d8ab1ea25c59a517667db76816945b97347b4 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 14:47:45 +0800 Subject: [PATCH 12/29] eventService: use sync map to store dispatcher Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 47 ++++++++----------- pkg/eventservice/event_service.go | 4 +- .../event_service_performance_test.go | 3 -- pkg/eventservice/event_service_test.go | 1 - 4 files changed, 21 insertions(+), 34 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index cbf23ae7a..541a99470 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -34,10 +34,7 @@ type eventBroker struct { msgSender messaging.MessageSender // All the dispatchers that register to the eventBroker. - dispatchers struct { - mu sync.RWMutex - m map[string]*dispatcherStat - } + dispatchers sync.Map // changedCh is used to notify span subscription has new events. changedCh chan *subscriptionChange // taskPool is used to store the scan tasks and merge the tasks of same dispatcher. @@ -69,12 +66,9 @@ func newEventBroker( ctx, cancel := context.WithCancel(ctx) wg := &sync.WaitGroup{} c := &eventBroker{ - tidbClusterID: id, - eventStore: eventStore, - dispatchers: struct { - mu sync.RWMutex - m map[string]*dispatcherStat - }{m: make(map[string]*dispatcherStat)}, + tidbClusterID: id, + eventStore: eventStore, + dispatchers: sync.Map{}, msgSender: mc, changedCh: make(chan *subscriptionChange, defaultChannelSize), taskPool: newScanTaskPool(), @@ -124,13 +118,12 @@ func (c *eventBroker) runGenerateScanTask(ctx context.Context) { case <-ctx.Done(): return case change := <-c.changedCh: - c.dispatchers.mu.RLock() - dispatcher, ok := c.dispatchers.m[change.dispatcherInfo.GetID()] - c.dispatchers.mu.RUnlock() + v, ok := c.dispatchers.Load(change.dispatcherInfo.GetID()) // The dispatcher may be deleted. In such case, we just the stale notification. if !ok { continue } + dispatcher := v.(*dispatcherStat) startTs := dispatcher.watermark.Load() endTs := dispatcher.spanSubscription.watermark.Load() dataRange := common.NewDataRange(c.tidbClusterID, dispatcher.info.GetTableSpan(), startTs, endTs) @@ -281,7 +274,6 @@ func (c *eventBroker) runSendMessageWorker(ctx context.Context) { func (c *eventBroker) logSlowDispatchers(ctx context.Context) { c.wg.Add(1) ticker := time.NewTicker(time.Second * 10) - logDispatcherCount := 0 log.Info("start log slow dispatchers") go func() { defer c.wg.Done() @@ -290,14 +282,15 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { case <-ctx.Done(): return case <-ticker.C: - c.dispatchers.mu.RLock() - for _, dispatcher := range c.dispatchers.m { + logDispatcherCount := 0 + findSlow := func(key, value interface{}) bool { + dispatcher := value.(*dispatcherStat) lastUpdate := dispatcher.spanSubscription.lastUpdate.Load().(time.Time) lastSent := dispatcher.lastSent.Load().(time.Time) if time.Since(lastSent) > time.Second*30 { // limit the log count to avoid log flooding. if logDispatcherCount > 10 { - break + return false } logDispatcherCount++ _, id := dispatcher.info.GetChangefeedID() @@ -310,8 +303,9 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { zap.Uint64("dispatcher-watermark", dispatcher.watermark.Load()), zap.Uint64("subscription-eventCount", dispatcher.spanSubscription.newEventCount.Load())) } + return true } - c.dispatchers.mu.RUnlock() + c.dispatchers.Range(findSlow) } } }() @@ -327,14 +321,16 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { case <-ctx.Done(): return case <-ticker.C: - c.dispatchers.mu.RLock() - for _, dispatcher := range c.dispatchers.m { + + c.dispatchers.Range(func(key, value interface{}) bool { + dispatcher := value.(*dispatcherStat) resolvedTs := dispatcher.spanSubscription.watermark.Load() if resolvedTs < minResolvedTs { minResolvedTs = resolvedTs } - } - c.dispatchers.mu.RUnlock() + return true + }) + phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) @@ -350,15 +346,12 @@ func (c *eventBroker) close() { } func (c *eventBroker) removeDispatcher(id string) { - c.dispatchers.mu.Lock() - defer c.dispatchers.mu.Unlock() - - _, ok := c.dispatchers.m[id] + _, ok := c.dispatchers.Load(id) if !ok { return } c.eventStore.UnregisterDispatcher(id) - delete(c.dispatchers.m, id) + c.dispatchers.Delete(id) c.taskPool.removeTask(id) } diff --git a/pkg/eventservice/event_service.go b/pkg/eventservice/event_service.go index 07cd4d816..3f5093b84 100644 --- a/pkg/eventservice/event_service.go +++ b/pkg/eventservice/event_service.go @@ -115,9 +115,7 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn } dispatcher := newDispatcherStat(startTs, info, c.changedCh) - c.dispatchers.mu.Lock() - c.dispatchers.m[info.GetID()] = dispatcher - c.dispatchers.mu.Unlock() + c.dispatchers.Store(info.GetID(), dispatcher) firstDuration := time.Since(start) start = time.Now() diff --git a/pkg/eventservice/event_service_performance_test.go b/pkg/eventservice/event_service_performance_test.go index 79d3c9218..609846e95 100644 --- a/pkg/eventservice/event_service_performance_test.go +++ b/pkg/eventservice/event_service_performance_test.go @@ -11,7 +11,6 @@ import ( "github.com/flowbehappy/tigate/pkg/messaging" "github.com/google/uuid" "github.com/pingcap/log" - "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -63,7 +62,6 @@ func TestEventServiceOneMillionTable(t *testing.T) { }() start := time.Now() - clusterID := uint64(1) dispatchers := make([]DispatcherInfo, 0, tableNum) // register 1000,000 tables for i := 0; i < tableNum; i++ { @@ -73,7 +71,6 @@ func TestEventServiceOneMillionTable(t *testing.T) { for _, dispatcher := range dispatchers { esImpl.registerDispatcher(ctx, dispatcher) } - require.Len(t, esImpl.brokers[clusterID].dispatchers.m, tableNum) log.Info("register 1 million tables", zap.Duration("cost", time.Since(start))) // Update resolvedTs for each table 10 times diff --git a/pkg/eventservice/event_service_test.go b/pkg/eventservice/event_service_test.go index c49cdfca2..47d9dd0c6 100644 --- a/pkg/eventservice/event_service_test.go +++ b/pkg/eventservice/event_service_test.go @@ -317,7 +317,6 @@ func TestEventServiceBasic(t *testing.T) { require.Equal(t, 1, len(esImpl.brokers)) require.NotNil(t, esImpl.brokers[acceptorInfo.GetClusterID()]) - require.Equal(t, 1, len(esImpl.brokers[acceptorInfo.GetClusterID()].dispatchers.m)) // add events to logpuller txnEvent := &common.TxnEvent{ From 0b3b9b5bbd92c9aa84bda1bf69ee7e82285b3105 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 15 Aug 2024 15:31:51 +0800 Subject: [PATCH 13/29] refine metrics Signed-off-by: dongmen <414110582@qq.com> --- .../eventcollector/event_collector.go | 5 ++--- pkg/eventservice/event_broker.go | 15 ++++++--------- pkg/metrics/eventService.go | 12 ++++++------ 3 files changed, 14 insertions(+), 18 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index cdca18be3..c8d8d0691 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -16,7 +16,6 @@ package eventcollector import ( "context" "fmt" - "math" "sync" "time" @@ -273,7 +272,7 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T func (c *EventCollector) updateMetrics(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() - minResolvedTs := uint64(math.MaxUint64) + minResolvedTs := uint64(0) go func() { for { select { @@ -282,7 +281,7 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { case <-ticker.C: c.dispatcherMap.RLock() for _, d := range c.dispatcherMap.m { - if d.GetResolvedTs() < minResolvedTs { + if minResolvedTs == 0 || d.GetResolvedTs() < minResolvedTs { minResolvedTs = d.GetResolvedTs() } } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 541a99470..d59684a65 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -2,7 +2,6 @@ package eventservice import ( "context" - "math" "sync" "sync/atomic" "time" @@ -76,9 +75,9 @@ func newEventBroker( messageCh: make(chan *wrapMessage, defaultChannelSize), cancel: cancel, wg: wg, - metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge.WithLabelValues("all"), - metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("all"), - metricTaskInQueueDuration: metrics.EventServiceScanTaskInQueueDuration.WithLabelValues("all"), + metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge, + metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge, + metricTaskInQueueDuration: metrics.EventServiceScanTaskInQueueDuration, } c.runGenerateScanTask(ctx) c.runScanWorker(ctx) @@ -312,25 +311,23 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { } func (c *eventBroker) updateMetrics(ctx context.Context) { - ticker := time.NewTicker(time.Second * 10) + ticker := time.NewTicker(time.Second * 5) defer ticker.Stop() - minResolvedTs := uint64(math.MaxUint64) + minResolvedTs := uint64(0) go func() { for { select { case <-ctx.Done(): return case <-ticker.C: - c.dispatchers.Range(func(key, value interface{}) bool { dispatcher := value.(*dispatcherStat) resolvedTs := dispatcher.spanSubscription.watermark.Load() - if resolvedTs < minResolvedTs { + if minResolvedTs == 0 || resolvedTs < minResolvedTs { minResolvedTs = resolvedTs } return true }) - phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index 5c745d3e1..39e3467a4 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -42,28 +42,28 @@ var ( Help: "The duration of sending events by the event service", Buckets: prometheus.DefBuckets, }, []string{"type"}) - EventServiceResolvedTsGauge = prometheus.NewGaugeVec( + EventServiceResolvedTsGauge = prometheus.NewGauge( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts", Help: "resolved ts of eventService", - }, []string{"changefeed"}) - EventServiceResolvedTsLagGauge = prometheus.NewGaugeVec( + }) + EventServiceResolvedTsLagGauge = prometheus.NewGauge( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts_lag", Help: "resolved ts lag of eventService in seconds", - }, []string{"changefeed"}) - EventServiceScanTaskInQueueDuration = prometheus.NewHistogramVec( + }) + EventServiceScanTaskInQueueDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "scan_task_in_queue_duration", Help: "The duration of scanning task in queue", Buckets: prometheus.DefBuckets, - }, []string{"changefeed"}) + }) ) // InitMetrics registers all metrics in this file. From db830e8ac4f7942a39a84e6d44c96adc7dbacffc Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 10:25:28 +0800 Subject: [PATCH 14/29] fix some error Signed-off-by: dongmen <414110582@qq.com> --- .../dispathermanagermanager/dispatcher_manager_manager.go | 3 ++- pkg/messaging/router.go | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go index 11e5a8187..d993296b8 100644 --- a/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go +++ b/downstreamadapter/dispathermanagermanager/dispatcher_manager_manager.go @@ -99,7 +99,8 @@ func (m *DispatcherManagerManager) handleAddDispatcherManager(from messaging.Ser eventDispatcherManager.SetMaintainerID(from) } response := &heartbeatpb.MaintainerBootstrapResponse{ - Statuses: make([]*heartbeatpb.TableSpanStatus, 0, eventDispatcherManager.GetDispatcherMap().Len()), + ChangefeedID: maintainerBootstrapRequest.ChangefeedID, + Statuses: make([]*heartbeatpb.TableSpanStatus, 0, eventDispatcherManager.GetDispatcherMap().Len()), } eventDispatcherManager.GetDispatcherMap().ForEach(func(tableSpan *common.TableSpan, tableEventDispatcher *dispatcher.TableEventDispatcher) { response.Statuses = append(response.Statuses, &heartbeatpb.TableSpanStatus{ diff --git a/pkg/messaging/router.go b/pkg/messaging/router.go index 3f0b758dc..b6f418ab3 100644 --- a/pkg/messaging/router.go +++ b/pkg/messaging/router.go @@ -54,7 +54,6 @@ func (r *router) runDispatch(ctx context.Context, wg *sync.WaitGroup, out <-chan err := handler(ctx, msg) if err != nil { log.Error("router: close, since handle message failed", zap.Error(err), zap.Any("msg", msg)) - return } } } From 07b444b03a27ec24918df5e7d005c915ac446963 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 11:18:26 +0800 Subject: [PATCH 15/29] fix metric error Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 4 ++-- pkg/eventservice/event_broker.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index c8d8d0691..056050914 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -272,16 +272,16 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T func (c *EventCollector) updateMetrics(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) defer ticker.Stop() - minResolvedTs := uint64(0) go func() { for { select { case <-ctx.Done(): return case <-ticker.C: + minResolvedTs := uint64(0) c.dispatcherMap.RLock() for _, d := range c.dispatcherMap.m { - if minResolvedTs == 0 || d.GetResolvedTs() < minResolvedTs { + if minResolvedTs == uint64(0) || d.GetResolvedTs() < minResolvedTs { minResolvedTs = d.GetResolvedTs() } } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index d59684a65..83df250a5 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -313,17 +313,17 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { func (c *eventBroker) updateMetrics(ctx context.Context) { ticker := time.NewTicker(time.Second * 5) defer ticker.Stop() - minResolvedTs := uint64(0) go func() { for { select { case <-ctx.Done(): return case <-ticker.C: + minResolvedTs := uint64(0) c.dispatchers.Range(func(key, value interface{}) bool { dispatcher := value.(*dispatcherStat) resolvedTs := dispatcher.spanSubscription.watermark.Load() - if minResolvedTs == 0 || resolvedTs < minResolvedTs { + if minResolvedTs == uint64(0) || resolvedTs < minResolvedTs { minResolvedTs = resolvedTs } return true From 7d06ea8f2d5f431192e681f4df45a876b816b5e5 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 14:46:11 +0800 Subject: [PATCH 16/29] add debug log Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 2 +- pkg/eventservice/event_broker.go | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 056050914..798d3b358 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -281,7 +281,7 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { minResolvedTs := uint64(0) c.dispatcherMap.RLock() for _, d := range c.dispatcherMap.m { - if minResolvedTs == uint64(0) || d.GetResolvedTs() < minResolvedTs { + if minResolvedTs == 0 || d.GetResolvedTs() < minResolvedTs { minResolvedTs = d.GetResolvedTs() } } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 83df250a5..50da050fd 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -320,14 +320,17 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { return case <-ticker.C: minResolvedTs := uint64(0) + iterCount := 0 c.dispatchers.Range(func(key, value interface{}) bool { dispatcher := value.(*dispatcherStat) resolvedTs := dispatcher.spanSubscription.watermark.Load() - if minResolvedTs == uint64(0) || resolvedTs < minResolvedTs { + if minResolvedTs == 0 || resolvedTs < minResolvedTs { minResolvedTs = resolvedTs } + iterCount++ return true }) + log.Info("update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Int("dispatcherCount", iterCount)) phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) From bbbe61500ed213c05478a6ef300c26a2f8e22553 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 15:09:33 +0800 Subject: [PATCH 17/29] fix metrics Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 3 +++ pkg/eventservice/event_broker.go | 5 +++++ 2 files changed, 8 insertions(+) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 798d3b358..03f8afcbe 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -285,6 +285,9 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { minResolvedTs = d.GetResolvedTs() } } + if minResolvedTs == 0 { + continue + } c.dispatcherMap.RUnlock() phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 50da050fd..76f885dd2 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -311,9 +311,11 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { } func (c *eventBroker) updateMetrics(ctx context.Context) { + c.wg.Add(1) ticker := time.NewTicker(time.Second * 5) defer ticker.Stop() go func() { + defer c.wg.Done() for { select { case <-ctx.Done(): @@ -330,6 +332,9 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { iterCount++ return true }) + if minResolvedTs == 0 { + continue + } log.Info("update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Int("dispatcherCount", iterCount)) phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 From b0e0f87107a108bf695e78a9feeffccbf49b52b0 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 15:36:14 +0800 Subject: [PATCH 18/29] fix metrics 2 Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 1 + pkg/eventservice/event_broker.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 03f8afcbe..1bd1b8ec2 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -285,6 +285,7 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { minResolvedTs = d.GetResolvedTs() } } + log.Info("fizz update resolved ts lag", zap.Uint64("minResolvedTs", minResolvedTs)) if minResolvedTs == 0 { continue } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 76f885dd2..412271ec0 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -332,10 +332,10 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { iterCount++ return true }) + log.Info("fizz update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Int("dispatcherCount", iterCount)) if minResolvedTs == 0 { continue } - log.Info("update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Int("dispatcherCount", iterCount)) phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) From 09aac0a66bf4b05619685124338e0326def32cd0 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 15:44:50 +0800 Subject: [PATCH 19/29] fix metrics 3 Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 412271ec0..91705960a 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -316,9 +316,11 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { defer ticker.Stop() go func() { defer c.wg.Done() + log.Info("fizz update metrics goroutine is started") for { select { case <-ctx.Done(): + log.Info("update metrics goroutine is closing") return case <-ticker.C: minResolvedTs := uint64(0) From cf028ba47c4f0264abb055589a6398f7b41eea9d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 15:52:58 +0800 Subject: [PATCH 20/29] fix metrics 4 Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 2 +- pkg/eventservice/event_broker.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index 1bd1b8ec2..a92f347b7 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -271,8 +271,8 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T func (c *EventCollector) updateMetrics(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) - defer ticker.Stop() go func() { + defer ticker.Stop() for { select { case <-ctx.Done(): diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 91705960a..aa5d51c4e 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -313,8 +313,8 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { func (c *eventBroker) updateMetrics(ctx context.Context) { c.wg.Add(1) ticker := time.NewTicker(time.Second * 5) - defer ticker.Stop() go func() { + defer ticker.Stop() defer c.wg.Done() log.Info("fizz update metrics goroutine is started") for { From 429fa6c6a495b1d079b156121754e9090e1a6e91 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 15:53:24 +0800 Subject: [PATCH 21/29] fix metrics 4 Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 1 - pkg/eventservice/event_broker.go | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index a92f347b7..ac31f33e2 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -285,7 +285,6 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { minResolvedTs = d.GetResolvedTs() } } - log.Info("fizz update resolved ts lag", zap.Uint64("minResolvedTs", minResolvedTs)) if minResolvedTs == 0 { continue } diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index aa5d51c4e..03f992c79 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -316,7 +316,7 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { go func() { defer ticker.Stop() defer c.wg.Done() - log.Info("fizz update metrics goroutine is started") + log.Info("update metrics goroutine is started") for { select { case <-ctx.Done(): @@ -334,7 +334,6 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { iterCount++ return true }) - log.Info("fizz update metrics", zap.Uint64("minResolvedTs", minResolvedTs), zap.Int("dispatcherCount", iterCount)) if minResolvedTs == 0 { continue } From 986ee331422d1c2f100da2274bbb00a799b4fd87 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 16:43:13 +0800 Subject: [PATCH 22/29] fix metrics 5 Signed-off-by: dongmen <414110582@qq.com> --- downstreamadapter/eventcollector/event_collector.go | 3 --- pkg/eventservice/event_broker.go | 1 - 2 files changed, 4 deletions(-) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index ac31f33e2..c836937e0 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -272,14 +272,12 @@ func (c *EventCollector) RecvEventsMessage(ctx context.Context, msg *messaging.T func (c *EventCollector) updateMetrics(ctx context.Context) error { ticker := time.NewTicker(10 * time.Second) go func() { - defer ticker.Stop() for { select { case <-ctx.Done(): return case <-ticker.C: minResolvedTs := uint64(0) - c.dispatcherMap.RLock() for _, d := range c.dispatcherMap.m { if minResolvedTs == 0 || d.GetResolvedTs() < minResolvedTs { minResolvedTs = d.GetResolvedTs() @@ -288,7 +286,6 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { if minResolvedTs == 0 { continue } - c.dispatcherMap.RUnlock() phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 c.metricResolvedTsLag.Set(float64(lag)) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 03f992c79..274375528 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -314,7 +314,6 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { c.wg.Add(1) ticker := time.NewTicker(time.Second * 5) go func() { - defer ticker.Stop() defer c.wg.Done() log.Info("update metrics goroutine is started") for { From cce86c52c5012738ff559fadc1fb97c2472a24b2 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Mon, 19 Aug 2024 17:36:21 +0800 Subject: [PATCH 23/29] add event service dispatcher watermark metric Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 56 ++++++++++++++++++-------------- pkg/metrics/eventService.go | 4 +-- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 274375528..9cd7ade09 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -51,9 +51,10 @@ type eventBroker struct { // cancel is used to cancel the goroutines spawned by the eventBroker. cancel context.CancelFunc - metricEventServiceResolvedTs prometheus.Gauge - metricEventServiceResolvedTsLag prometheus.Gauge - metricTaskInQueueDuration prometheus.Observer + metricEventServicePullerResolvedTs prometheus.Gauge + metricEventServiceDispatcherResolvedTs prometheus.Gauge + metricEventServiceResolvedTsLag prometheus.Gauge + metricTaskInQueueDuration prometheus.Observer } func newEventBroker( @@ -65,19 +66,20 @@ func newEventBroker( ctx, cancel := context.WithCancel(ctx) wg := &sync.WaitGroup{} c := &eventBroker{ - tidbClusterID: id, - eventStore: eventStore, - dispatchers: sync.Map{}, - msgSender: mc, - changedCh: make(chan *subscriptionChange, defaultChannelSize), - taskPool: newScanTaskPool(), - scanWorkerCount: defaultWorkerCount, - messageCh: make(chan *wrapMessage, defaultChannelSize), - cancel: cancel, - wg: wg, - metricEventServiceResolvedTs: metrics.EventServiceResolvedTsGauge, - metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge, - metricTaskInQueueDuration: metrics.EventServiceScanTaskInQueueDuration, + tidbClusterID: id, + eventStore: eventStore, + dispatchers: sync.Map{}, + msgSender: mc, + changedCh: make(chan *subscriptionChange, defaultChannelSize), + taskPool: newScanTaskPool(), + scanWorkerCount: defaultWorkerCount, + messageCh: make(chan *wrapMessage, defaultChannelSize), + cancel: cancel, + wg: wg, + metricEventServicePullerResolvedTs: metrics.EventServiceResolvedTsGauge, + metricEventServiceResolvedTsLag: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("puller"), + metricEventServiceDispatcherResolvedTs: metrics.EventServiceResolvedTsLagGauge.WithLabelValues("dispatcher"), + metricTaskInQueueDuration: metrics.EventServiceScanTaskInQueueDuration, } c.runGenerateScanTask(ctx) c.runScanWorker(ctx) @@ -322,24 +324,30 @@ func (c *eventBroker) updateMetrics(ctx context.Context) { log.Info("update metrics goroutine is closing") return case <-ticker.C: - minResolvedTs := uint64(0) - iterCount := 0 + pullerMinResolvedTs := uint64(0) + dispatcherMinWaterMark := uint64(0) c.dispatchers.Range(func(key, value interface{}) bool { dispatcher := value.(*dispatcherStat) resolvedTs := dispatcher.spanSubscription.watermark.Load() - if minResolvedTs == 0 || resolvedTs < minResolvedTs { - minResolvedTs = resolvedTs + if pullerMinResolvedTs == 0 || resolvedTs < pullerMinResolvedTs { + pullerMinResolvedTs = resolvedTs + } + watermark := dispatcher.watermark.Load() + if dispatcherMinWaterMark == 0 || watermark < dispatcherMinWaterMark { + dispatcherMinWaterMark = watermark } - iterCount++ return true }) - if minResolvedTs == 0 { + if pullerMinResolvedTs == 0 { continue } - phyResolvedTs := oracle.ExtractPhysical(minResolvedTs) + phyResolvedTs := oracle.ExtractPhysical(pullerMinResolvedTs) lag := (oracle.GetPhysical(time.Now()) - phyResolvedTs) / 1e3 - c.metricEventServiceResolvedTs.Set(float64(phyResolvedTs)) + c.metricEventServicePullerResolvedTs.Set(float64(phyResolvedTs)) c.metricEventServiceResolvedTsLag.Set(float64(lag)) + + lag = (oracle.GetPhysical(time.Now()) - oracle.ExtractPhysical(dispatcherMinWaterMark)) / 1e3 + c.metricEventServiceDispatcherResolvedTs.Set(float64(lag)) } } }() diff --git a/pkg/metrics/eventService.go b/pkg/metrics/eventService.go index 39e3467a4..0082b2ac3 100644 --- a/pkg/metrics/eventService.go +++ b/pkg/metrics/eventService.go @@ -49,13 +49,13 @@ var ( Name: "resolved_ts", Help: "resolved ts of eventService", }) - EventServiceResolvedTsLagGauge = prometheus.NewGauge( + EventServiceResolvedTsLagGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "event_service", Name: "resolved_ts_lag", Help: "resolved ts lag of eventService in seconds", - }) + }, []string{"type"}) EventServiceScanTaskInQueueDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "ticdc", From b7dc129a5e8130f782801dcb8764fa44e2dab301 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Mon, 19 Aug 2024 18:59:33 +0800 Subject: [PATCH 24/29] add panel --- metrics/grafana/ticdc.json | 1695 +++++++++++++++++++++++++++--------- pkg/metrics/dispatcher.go | 8 +- 2 files changed, 1268 insertions(+), 435 deletions(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index fe07fb38c..4268d3930 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -1,8 +1,8 @@ { "__inputs": [ { - "name": "DS_LOCAL", - "label": "local", + "name": "DS_C1", + "label": "c1", "description": "", "type": "datasource", "pluginId": "prometheus", @@ -66,7 +66,7 @@ "useValueForTime": false }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "enable": true, "expr": "max(ticdc_owner_checkpoint_ts_lag) by (changefeed, instance) > BOOL $spike_threshold", "hide": true, @@ -81,7 +81,7 @@ "useValueForTime": false }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "enable": false, "expr": "delta(up{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=~\"tikv|ticdc|tidb|pd\"}[30s]) < BOOL 0", "hide": false, @@ -97,7 +97,7 @@ "type": "tags" }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "enable": false, "expr": "sum(ALERTS{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", alertstate=\"firing\", alertname=~\"ticdc.*\"}) by (alertname) > BOOL 0", "hide": false, @@ -111,7 +111,7 @@ "type": "tags" }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "enable": false, "expr": "delta(tikv_cdc_region_resolve_status{status=\"resolved\"}[30s]) < BOOL -800", "hide": false, @@ -131,7 +131,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1723133060695, + "iteration": 1724063264282, "links": [], "panels": [ { @@ -143,6 +143,839 @@ "x": 0, "y": 0 }, + "id": 12013, + "panels": [ + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 1 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 12034, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_event_service_scan_task_in_queue_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Event service scan task queue duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 1 + }, + "hiddenSeries": false, + "id": 12029, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "ticdc_event_service_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}-resolvedts", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Event service resolved ts lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1162", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:1163", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 7 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 12035, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_dispatcher_received_event_lag_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Event collector receive event lag", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 7 + }, + "hiddenSeries": false, + "id": 12032, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "ticdc_event_collector_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-resolvedts", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Event collector resolved ts lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1162", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:1163", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 13 + }, + "hiddenSeries": false, + "id": 12026, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "ticdc_dispatcher_checkpoint_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-resolvedts", + "refId": "A" + }, + { + "exemplar": true, + "expr": "ticdc_dispatchermanager_checkpoint_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-resolvedts", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Dispatcher manager checkpoint ts lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1162", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:1163", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 13 + }, + "hiddenSeries": false, + "id": 12027, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "ticdc_dispatcher_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-resolvedts", + "refId": "A" + }, + { + "exemplar": true, + "expr": "ticdc_dispatchermanager_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-resolvedts", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Dispatcher manager resolved ts lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1162", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:1163", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 12025, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": false, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "max(ticdc_owner_checkpoint_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\"}) by (namespace, changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Maintainer checkpoint lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_C1}", + "description": "The lag between changefeed resolved ts and the latest ts of upstream TiDB.", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 17 + }, + "hiddenSeries": false, + "id": 12022, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": false, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "max(ticdc_owner_resolved_ts_lag{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namesapce=~\"$namespace\", changefeed=~\"$changefeed\"}) by (namespace,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{namespace}}-{{changefeed}}-resolvedts", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Maintainer resolved ts lag", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "TiGate Lag", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, "id": 10063, "panels": [ { @@ -150,7 +983,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "editable": true, @@ -168,7 +1001,7 @@ "h": 6, "w": 12, "x": 0, - "y": 1 + "y": 2 }, "hiddenSeries": false, "id": 10069, @@ -276,7 +1109,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -287,7 +1120,7 @@ "h": 6, "w": 12, "x": 12, - "y": 1 + "y": 2 }, "hiddenSeries": false, "id": 10071, @@ -380,7 +1213,7 @@ "h": 1, "w": 24, "x": 0, - "y": 1 + "y": 2 }, "id": 10043, "panels": [ @@ -389,7 +1222,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "editable": true, @@ -407,7 +1240,7 @@ "h": 6, "w": 12, "x": 0, - "y": 8 + "y": 3 }, "hiddenSeries": false, "id": 10049, @@ -476,22 +1309,22 @@ }, "yaxes": [ { - "format": "short", + "$$hashKey": "object:1202", + "format": "none", "label": null, "logBase": 1, "max": null, "min": "0", - "show": true, - "$$hashKey": "object:1202" + "show": true }, { + "$$hashKey": "object:1203", "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false, - "$$hashKey": "object:1203" + "show": false } ], "yaxis": { @@ -504,7 +1337,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "editable": true, @@ -522,7 +1355,7 @@ "h": 6, "w": 12, "x": 12, - "y": 8 + "y": 3 }, "hiddenSeries": false, "id": 10050, @@ -591,22 +1424,22 @@ }, "yaxes": [ { - "format": "short", + "$$hashKey": "object:1227", + "format": "none", "label": null, "logBase": 1, "max": null, "min": "0", - "show": true, - "$$hashKey": "object:1227" + "show": true }, { + "$$hashKey": "object:1228", "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false, - "$$hashKey": "object:1228" + "show": false } ], "yaxis": { @@ -619,7 +1452,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -630,7 +1463,7 @@ "h": 6, "w": 12, "x": 0, - "y": 14 + "y": 9 }, "hiddenSeries": false, "id": 10047, @@ -715,7 +1548,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -726,7 +1559,7 @@ "h": 6, "w": 12, "x": 12, - "y": 14 + "y": 9 }, "hiddenSeries": false, "id": 10052, @@ -813,7 +1646,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -824,7 +1657,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 15 }, "hiddenSeries": false, "id": 10054, @@ -927,7 +1760,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Duration of dispatcher creation", "fieldConfig": { "defaults": {}, @@ -939,7 +1772,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 15 }, "hiddenSeries": false, "id": 10056, @@ -1042,24 +1875,30 @@ }, { "aliasColors": {}, + "bars": false, "dashLength": 10, - "datasource": "${DS_LOCAL}", + "dashes": false, + "datasource": "${DS_C1}", "decimals": 1, "description": "the number of registered dispatchers in event collector", "editable": true, + "error": false, "fieldConfig": { "defaults": { "unit": "none" }, "overrides": [] }, + "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 26 + "y": 23 }, + "hiddenSeries": false, "id": 12006, "legend": { "alignAsTable": false, @@ -1083,27 +1922,32 @@ "alertThreshold": true }, "paceLength": 10, + "percentage": false, "pluginVersion": "7.5.17", "pointradius": 1, "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_dispatchermanager_event_collector_registered_dispatcher_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}) by (instance)", - "legendFormat": "{{instance}}", - "interval": "", "exemplar": true, + "expr": "sum(ticdc_dispatchermanager_event_collector_registered_dispatcher_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}) by (instance)", "format": "time_series", "hide": false, + "interval": "", "intervalFactor": 2, + "legendFormat": "{{instance}}", "refId": "A", "step": 10 } ], "thresholds": [], + "timeFrom": null, "timeRegions": [], + "timeShift": null, "title": "Event Collector Registered Dispatcher Count", "tooltip": { "msResolution": false, @@ -1121,39 +1965,28 @@ }, "yaxes": [ { - "format": "short", + "$$hashKey": "object:1257", + "format": "none", "label": null, "logBase": 1, "max": null, "min": "0", - "show": true, - "$$hashKey": "object:1257" + "show": true }, { + "$$hashKey": "object:1258", "format": "none", "label": null, "logBase": 1, - "max": null, - "min": null, - "show": false, - "$$hashKey": "object:1258" - } - ], - "yaxis": { - "align": false, - "alignLevel": null - }, - "bars": false, - "dashes": false, - "error": false, - "fill": 0, - "fillGradient": 0, - "hiddenSeries": false, - "percentage": false, - "stack": false, - "steppedLine": false, - "timeFrom": null, - "timeShift": null + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "title": "TiGate", @@ -1166,7 +1999,7 @@ "h": 1, "w": 24, "x": 0, - "y": 2 + "y": 3 }, "id": 397, "panels": [ @@ -1175,7 +2008,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -1189,7 +2022,7 @@ "h": 4, "w": 12, "x": 0, - "y": 3 + "y": 8 }, "hiddenSeries": false, "id": 398, @@ -1278,7 +2111,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed resolved ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -1292,7 +2125,7 @@ "h": 4, "w": 12, "x": 12, - "y": 3 + "y": 8 }, "hiddenSeries": false, "id": 468, @@ -1381,7 +2214,7 @@ "bars": true, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of errors that interrupt Eventfeed RPC", "fieldConfig": { "defaults": { @@ -1395,7 +2228,7 @@ "h": 4, "w": 12, "x": 0, - "y": 7 + "y": 12 }, "hiddenSeries": false, "id": 469, @@ -1486,7 +2319,7 @@ "bars": true, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of PD scheduling operator.", "fieldConfig": { "defaults": { @@ -1500,7 +2333,7 @@ "h": 4, "w": 12, "x": 12, - "y": 7 + "y": 12 }, "hiddenSeries": false, "id": 466, @@ -1591,7 +2424,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "99.9% of TiDB query durations.", "fieldConfig": { "defaults": {}, @@ -1603,7 +2436,7 @@ "h": 4, "w": 12, "x": 0, - "y": 11 + "y": 16 }, "hiddenSeries": false, "id": 479, @@ -1691,7 +2524,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The min resolved ts lag of each TiKV", "editable": true, @@ -1707,7 +2540,7 @@ "h": 4, "w": 12, "x": 12, - "y": 11 + "y": 16 }, "hiddenSeries": false, "id": 473, @@ -1823,7 +2656,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of changed rows that are written to downstream per second", "fieldConfig": { "defaults": { @@ -1837,7 +2670,7 @@ "h": 4, "w": 12, "x": 0, - "y": 15 + "y": 20 }, "hiddenSeries": false, "id": 486, @@ -1909,35 +2742,35 @@ }, "yaxes": [ { + "$$hashKey": "object:1127", "format": "bytes", "label": null, "logBase": 1, "max": null, "min": null, - "show": true, - "$$hashKey": "object:1127" + "show": true }, { + "$$hashKey": "object:1128", "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true, - "$$hashKey": "object:1128" + "show": true } ], "yaxis": { "align": false, "alignLevel": null - }, + } }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "99.9% of TiCDC sink write durations.", "fieldConfig": { "defaults": {}, @@ -1949,7 +2782,7 @@ "h": 4, "w": 12, "x": 12, - "y": 15 + "y": 20 }, "hiddenSeries": false, "id": 485, @@ -2048,7 +2881,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of incremental scan tasks per minute", "fieldConfig": { @@ -2061,7 +2894,7 @@ "h": 4, "w": 12, "x": 0, - "y": 19 + "y": 24 }, "hiddenSeries": false, "id": 401, @@ -2154,7 +2987,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total time of incremental scan region takes per minute", "editable": true, @@ -2170,7 +3003,7 @@ "h": 4, "w": 12, "x": 12, - "y": 19 + "y": 24 }, "hiddenSeries": false, "id": 467, @@ -2263,7 +3096,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of leaders on each TiKV instance", "editable": true, @@ -2279,7 +3112,7 @@ "h": 4, "w": 12, "x": 0, - "y": 23 + "y": 28 }, "hiddenSeries": false, "id": 407, @@ -2371,7 +3204,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of the processed TiKV admin command", "editable": true, @@ -2387,7 +3220,7 @@ "h": 4, "w": 12, "x": 12, - "y": 23 + "y": 28 }, "hiddenSeries": false, "id": 470, @@ -2480,7 +3313,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The rate of TiKV advancing resolved ts.", "fieldConfig": { @@ -2493,7 +3326,7 @@ "h": 4, "w": 12, "x": 0, - "y": 27 + "y": 32 }, "hiddenSeries": false, "id": 471, @@ -2586,7 +3419,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of unresolved region per TiKV", "editable": true, @@ -2602,7 +3435,7 @@ "h": 4, "w": 12, "x": 12, - "y": 27 + "y": 32 }, "hiddenSeries": false, "id": 400, @@ -2695,7 +3528,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "99.99% of the number of regions that TiKV checks. ", "fieldConfig": { @@ -2708,7 +3541,7 @@ "h": 4, "w": 12, "x": 0, - "y": 31 + "y": 36 }, "hiddenSeries": false, "id": 433, @@ -2800,7 +3633,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of failed count of advancing resolved ts.", "fieldConfig": { @@ -2813,7 +3646,7 @@ "h": 4, "w": 12, "x": 12, - "y": 31 + "y": 36 }, "hiddenSeries": false, "id": 427, @@ -2906,7 +3739,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of tables rebalance on each TiCDC instance", "editable": true, @@ -2922,7 +3755,7 @@ "h": 4, "w": 12, "x": 0, - "y": 35 + "y": 40 }, "hiddenSeries": false, "id": 399, @@ -3014,7 +3847,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "99.99% of the duration that TiKV check leader takes. ", "fieldConfig": { @@ -3027,7 +3860,7 @@ "h": 4, "w": 12, "x": 12, - "y": 35 + "y": 40 }, "hiddenSeries": false, "id": 472, @@ -3126,7 +3959,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between the commit ts of the events and the time when they are flushed to downstream. ", "fieldConfig": { "defaults": {}, @@ -3138,7 +3971,7 @@ "h": 4, "w": 12, "x": 0, - "y": 39 + "y": 44 }, "hiddenSeries": false, "id": 727, @@ -3242,7 +4075,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of regions that take a long time(more than 60s) to scan", "editable": true, @@ -3258,7 +4091,7 @@ "h": 4, "w": 12, "x": 12, - "y": 39 + "y": 44 }, "hiddenSeries": false, "id": 10038, @@ -3361,7 +4194,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 4 }, "id": 21, "panels": [ @@ -3370,7 +4203,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Uptime of TiCDC and TiKV", "fieldConfig": { "defaults": { @@ -3384,7 +4217,7 @@ "h": 7, "w": 8, "x": 0, - "y": 30 + "y": 11 }, "hiddenSeries": false, "id": 19, @@ -3485,7 +4318,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Goroutine count of TiCDC", "fieldConfig": { "defaults": { @@ -3499,7 +4332,7 @@ "h": 7, "w": 6, "x": 8, - "y": 30 + "y": 11 }, "hiddenSeries": false, "id": 26, @@ -3594,7 +4427,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The count of open FD count of TiCDC", "fieldConfig": { "defaults": { @@ -3608,7 +4441,7 @@ "h": 7, "w": 6, "x": 14, - "y": 30 + "y": 11 }, "hiddenSeries": false, "id": 27, @@ -3691,7 +4524,7 @@ } }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "TiCDC cluster ownership status", "fieldConfig": { "defaults": { @@ -3804,7 +4637,7 @@ "h": 7, "w": 4, "x": 20, - "y": 30 + "y": 11 }, "hideTimeOverride": true, "id": 113, @@ -3842,7 +4675,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "CPU usage of TiCDC", "fieldConfig": { "defaults": { @@ -3856,7 +4689,7 @@ "h": 7, "w": 6, "x": 0, - "y": 37 + "y": 18 }, "hiddenSeries": false, "id": 24, @@ -3958,7 +4791,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Memory usage of TiCDC", "fieldConfig": { "defaults": { @@ -3972,7 +4805,7 @@ "h": 7, "w": 6, "x": 6, - "y": 37 + "y": 18 }, "hiddenSeries": false, "id": 23, @@ -4067,7 +4900,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The history of TiCDC cluster ownership, owner node has a value that is great than 0", "fieldConfig": { "defaults": {}, @@ -4079,7 +4912,7 @@ "h": 7, "w": 6, "x": 12, - "y": 37 + "y": 18 }, "hiddenSeries": false, "id": 110, @@ -4169,7 +5002,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The history of PD cluster leadership, leader node has a value that is great than 0", "fieldConfig": { "defaults": {}, @@ -4181,7 +5014,7 @@ "h": 7, "w": 6, "x": 18, - "y": 37 + "y": 18 }, "hiddenSeries": false, "id": 293, @@ -4275,12 +5108,12 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 5 }, "id": 11, "panels": [ { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of captured table of TiCDC nodes ", "fieldConfig": { "defaults": { @@ -4347,7 +5180,7 @@ "h": 7, "w": 6, "x": 0, - "y": 31 + "y": 24 }, "id": 4, "links": [], @@ -4387,7 +5220,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of tables in different capture.", "editable": true, @@ -4405,7 +5238,7 @@ "h": 7, "w": 6, "x": 6, - "y": 31 + "y": 24 }, "hiddenSeries": false, "id": 642, @@ -4501,7 +5334,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "unit": "dateTimeAsIso" @@ -4514,7 +5347,7 @@ "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 24 }, "hiddenSeries": false, "id": 10037, @@ -4607,7 +5440,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The checkpoint ts of changefeeds.", "fieldConfig": { "defaults": { @@ -4621,7 +5454,7 @@ "h": 7, "w": 12, "x": 0, - "y": 38 + "y": 31 }, "hiddenSeries": false, "id": 86, @@ -4735,7 +5568,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The resolved ts of changefeeds.", "fieldConfig": { "defaults": {}, @@ -4747,7 +5580,7 @@ "h": 7, "w": 12, "x": 12, - "y": 38 + "y": 31 }, "hiddenSeries": false, "id": 512, @@ -4860,7 +5693,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -4874,7 +5707,7 @@ "h": 7, "w": 12, "x": 0, - "y": 45 + "y": 38 }, "hiddenSeries": false, "id": 3, @@ -4963,7 +5796,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed resolved ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -4977,7 +5810,7 @@ "h": 7, "w": 12, "x": 12, - "y": 45 + "y": 38 }, "hiddenSeries": false, "id": 513, @@ -5074,7 +5907,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The difference between changefeed resolved ts and changefeed checkpointTs.", "fieldConfig": { "defaults": { @@ -5088,7 +5921,7 @@ "h": 7, "w": 12, "x": 0, - "y": 52 + "y": 45 }, "hiddenSeries": false, "id": 603, @@ -5177,7 +6010,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "each changefeed tick periodically, the time cost of each tick", "fieldConfig": { "defaults": {}, @@ -5189,7 +6022,7 @@ "h": 7, "w": 12, "x": 12, - "y": 52 + "y": 45 }, "hiddenSeries": false, "id": 597, @@ -5279,7 +6112,7 @@ } }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The status of each changefeed.\n\n0: Normal\n\n1 and 6: Warning\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", "fieldConfig": { "defaults": { @@ -5416,7 +6249,7 @@ "h": 7, "w": 12, "x": 0, - "y": 59 + "y": 52 }, "id": 163, "links": [], @@ -5457,7 +6290,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -5469,7 +6302,7 @@ "h": 7, "w": 12, "x": 12, - "y": 59 + "y": 52 }, "hiddenSeries": false, "id": 514, @@ -5574,7 +6407,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Request count of etcd operation per second", "fieldConfig": { "defaults": { @@ -5588,7 +6421,7 @@ "h": 7, "w": 12, "x": 0, - "y": 66 + "y": 59 }, "hiddenSeries": false, "id": 102, @@ -5679,7 +6512,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of errors that interrupt changefeed per minute ", "fieldConfig": { "defaults": {}, @@ -5691,7 +6524,7 @@ "h": 7, "w": 12, "x": 12, - "y": 66 + "y": 59 }, "hiddenSeries": false, "id": 82, @@ -5784,7 +6617,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 206, "panels": [ @@ -5793,7 +6626,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that puller outputs to sorter \n per second", "fieldConfig": { "defaults": { @@ -5807,7 +6640,7 @@ "h": 4, "w": 12, "x": 0, - "y": 13 + "y": 6 }, "hiddenSeries": false, "id": 218, @@ -5898,7 +6731,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total number of events that puller outputs", "fieldConfig": { "defaults": { @@ -5912,7 +6745,7 @@ "h": 4, "w": 12, "x": 12, - "y": 13 + "y": 6 }, "hiddenSeries": false, "id": 229, @@ -6005,7 +6838,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that mounter outputs to sink per second", "fieldConfig": { "defaults": { @@ -6019,7 +6852,7 @@ "h": 4, "w": 12, "x": 0, - "y": 17 + "y": 10 }, "hiddenSeries": false, "id": 219, @@ -6110,7 +6943,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total number of events that mounter outputs", "fieldConfig": { "defaults": { @@ -6124,7 +6957,7 @@ "h": 4, "w": 12, "x": 12, - "y": 17 + "y": 10 }, "hiddenSeries": false, "id": 224, @@ -6215,7 +7048,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that sorter outputs to mounter(or table sink worker) per second", "fieldConfig": { "defaults": { @@ -6229,7 +7062,7 @@ "h": 4, "w": 12, "x": 0, - "y": 21 + "y": 14 }, "hiddenSeries": false, "id": 228, @@ -6320,7 +7153,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that sorter outputs to mounter(or table sink worker) per second", "fieldConfig": { "defaults": { @@ -6334,7 +7167,7 @@ "h": 4, "w": 12, "x": 12, - "y": 21 + "y": 14 }, "hiddenSeries": false, "id": 10040, @@ -6425,7 +7258,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total number of events that EventService outputs", "fieldConfig": { "defaults": { @@ -6439,7 +7272,7 @@ "h": 4, "w": 12, "x": 0, - "y": 25 + "y": 18 }, "hiddenSeries": false, "id": 220, @@ -6530,7 +7363,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The p99 send event duration of EventService", "fieldConfig": { "defaults": { @@ -6544,7 +7377,7 @@ "h": 4, "w": 12, "x": 12, - "y": 25 + "y": 18 }, "hiddenSeries": false, "id": 10041, @@ -6635,7 +7468,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that messageCenter send /s", "fieldConfig": { "defaults": { @@ -6649,7 +7482,7 @@ "h": 4, "w": 12, "x": 0, - "y": 29 + "y": 22 }, "hiddenSeries": false, "id": 223, @@ -6748,7 +7581,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total number of events that messageCenter send", "fieldConfig": { "defaults": { @@ -6762,7 +7595,7 @@ "h": 4, "w": 12, "x": 12, - "y": 29 + "y": 22 }, "hiddenSeries": false, "id": 221, @@ -6848,8 +7681,10 @@ }, { "aliasColors": {}, + "bars": false, "dashLength": 10, - "datasource": "${DS_LOCAL}", + "dashes": false, + "datasource": "${DS_C1}", "description": "The number of rows that event collector received per second.", "fieldConfig": { "defaults": { @@ -6858,12 +7693,14 @@ "overrides": [] }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 0, - "y": 33 + "y": 26 }, + "hiddenSeries": false, "id": 12000, "legend": { "alignAsTable": true, @@ -6885,24 +7722,30 @@ "options": { "alertThreshold": true }, + "percentage": false, "pluginVersion": "7.5.17", "pointradius": 2, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_dispatcher_received_event_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}[1m])) by (instance, type)", - "legendFormat": "{{instance}}-{{type}}", - "interval": "", "exemplar": true, + "expr": "sum(rate(ticdc_dispatcher_received_event_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$ticdc_instance\"}[1m])) by (instance, type)", "format": "time_series", + "interval": "", "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], "thresholds": [], + "timeFrom": null, "timeRegions": [], + "timeShift": null, "title": "eventCollector received event/s", "tooltip": { "shared": true, @@ -6940,22 +7783,14 @@ "yaxis": { "align": false, "alignLevel": null - }, - "bars": false, - "dashes": false, - "fillGradient": 0, - "hiddenSeries": false, - "percentage": false, - "points": false, - "stack": false, - "steppedLine": false, - "timeFrom": null, - "timeShift": null + } }, { "aliasColors": {}, + "bars": false, "dashLength": 10, - "datasource": "${DS_LOCAL}", + "dashes": false, + "datasource": "${DS_C1}", "description": "The total number of events that Event Collector received", "fieldConfig": { "defaults": { @@ -6964,12 +7799,14 @@ "overrides": [] }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 12, - "y": 33 + "y": 26 }, + "hiddenSeries": false, "id": 10064, "legend": { "alignAsTable": true, @@ -6990,24 +7827,30 @@ "alertThreshold": true }, "paceLength": 10, + "percentage": false, "pluginVersion": "7.5.17", "pointradius": 2, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_dispatcher_received_event_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\",changefeed=~\"$changefeed\", instance=~\"$ticdc_instance\"}) by (namespace,changefeed, instance,type)", - "legendFormat": "{{instance}}-{{type}}", - "interval": "", "exemplar": true, + "expr": "sum(ticdc_dispatcher_received_event_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\",changefeed=~\"$changefeed\", instance=~\"$ticdc_instance\"}) by (namespace,changefeed, instance,type)", "format": "time_series", + "interval": "", "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], "thresholds": [], + "timeFrom": null, "timeRegions": [], + "timeShift": null, "title": "Event collector Received events", "tooltip": { "shared": true, @@ -7043,24 +7886,14 @@ "yaxis": { "align": false, "alignLevel": null - }, - "bars": false, - "dashes": false, - "fillGradient": 0, - "hiddenSeries": false, - "percentage": false, - "points": false, - "stack": false, - "steppedLine": false, - "timeFrom": null, - "timeShift": null + } }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of rows that sink flushes to downstream per second.", "fieldConfig": { "defaults": { @@ -7074,7 +7907,7 @@ "h": 4, "w": 12, "x": 0, - "y": 37 + "y": 30 }, "hiddenSeries": false, "id": 664, @@ -7166,7 +7999,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of rows(events) that are flushed by sink.", "fieldConfig": { "defaults": { @@ -7180,7 +8013,7 @@ "h": 4, "w": 12, "x": 12, - "y": 37 + "y": 30 }, "hiddenSeries": false, "id": 665, @@ -7276,7 +8109,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 608, "panels": [ @@ -7285,7 +8118,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Changefeed memory quota", "fieldConfig": { "defaults": {}, @@ -7388,7 +8221,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Ratio of redo event cache hit", "fieldConfig": { "defaults": {}, @@ -7484,7 +8317,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Row count for batch to the downstream sink.", "fieldConfig": { "defaults": {}, @@ -7590,7 +8423,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Row count for total output rows.", "fieldConfig": { "defaults": {}, @@ -7688,7 +8521,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "DDL count and executing duration", "fieldConfig": { "defaults": {}, @@ -7799,7 +8632,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Size of large rows (size >= 2K).", "fieldConfig": { "defaults": {}, @@ -7905,7 +8738,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Count of errors in the last minute.", "fieldConfig": { "defaults": {}, @@ -8009,7 +8842,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 8 }, "id": 616, "panels": [ @@ -8018,7 +8851,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Duration of event staying in conflict detector", "fieldConfig": { "defaults": {}, @@ -8140,7 +8973,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Full flush (backend flush + callback + conflict detector notify) duration", "fieldConfig": { "defaults": {}, @@ -8246,7 +9079,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Sink worker busy ratio", "fieldConfig": { "defaults": {}, @@ -8344,7 +9177,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -8441,7 +9274,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Distribution of flush transaction duration to backend", "fieldConfig": { "defaults": {}, @@ -8552,7 +9385,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 9 }, "id": 623, "panels": [ @@ -8561,7 +9394,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker send messages to Kafka, this metric record the time cost on send every message.", "fieldConfig": { "defaults": {}, @@ -8667,7 +9500,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Bytes/second written off all brokers.\nvalue = one-minute moving average rate of Bytes per second", "fieldConfig": { "defaults": { @@ -8770,7 +9603,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The current number of in-flight requests awaiting a response for all brokers.", "fieldConfig": { "defaults": { @@ -8873,7 +9706,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The request latency in ms for all brokers.\n\nvalue = request latency histogram's mean", "fieldConfig": { "defaults": { @@ -8976,7 +9809,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Requests/second sent to all brokers.\nvalue = one-minute moving average rate of events per second", "fieldConfig": { "defaults": { @@ -9079,7 +9912,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Records count per request send to the kafka\nvalue = one-minute moving average of response receive rate", "fieldConfig": { "defaults": { @@ -9182,7 +10015,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The compression ratio times 100 of record batches for all topics. Compression ratio = Size of original data / Size of compressed data * 100", "fieldConfig": { "defaults": {}, @@ -9283,7 +10116,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -9381,7 +10214,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -9479,7 +10312,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker batch multiple messages into one when using batched encode protocol, this metric record the time cost on batch messages.", "fieldConfig": { "defaults": {}, @@ -9585,7 +10418,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker batch multiple messages into one when using batched encode protocol, this metrics track each batch's size", "fieldConfig": { "defaults": {}, @@ -9692,7 +10525,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Kafka-Go Writer batch messages internally before send to the Kafka cluster.", "fieldConfig": { "defaults": {}, @@ -9790,7 +10623,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker batch multiple messages into one when using batched encode protocol, this metric record the time cost on batch messages.", "fieldConfig": { "defaults": {}, @@ -9888,7 +10721,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Kafka-Go Writer batch messages internally before send to the Kafka cluster.", "fieldConfig": { "defaults": {}, @@ -9986,7 +10819,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Count of errors in the last minute.", "fieldConfig": { "defaults": {}, @@ -10084,7 +10917,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Count of errors in the last minute.", "fieldConfig": { "defaults": {}, @@ -10182,7 +11015,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker send large message to the external storage, this metric record the time cost on send every message.", "fieldConfig": { "defaults": {}, @@ -10288,7 +11121,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "MQ worker send large message to the external storage, this metrics record the message count", "fieldConfig": { "defaults": {}, @@ -10392,7 +11225,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 10 }, "id": 713, "panels": [ @@ -10401,7 +11234,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -10497,7 +11330,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -10602,7 +11435,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of write storage by a cloud storage sink", "fieldConfig": { "defaults": {}, @@ -10678,7 +11511,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of flush storage by a cloud storage sink", "fieldConfig": { "defaults": {}, @@ -10744,7 +11577,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Busy ratio (X ms in 1s) for cloud storage sink dml worker", "fieldConfig": { "defaults": {}, @@ -10847,7 +11680,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 11 }, "id": 528, "panels": [ @@ -10856,7 +11689,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -10870,7 +11703,7 @@ "h": 4, "w": 12, "x": 0, - "y": 37 + "y": 16 }, "hiddenSeries": false, "id": 571, @@ -10959,7 +11792,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The lag between changefeed resolved ts and the latest ts of upstream TiDB.", "fieldConfig": { "defaults": { @@ -10973,7 +11806,7 @@ "h": 4, "w": 12, "x": 12, - "y": 37 + "y": 16 }, "hiddenSeries": false, "id": 589, @@ -11062,7 +11895,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total number of tables in different replication states.\n\n0: ReplicationSetStateUnknown means the replication state is unknown, it should not happen.\n\n1: ReplicationSetStateAbsent means there is no one replicates or prepares it.\n\n2: ReplicationSetStatePrepare means it needs to add a secondary.\n\n3: ReplicationSetStateCommit means it needs to promote secondary to primary.\n\n4: ReplicationSetStateReplicating means there is exactly one capture that is replicating the table.\n\n5: ReplicationSetStateRemoving means all captures need to stop replication eventually.\n\n", "editable": true, @@ -11080,7 +11913,7 @@ "h": 6, "w": 12, "x": 0, - "y": 41 + "y": 20 }, "hiddenSeries": false, "id": 546, @@ -11175,7 +12008,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total number of accepted schedule tasks.", "editable": true, @@ -11193,7 +12026,7 @@ "h": 6, "w": 12, "x": 12, - "y": 41 + "y": 20 }, "hiddenSeries": false, "id": 549, @@ -11288,7 +12121,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total number of different schedule tasks.", "editable": true, @@ -11306,7 +12139,7 @@ "h": 6, "w": 12, "x": 0, - "y": 47 + "y": 26 }, "hiddenSeries": false, "id": 548, @@ -11401,7 +12234,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total number of running schedule tasks.", "editable": true, @@ -11419,7 +12252,7 @@ "h": 6, "w": 12, "x": 12, - "y": 47 + "y": 26 }, "hiddenSeries": false, "id": 588, @@ -11514,7 +12347,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total number of tables", "fieldConfig": { "defaults": { @@ -11528,7 +12361,7 @@ "h": 6, "w": 12, "x": 0, - "y": 53 + "y": 32 }, "hiddenSeries": false, "id": 545, @@ -11617,7 +12450,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of tables in different capture.", "editable": true, @@ -11635,7 +12468,7 @@ "h": 6, "w": 12, "x": 12, - "y": 53 + "y": 32 }, "hiddenSeries": false, "id": 547, @@ -11731,7 +12564,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The ID of the slowest table", "editable": true, @@ -11747,7 +12580,7 @@ "h": 6, "w": 12, "x": 0, - "y": 59 + "y": 38 }, "hiddenSeries": false, "id": 543, @@ -11842,7 +12675,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The state of the slowest table.\n\n0: ReplicationSetStateUnknown means the replication state is unknown, it should not happen.\n\n1: ReplicationSetStateAbsent means there is no one replicates or prepares it.\n\n2: ReplicationSetStatePrepare means it needs to add a secondary.\n\n3: ReplicationSetStateCommit means it needs to promote secondary to primary.\n\n4: ReplicationSetStateReplicating means there is exactly one capture that is replicating the table.\n\n5: ReplicationSetStateRemoving means all captures need to stop replication eventually.\n\n", "editable": true, @@ -11860,7 +12693,7 @@ "h": 6, "w": 12, "x": 12, - "y": 59 + "y": 38 }, "hiddenSeries": false, "id": 544, @@ -11956,7 +12789,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The checkpoint ts of the slowest table.", "fieldConfig": { "defaults": { @@ -11970,7 +12803,7 @@ "h": 6, "w": 12, "x": 0, - "y": 65 + "y": 44 }, "hiddenSeries": false, "id": 541, @@ -12077,7 +12910,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The resolved ts of the slowest table.", "fieldConfig": { "defaults": {}, @@ -12089,7 +12922,7 @@ "h": 6, "w": 12, "x": 12, - "y": 65 + "y": 44 }, "hiddenSeries": false, "id": 590, @@ -12195,7 +13028,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of incremental scan tasks per minute", "fieldConfig": { @@ -12208,7 +13041,7 @@ "h": 6, "w": 12, "x": 0, - "y": 71 + "y": 50 }, "hiddenSeries": false, "id": 560, @@ -12309,7 +13142,7 @@ "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 12 }, "id": 677, "panels": [ @@ -12318,7 +13151,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The ID of the slowest table", "editable": true, @@ -12429,7 +13262,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of regions captured by the slowest table.", "editable": true, @@ -12541,7 +13374,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Checkpoint ts of the slowest table each stage.", "fieldConfig": { "defaults": { @@ -12653,7 +13486,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Resolved ts of the slowest table each stage.", "fieldConfig": { "defaults": { @@ -12764,7 +13597,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The details of changefeed lag including resolved ts and checkpoint ts in different stages.", "fieldConfig": { "defaults": { @@ -12883,7 +13716,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Lag distribution of the slowest table in different stages.", "fieldConfig": { "defaults": {}, @@ -12995,7 +13828,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 13 }, "id": 269, "panels": [ @@ -13004,7 +13837,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13015,7 +13848,7 @@ "h": 6, "w": 8, "x": 0, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 271, @@ -13035,7 +13868,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13098,7 +13931,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13109,7 +13942,7 @@ "h": 6, "w": 8, "x": 8, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 273, @@ -13130,7 +13963,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13205,7 +14038,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The count of files of each levels", "fieldConfig": { "defaults": {}, @@ -13217,7 +14050,7 @@ "h": 6, "w": 8, "x": 16, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 272, @@ -13239,7 +14072,7 @@ }, "paceLength": 10, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13311,7 +14144,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter write", "fieldConfig": { "defaults": {}, @@ -13321,7 +14154,7 @@ "h": 6, "w": 8, "x": 0, - "y": 17 + "y": 20 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13386,7 +14219,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13397,7 +14230,7 @@ "h": 6, "w": 8, "x": 8, - "y": 17 + "y": 20 }, "hiddenSeries": false, "id": 288, @@ -13420,7 +14253,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13511,7 +14344,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "CPU usage of LevelDB sorter", "fieldConfig": { "defaults": { @@ -13525,7 +14358,7 @@ "h": 6, "w": 8, "x": 16, - "y": 17 + "y": 20 }, "hiddenSeries": false, "id": 286, @@ -13550,7 +14383,7 @@ }, "paceLength": 10, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13627,7 +14460,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter write", "fieldConfig": { "defaults": {}, @@ -13637,7 +14470,7 @@ "h": 6, "w": 8, "x": 0, - "y": 23 + "y": 26 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13702,7 +14535,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13713,7 +14546,7 @@ "h": 6, "w": 8, "x": 8, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 279, @@ -13736,7 +14569,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13802,7 +14635,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The count and duration of write delay", "fieldConfig": { "defaults": { @@ -13816,7 +14649,7 @@ "h": 6, "w": 8, "x": 16, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 275, @@ -13840,7 +14673,7 @@ }, "paceLength": 10, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -13929,7 +14762,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter iterator read", "fieldConfig": { "defaults": {}, @@ -13939,7 +14772,7 @@ "h": 6, "w": 8, "x": 0, - "y": 29 + "y": 32 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14013,7 +14846,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter iterator read", "fieldConfig": { "defaults": {}, @@ -14023,7 +14856,7 @@ "h": 6, "w": 8, "x": 8, - "y": 29 + "y": 32 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14097,7 +14930,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter iterator read", "fieldConfig": { "defaults": {}, @@ -14107,7 +14940,7 @@ "h": 6, "w": 8, "x": 16, - "y": 29 + "y": 32 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14172,7 +15005,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -14183,7 +15016,7 @@ "h": 6, "w": 8, "x": 0, - "y": 35 + "y": 38 }, "hiddenSeries": false, "id": 280, @@ -14206,7 +15039,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -14272,7 +15105,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -14283,7 +15116,7 @@ "h": 6, "w": 8, "x": 8, - "y": 35 + "y": 38 }, "hiddenSeries": false, "id": 283, @@ -14306,7 +15139,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -14372,7 +15205,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -14383,7 +15216,7 @@ "h": 6, "w": 8, "x": 16, - "y": 35 + "y": 38 }, "hiddenSeries": false, "id": 287, @@ -14406,7 +15239,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -14481,7 +15314,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time of sorter compact", "fieldConfig": { "defaults": {}, @@ -14491,7 +15324,7 @@ "h": 6, "w": 8, "x": 0, - "y": 41 + "y": 44 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14562,7 +15395,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 14 }, "id": 447, "panels": [ @@ -14571,7 +15404,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of established Eventfeed RPC between TiCDC and TiKV", "fieldConfig": { "defaults": { @@ -14703,7 +15536,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of KV client received events from TiKV per seconds", "fieldConfig": { "defaults": { @@ -14810,7 +15643,7 @@ "bars": true, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of errors that interrupt Eventfeed RPC", "fieldConfig": { "defaults": { @@ -14932,7 +15765,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of KV client dispatched event per second", "fieldConfig": { "defaults": { @@ -15049,7 +15882,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Active stream count of each gRPC connection", "fieldConfig": { "defaults": { @@ -15152,7 +15985,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": null, "description": "Percentiles of Eventfeed message size", "fieldConfig": { @@ -15261,7 +16094,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of regions that are being scanned", "fieldConfig": { "defaults": { @@ -15366,7 +16199,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The count of regoin cache operation initiated by TiCDC", "fieldConfig": { @@ -15481,7 +16314,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The size of batch resolved regions count", "fieldConfig": { "defaults": {}, @@ -15554,7 +16387,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of regions that have not connected to TiKV", "fieldConfig": { "defaults": { @@ -15659,7 +16492,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The count of resolve lock RPC initiated by TiCDC", "fieldConfig": { @@ -15765,7 +16598,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "fieldConfig": { @@ -15873,7 +16706,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The number of events that puller outputs to sorter \n per second", "fieldConfig": { "defaults": { @@ -15976,7 +16809,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Estimate the remaining time for a changefeed initialization (on a specific capture)", "fieldConfig": { "defaults": { @@ -16090,7 +16923,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Duration of sorting DDL events", "fieldConfig": { "defaults": {}, @@ -16172,7 +17005,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Duration of merging sorted events", "fieldConfig": { "defaults": {}, @@ -16243,7 +17076,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Percentiles of sorting events duration", "fieldConfig": { "defaults": { @@ -16357,7 +17190,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Percentiles of merging sorted events duration", "fieldConfig": { "defaults": { @@ -16471,7 +17304,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "batch read events from the sorter, record each batch size ", "fieldConfig": { "defaults": {}, @@ -16569,7 +17402,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "batch read events from the sorter, record each batch read duration ", "fieldConfig": { "defaults": {}, @@ -16667,7 +17500,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -16765,7 +17598,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The count of regions that initialize slow. You can search the log [event feed initializes a region too slow] to get slow region id", "fieldConfig": { "defaults": {}, @@ -16871,7 +17704,7 @@ "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 15 }, "id": 266, "panels": [ @@ -16880,7 +17713,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -16980,7 +17813,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "links": [] @@ -17105,7 +17938,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -17171,7 +18004,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -17285,7 +18118,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -17351,7 +18184,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": { @@ -17470,7 +18303,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -17534,7 +18367,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": {}, @@ -17638,7 +18471,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time consumed of writing WAL into the persistent storage in .99", "editable": true, "error": false, @@ -17746,7 +18579,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time consumed of handling etcd transactions in .99", "editable": true, "error": false, @@ -17857,7 +18690,7 @@ "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 16 }, "id": 58, "panels": [ @@ -17866,7 +18699,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "CPU usage of TiKV CDC component", "fieldConfig": { @@ -17999,7 +18832,7 @@ "bars": true, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "Outbound network traffic of TiKV CDC component", "editable": true, @@ -18120,7 +18953,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The count of different kinds of gRPC message", "fieldConfig": { @@ -18226,7 +19059,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The memory usage per TiKV instance", "editable": true, @@ -18400,7 +19233,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The min resolved ts of each TiKV", "editable": true, @@ -18541,7 +19374,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The ID of the min resolved region of each TiKV", "editable": true, @@ -18652,7 +19485,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "fieldConfig": { @@ -18766,7 +19599,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time consumed to CDC incremental scan", "fieldConfig": { "defaults": {}, @@ -18840,7 +19673,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "fieldConfig": { @@ -18945,7 +19778,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The number of incremental scan task in different status.", "fieldConfig": { @@ -19064,7 +19897,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The memory usage per TiKV instance", "editable": true, @@ -19182,7 +20015,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The speed of TiKV CDC incremental scan", "editable": true, @@ -19293,7 +20126,7 @@ "bars": true, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total bytes of TiKV CDC incremental scan", "editable": true, @@ -19404,7 +20237,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "fieldConfig": { @@ -19541,7 +20374,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "The total number of cache entries in the old value cache.", "fieldConfig": { @@ -19669,7 +20502,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "editable": true, @@ -19789,7 +20622,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The time consumed to get an old value (both from cache and from disk)", "fieldConfig": { "defaults": {}, @@ -19862,7 +20695,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "decimals": 1, "description": "", "fieldConfig": { @@ -19987,7 +20820,7 @@ "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 17 }, "id": 294, "panels": [ @@ -19996,7 +20829,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20091,7 +20924,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20186,7 +21019,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20300,7 +21133,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "unit": "short" @@ -20413,7 +21246,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20520,7 +21353,7 @@ } }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "color": { @@ -20630,7 +21463,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20739,7 +21572,7 @@ "h": 1, "w": 24, "x": 0, - "y": 17 + "y": 18 }, "id": 10021, "panels": [ @@ -20757,7 +21590,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of fsync called by redo writer", "fieldConfig": { "defaults": {}, @@ -20831,7 +21664,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of flushall called by redo writer", "fieldConfig": { "defaults": {}, @@ -20905,7 +21738,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of writeLog called by redoManager", "fieldConfig": { "defaults": {}, @@ -20979,7 +21812,7 @@ "mode": "spectrum" }, "dataFormat": "tsbuckets", - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The latency distributions of flushLog called by redoManager", "fieldConfig": { "defaults": {}, @@ -21044,7 +21877,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The total count of rows that are processed by redo writer", "fieldConfig": { "defaults": {}, @@ -21154,7 +21987,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Total number of bytes redo log written", "fieldConfig": { "defaults": {}, @@ -21250,7 +22083,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Redo bgUpdateLog worker busy ratio", "fieldConfig": { "defaults": {}, @@ -21354,7 +22187,7 @@ "h": 1, "w": 24, "x": 0, - "y": 18 + "y": 19 }, "id": 155, "panels": [ @@ -21364,7 +22197,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", "editable": true, "error": false, @@ -21411,8 +22244,8 @@ }, "runtime_instance": { "selected": false, - "text": "10.2.6.132:8300", - "value": "10.2.6.132:8300" + "text": "10.2.7.144:8300", + "value": "10.2.7.144:8300" } }, "seriesOverrides": [ @@ -21550,7 +22383,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "Count of live objects.", "editable": true, "error": false, @@ -21597,8 +22430,8 @@ }, "runtime_instance": { "selected": false, - "text": "10.2.6.132:8300", - "value": "10.2.6.132:8300" + "text": "10.2.7.144:8300", + "value": "10.2.7.144:8300" } }, "seriesOverrides": [], @@ -21663,7 +22496,7 @@ "cacheTimeout": null, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "TiCDC process Go garbage collection STW pause duration", "editable": true, "error": false, @@ -21704,8 +22537,8 @@ }, "runtime_instance": { "selected": false, - "text": "10.2.6.132:8300", - "value": "10.2.6.132:8300" + "text": "10.2.7.144:8300", + "value": "10.2.7.144:8300" } }, "seriesOverrides": [ @@ -21789,7 +22622,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "The throughput of Go's memory allocator.", "fill": 1, "gridPos": { @@ -21826,8 +22659,8 @@ }, "runtime_instance": { "selected": false, - "text": "10.2.6.132:8300", - "value": "10.2.6.132:8300" + "text": "10.2.7.144:8300", + "value": "10.2.7.144:8300" } }, "seriesOverrides": [ @@ -21940,7 +22773,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22045,7 +22878,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22150,7 +22983,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22255,7 +23088,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22351,7 +23184,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22447,7 +23280,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22543,7 +23376,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22640,7 +23473,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22739,7 +23572,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": {}, "overrides": [] @@ -22834,7 +23667,7 @@ } }, { - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "color": { @@ -22929,7 +23762,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "fieldConfig": { "defaults": { "unit": "s" @@ -23029,7 +23862,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": { @@ -23130,7 +23963,7 @@ "bars": false, "dashLength": 10, "dashes": false, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "description": "", "fieldConfig": { "defaults": { @@ -23240,7 +24073,7 @@ { "allValue": null, "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "", "description": null, "error": null, @@ -23267,7 +24100,7 @@ { "allValue": null, "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "", "description": null, "error": null, @@ -23294,7 +24127,7 @@ { "allValue": ".*", "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(ticdc_processor_processor_tick_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, namespace)", "description": null, "error": null, @@ -23321,7 +24154,7 @@ { "allValue": ".*", "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(ticdc_processor_processor_tick_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, changefeed)", "description": null, "error": null, @@ -23348,7 +24181,7 @@ { "allValue": ".*", "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "description": null, "error": null, @@ -23375,7 +24208,7 @@ { "allValue": ".*", "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", "description": null, "error": null, @@ -23458,7 +24291,7 @@ { "allValue": "", "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "description": null, "error": null, @@ -23485,7 +24318,7 @@ { "allValue": null, "current": {}, - "datasource": "${DS_LOCAL}", + "datasource": "${DS_C1}", "definition": "label_values(ticdc_actor_number_of_workers{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, name)", "description": null, "error": null, @@ -23542,6 +24375,6 @@ }, "timezone": "browser", "title": "test-TiCDC", - "uid": "YiGL8hBZ0", - "version": 15 + "uid": "YiGL8hBZ0aaa", + "version": 14 } \ No newline at end of file diff --git a/pkg/metrics/dispatcher.go b/pkg/metrics/dispatcher.go index 0a9ee868f..2e8301966 100644 --- a/pkg/metrics/dispatcher.go +++ b/pkg/metrics/dispatcher.go @@ -46,7 +46,7 @@ var ( EventDispatcherManagerResolvedTsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "dispatcher", + Subsystem: "dispatchermanager", Name: "resolved_ts", Help: "Resolved ts of event dispatcher manager(changefeed)", }, []string{"namespace", "changefeed"}) @@ -54,7 +54,7 @@ var ( EventDispatcherManagerResolvedTsLagGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "dispatcher", + Subsystem: "dispatchermanager", Name: "resolved_ts_lag", Help: "Resolved ts lag of event dispatcher manager(changefeed) in seconds", }, []string{"namespace", "changefeed"}) @@ -62,7 +62,7 @@ var ( EventDispatcherManagerCheckpointTsGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "dispatcher", + Subsystem: "dispatchermanager", Name: "checkpoint_ts", Help: "Checkpoint ts of event dispatcher manager(changefeed)", }, []string{"namespace", "changefeed"}) @@ -70,7 +70,7 @@ var ( EventDispatcherManagerCheckpointTsLagGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", - Subsystem: "dispatcher", + Subsystem: "dispatchermanager", Name: "checkpoint_ts_lag", Help: "Checkpoint ts lag of event dispatcher manager(changefeed) in seconds", }, []string{"namespace", "changefeed"}) From 82bee8894adc956f3d7c3b8b5623f3f269907f81 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 20 Aug 2024 00:19:11 +0800 Subject: [PATCH 25/29] refactor event broker --- pkg/eventservice/event_broker.go | 31 ++++++++++--------- pkg/eventservice/event_broker_test.go | 15 ++++----- pkg/eventservice/event_service.go | 13 ++++---- .../event_service_performance_test.go | 8 ++--- pkg/eventservice/event_service_test.go | 6 ++++ 5 files changed, 40 insertions(+), 33 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 9cd7ade09..685fecbd8 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -110,6 +110,14 @@ func (c *eventBroker) sendWatermark( } } +func (c *eventBroker) onAsyncNotify(change *subscriptionChange) { + select { + case c.changedCh <- change: + default: + // TODO: add metrics to record the drop count. + } +} + func (c *eventBroker) runGenerateScanTask(ctx context.Context) { c.wg.Add(1) go func() { @@ -314,7 +322,7 @@ func (c *eventBroker) logSlowDispatchers(ctx context.Context) { func (c *eventBroker) updateMetrics(ctx context.Context) { c.wg.Add(1) - ticker := time.NewTicker(time.Second * 5) + ticker := time.NewTicker(time.Second * 10) go func() { defer c.wg.Done() log.Info("update metrics goroutine is started") @@ -374,8 +382,8 @@ type dispatcherStat struct { info DispatcherInfo spanSubscription *spanSubscription // The watermark of the events that have been sent to the dispatcher. - watermark atomic.Uint64 - notify chan *subscriptionChange + watermark atomic.Uint64 + onAsyncNotify func(*subscriptionChange) // The index of the task queue channel in the taskPool. // We need to make sure the tasks of the same dispatcher are sent to the same task queue // so that it will be handle by the same scan worker. To ensure all events of the dispatcher @@ -390,14 +398,12 @@ type dispatcherStat struct { } func newDispatcherStat( - startTs uint64, - info DispatcherInfo, - notify chan *subscriptionChange) *dispatcherStat { + startTs uint64, info DispatcherInfo, onAsyncNotify func(*subscriptionChange), +) *dispatcherStat { subscription := &spanSubscription{ span: info.GetTableSpan(), lastUpdate: atomic.Value{}, } - subscription.lastUpdate.Store(time.Now()) subscription.watermark.Store(uint64(startTs)) @@ -405,7 +411,7 @@ func newDispatcherStat( res := &dispatcherStat{ info: info, spanSubscription: subscription, - notify: notify, + onAsyncNotify: onAsyncNotify, metricSorterOutputEventCountKV: metrics.SorterOutputEventCount.WithLabelValues(namespace, id, "kv"), metricEventServiceSendKvCount: metrics.EventServiceSendEventCount.WithLabelValues(namespace, id, "kv"), @@ -428,15 +434,10 @@ func (a *dispatcherStat) onSubscriptionWatermark(watermark uint64) { } a.spanSubscription.watermark.Store(watermark) a.spanSubscription.lastUpdate.Store(time.Now()) - - sub := &subscriptionChange{ + a.onAsyncNotify(&subscriptionChange{ dispatcherInfo: a.info, eventCount: a.spanSubscription.newEventCount.Swap(0), - } - select { - case a.notify <- sub: - default: - } + }) } // TODO: consider to use a better way to update the event count, may be we only need to diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index fad8329b5..2e0d9f35d 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -19,13 +19,9 @@ func TestNewDispatcherStat(t *testing.T) { startTs: startTs, } - notify := make(chan *subscriptionChange) - - stat := newDispatcherStat(startTs, info, notify) - + stat := newDispatcherStat(startTs, info, func(c *subscriptionChange) {}) require.Equal(t, info, stat.info) require.Equal(t, startTs, stat.watermark.Load()) - require.Equal(t, notify, stat.notify) require.NotNil(t, stat.spanSubscription) require.Equal(t, startTs, stat.spanSubscription.watermark.Load()) require.Equal(t, 0, int(stat.spanSubscription.newEventCount.Load())) @@ -43,7 +39,12 @@ func TestDispatcherStatUpdateWatermark(t *testing.T) { notify := make(chan *subscriptionChange) - stat := newDispatcherStat(startTs, info, notify) + stat := newDispatcherStat(startTs, info, func(c *subscriptionChange) { + select { + case notify <- c: + default: + } + }) // Case 1: no new events, only watermark change wg.Add(1) @@ -99,7 +100,7 @@ func TestScanTaskPool_PushTask(t *testing.T) { startTs: 1000, span: span, } - dispatcherStat := newDispatcherStat(dispatcherInfo.startTs, dispatcherInfo, make(chan *subscriptionChange)) + dispatcherStat := newDispatcherStat(dispatcherInfo.startTs, dispatcherInfo, func(c *subscriptionChange) {}) // Create two tasks with overlapping data ranges task1 := &scanTask{ dispatcherStat: dispatcherStat, diff --git a/pkg/eventservice/event_service.go b/pkg/eventservice/event_service.go index 3f5093b84..95e20643e 100644 --- a/pkg/eventservice/event_service.go +++ b/pkg/eventservice/event_service.go @@ -113,11 +113,10 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn c = newEventBroker(ctx, clusterID, s.eventStore, s.mc) s.brokers[clusterID] = c } - - dispatcher := newDispatcherStat(startTs, info, c.changedCh) + dispatcher := newDispatcherStat(startTs, info, c.onAsyncNotify) c.dispatchers.Store(info.GetID(), dispatcher) + brokerRegisterDuration := time.Since(start) - firstDuration := time.Since(start) start = time.Now() c.eventStore.RegisterDispatcher( info.GetID(), @@ -126,9 +125,12 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn dispatcher.onNewEvent, dispatcher.onSubscriptionWatermark, ) + eventStoreRegisterDuration := time.Since(start) - secondDuration := time.Since(start) - log.Info("register acceptor", zap.Uint64("clusterID", clusterID), zap.String("acceptorID", info.GetID()), zap.Uint64("tableID", span.TableID), zap.Uint64("startTs", startTs), zap.Duration("firstDuration", firstDuration), zap.Duration("secondDuration", secondDuration)) + log.Info("register acceptor", zap.Uint64("clusterID", clusterID), + zap.String("acceptorID", info.GetID()), zap.Uint64("tableID", span.TableID), + zap.Uint64("startTs", startTs), zap.Duration("brokerRegisterDuration", brokerRegisterDuration), + zap.Duration("eventStoreRegisterDuration", eventStoreRegisterDuration)) } func (s *eventService) deregisterDispatcher(dispatcherInfo DispatcherInfo) { @@ -142,7 +144,6 @@ func (s *eventService) deregisterDispatcher(dispatcherInfo DispatcherInfo) { log.Info("deregister acceptor", zap.Uint64("clusterID", clusterID), zap.String("acceptorID", id)) } -// TODO: implement the following functions func msgToDispatcherInfo(msg *messaging.TargetMessage) DispatcherInfo { return msg.Message.(messaging.RegisterDispatcherRequest) } diff --git a/pkg/eventservice/event_service_performance_test.go b/pkg/eventservice/event_service_performance_test.go index 609846e95..9a4170b6e 100644 --- a/pkg/eventservice/event_service_performance_test.go +++ b/pkg/eventservice/event_service_performance_test.go @@ -19,14 +19,14 @@ import ( // Result: // 1. It takes about 10 seconds to register 1 million tables. // 2. It takes about 300-500ms to update resolvedTs for 1 million tables. The call chain is: -// -> dispatcherStat.onSubscriptionWatermark() -> dispatcherStat.notify() -> taskPool.pushTask(), merge task -> scanWorker new Msg -> messageCenter.SendMsg() +// -> dispatcherStat.onSubscriptionWatermark() -> dispatcherStat.onAsyncNotify() -> taskPool.pushTask(), merge task -> scanWorker new Msg -> messageCenter.SendMsg() // It should be note that some task of the same dispatcher are merged into one task, so the messageCenter.SendMsg() is not called for each dispatcherStat.onSubscriptionWatermark(). func TestEventServiceOneMillionTable(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() wg := &sync.WaitGroup{} mockStore := newMockEventStore() - tableNum := 1000000 + tableNum := 100_0000 sendRound := 10 mc := &mockMessageCenter{ messageCh: make(chan *messaging.TargetMessage, 100), @@ -67,9 +67,7 @@ func TestEventServiceOneMillionTable(t *testing.T) { for i := 0; i < tableNum; i++ { acceptorInfo := newMockAcceptorInfo(uuid.New().String(), uint64(i)) dispatchers = append(dispatchers, acceptorInfo) - } - for _, dispatcher := range dispatchers { - esImpl.registerDispatcher(ctx, dispatcher) + esImpl.registerDispatcher(ctx, acceptorInfo) } log.Info("register 1 million tables", zap.Duration("cost", time.Since(start))) diff --git a/pkg/eventservice/event_service_test.go b/pkg/eventservice/event_service_test.go index 47d9dd0c6..698149416 100644 --- a/pkg/eventservice/event_service_test.go +++ b/pkg/eventservice/event_service_test.go @@ -24,11 +24,17 @@ import ( "go.uber.org/zap" ) +var _ messaging.MessageCenter = &mockMessageCenter{} + // mockMessageCenter is a mock implementation of the MessageCenter interface type mockMessageCenter struct { messageCh chan *messaging.TargetMessage } +func (m *mockMessageCenter) OnNodeChanges(newNodes []*common.NodeInfo, removedNodes []*common.NodeInfo) { + +} + func (m *mockMessageCenter) SendEvent(event ...*messaging.TargetMessage) error { for _, e := range event { m.messageCh <- e From c03ba1eaa099d6cecffb9e5be68223f4c3797db5 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 20 Aug 2024 11:04:02 +0800 Subject: [PATCH 26/29] ignore log file --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index d25c1dbfb..350a5dfb4 100644 --- a/.gitignore +++ b/.gitignore @@ -5,3 +5,4 @@ tools/include .vscode .idea +*.log From bd8a263e97c33936b59a571b3ab4e4205e6f18c8 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 20 Aug 2024 12:12:47 +0800 Subject: [PATCH 27/29] fix concurrent map iteration and map write --- downstreamadapter/eventcollector/event_collector.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index c836937e0..5265c131a 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -278,11 +278,13 @@ func (c *EventCollector) updateMetrics(ctx context.Context) error { return case <-ticker.C: minResolvedTs := uint64(0) + c.dispatcherMap.RLock() for _, d := range c.dispatcherMap.m { if minResolvedTs == 0 || d.GetResolvedTs() < minResolvedTs { minResolvedTs = d.GetResolvedTs() } } + c.dispatcherMap.RUnlock() if minResolvedTs == 0 { continue } From 0fed90f55d6cc5248a04b467ffbaa513580254fc Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 20 Aug 2024 12:56:44 +0800 Subject: [PATCH 28/29] event broker enlarge notify chan size Signed-off-by: dongmen <414110582@qq.com> --- pkg/eventservice/event_broker.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 685fecbd8..53583cdd7 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -70,7 +70,7 @@ func newEventBroker( eventStore: eventStore, dispatchers: sync.Map{}, msgSender: mc, - changedCh: make(chan *subscriptionChange, defaultChannelSize), + changedCh: make(chan *subscriptionChange, defaultChannelSize*16), taskPool: newScanTaskPool(), scanWorkerCount: defaultWorkerCount, messageCh: make(chan *wrapMessage, defaultChannelSize), @@ -111,11 +111,7 @@ func (c *eventBroker) sendWatermark( } func (c *eventBroker) onAsyncNotify(change *subscriptionChange) { - select { - case c.changedCh <- change: - default: - // TODO: add metrics to record the drop count. - } + c.changedCh <- change } func (c *eventBroker) runGenerateScanTask(ctx context.Context) { From dc4c63eff516db2c1d11b2d8b35f406dec0936a0 Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Tue, 20 Aug 2024 15:54:19 +0800 Subject: [PATCH 29/29] fix ut --- pkg/eventservice/event_broker_test.go | 8 ++++---- pkg/eventservice/event_service_performance_test.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index 2e0d9f35d..d868b33ca 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -14,7 +14,7 @@ func TestNewDispatcherStat(t *testing.T) { startTs := uint64(123) info := &mockDispatcherInfo{ - id: "test", + id: common.NewDispatcherID(), clusterID: 1, startTs: startTs, } @@ -32,7 +32,7 @@ func TestDispatcherStatUpdateWatermark(t *testing.T) { startTs := uint64(123) wg := &sync.WaitGroup{} info := &mockDispatcherInfo{ - id: "test", + id: common.NewDispatcherID(), clusterID: 1, startTs: startTs, } @@ -95,7 +95,7 @@ func TestScanTaskPool_PushTask(t *testing.T) { pool := newScanTaskPool() span := newTableSpan(1, "a", "b") dispatcherInfo := &mockDispatcherInfo{ - id: "dispatcher1", + id: common.NewDispatcherID(), clusterID: 1, startTs: 1000, span: span, @@ -139,7 +139,7 @@ func TestScanTaskPool_PushTask(t *testing.T) { pool.taskSet[dispatcherInfo.GetID()] = task1 // Verify that the task is in the taskSet - task, ok := pool.taskSet["dispatcher1"] + task, ok := pool.taskSet[dispatcherInfo.id] require.True(t, ok) require.Equal(t, task1, task) diff --git a/pkg/eventservice/event_service_performance_test.go b/pkg/eventservice/event_service_performance_test.go index 9a4170b6e..7be9bb3d4 100644 --- a/pkg/eventservice/event_service_performance_test.go +++ b/pkg/eventservice/event_service_performance_test.go @@ -7,9 +7,9 @@ import ( "testing" "time" + "github.com/flowbehappy/tigate/pkg/common" appcontext "github.com/flowbehappy/tigate/pkg/common/context" "github.com/flowbehappy/tigate/pkg/messaging" - "github.com/google/uuid" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -65,7 +65,7 @@ func TestEventServiceOneMillionTable(t *testing.T) { dispatchers := make([]DispatcherInfo, 0, tableNum) // register 1000,000 tables for i := 0; i < tableNum; i++ { - acceptorInfo := newMockAcceptorInfo(uuid.New().String(), uint64(i)) + acceptorInfo := newMockAcceptorInfo(common.NewDispatcherID(), uint64(i)) dispatchers = append(dispatchers, acceptorInfo) esImpl.registerDispatcher(ctx, acceptorInfo) }