From a05b81a23fce13a0ebaaaee1910284f79f93c5a2 Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 29 Dec 2020 17:39:50 +0800 Subject: [PATCH 01/44] panic when adding a dying table --- cdc/processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/processor.go b/cdc/processor.go index e8364af5b10..23f1176828b 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -966,7 +966,7 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo if table, ok := p.tables[tableID]; ok { if atomic.SwapUint32(&table.isDying, 0) == 1 { - log.Warn("The same table exists but is dying. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + log.Panic("The same table exists but is dying. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) table.cancel() } else { log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) From c05c08b4800ddd3a7fcfb1d62c27032aa371911e Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 30 Dec 2020 14:54:30 +0800 Subject: [PATCH 02/44] add a test --- cdc/changefeed.go | 7 +++ .../conf/diff_config.toml | 27 ++++++++++ tests/owner_remove_table_error/run.sh | 51 +++++++++++++++++++ 3 files changed, 85 insertions(+) create mode 100644 tests/owner_remove_table_error/conf/diff_config.toml create mode 100644 tests/owner_remove_table_error/run.sh diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 33fb690d752..36d3bf6cc25 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -20,6 +20,8 @@ import ( "sync" "time" + "github.com/pingcap/failpoint" + "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" @@ -362,6 +364,11 @@ func (c *changeFeed) balanceOrphanTables(ctx context.Context, captures map[model } c.taskStatus[captureID] = newStatus.Clone() log.Info("dispatch table success", zap.String("capture-id", captureID), zap.Stringer("status", newStatus)) + failpoint.Inject("OwnerRemoveTableError", func() { + if len(cleanedTables) > 0 { + failpoint.Return(errors.New("failpoint injected error")) + } + }) } for tableID := range cleanedTables { diff --git a/tests/owner_remove_table_error/conf/diff_config.toml b/tests/owner_remove_table_error/conf/diff_config.toml new file mode 100644 index 00000000000..74ddef0c33e --- /dev/null +++ b/tests/owner_remove_table_error/conf/diff_config.toml @@ -0,0 +1,27 @@ +# diff Configuration. + +log-level = "info" +chunk-size = 10 +check-thread-count = 4 +sample-percent = 100 +use-rowid = false +use-checksum = true +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] + schema = "owner_remove_table_error" + tables = ["~t.*"] + +[[source-db]] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + instance-id = "source-1" + +[target-db] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh new file mode 100644 index 00000000000..21d7177342e --- /dev/null +++ b/tests/owner_remove_table_error/run.sh @@ -0,0 +1,51 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +CDC_COUNT=3 +DB_COUNT=4 +MAX_RETRIES=20 + +export -f check_changefeed_state + +function run() { + # kafka is not supported yet. + if [ "$SINK_TYPE" == "kafka" ]; then + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR + cd $WORK_DIR + + pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1"; + + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr + changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') + + run_sql "CREATE DATABASE owner_remove_table_error;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE table owner_remove_table_error.t1(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO owner_remove_table_error.t1 VALUES (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DROP table owner_remove_table_error.t1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE table owner_remove_table_error.t2(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO owner_remove_table_error.t2 VALUES (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE table owner_remove_table_error.finished_mark(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + + check_table_exists "owner_remove_table_error.finished_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 21ec264056409e6a703e1f5fb274d79f2034c5a2 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 30 Dec 2020 15:12:18 +0800 Subject: [PATCH 03/44] add a test --- .../jenkins_ci/integration_test_common.groovy | 43 ++++++++++--------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 1fc01f4b935..8ae312e34d7 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -114,26 +114,29 @@ def tests(sink_type, node_label) { } - unstash 'cases_name' - def cases_name = sh ( - script: 'cat go/src/github.com/pingcap/ticdc/tests/CASES', - returnStdout: true - ).trim().split() - - def step_cases = [] - def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) - for(int i in 1..CONCURRENT_NUMBER) { - def end = i*step_length-1 - if (i == CONCURRENT_NUMBER){ - end = cases_name.size()-1 - } - step_cases.add(cases_name[(i-1)*step_length..end]) - } - step_cases.eachWithIndex{ case_names, index -> - def step_name = "step_${index}" - test_cases["integration test ${step_name}"] = { - run_integration_test(step_name, case_names.join(" ")) - } +// unstash 'cases_name' +// def cases_name = sh ( +// script: 'cat go/src/github.com/pingcap/ticdc/tests/CASES', +// returnStdout: true +// ).trim().split() +// +// def step_cases = [] +// def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) +// for(int i in 1..CONCURRENT_NUMBER) { +// def end = i*step_length-1 +// if (i == CONCURRENT_NUMBER){ +// end = cases_name.size()-1 +// } +// step_cases.add(cases_name[(i-1)*step_length..end]) +// } +// step_cases.eachWithIndex{ case_names, index -> +// def step_name = "step_${index}" +// test_cases["integration test ${step_name}"] = { +// run_integration_test(step_name, case_names.join(" ")) +// } +// } + test_cases["integration test owner_remove_table_error"] = { + run_integration_test("step_owner_remove_table_error", "owner_remove_table_error") } parallel test_cases From d72853d274513314af55e7d9780b8c961179e29c Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 30 Dec 2020 15:17:42 +0800 Subject: [PATCH 04/44] add a test --- tests/owner_remove_table_error/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh index 21d7177342e..8a587493cab 100644 --- a/tests/owner_remove_table_error/run.sh +++ b/tests/owner_remove_table_error/run.sh @@ -12,7 +12,6 @@ CDC_COUNT=3 DB_COUNT=4 MAX_RETRIES=20 -export -f check_changefeed_state function run() { # kafka is not supported yet. From fc77ae891ff97113b0da99d07f4f5ed4c035ce78 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 30 Dec 2020 15:58:50 +0800 Subject: [PATCH 05/44] add a test --- cdc/processor.go | 5 ++++- tests/owner_remove_table_error/run.sh | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index 23f1176828b..6f44d7c2329 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -966,7 +966,10 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo if table, ok := p.tables[tableID]; ok { if atomic.SwapUint32(&table.isDying, 0) == 1 { - log.Panic("The same table exists but is dying. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + failpoint.Inject("ProcessorPanicWhenDyingTableAdded", func() { + log.Panic("The same table exists but is dying.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + }) + log.Warn("The same table exists but is dying. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) table.cancel() } else { log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh index 8a587493cab..e485d8eb356 100644 --- a/tests/owner_remove_table_error/run.sh +++ b/tests/owner_remove_table_error/run.sh @@ -26,7 +26,7 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1"; - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true);github.com/pingcap/ticdc/cdc/ProcessorPanicWhenDyingTableAdded=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') From ef86a5b64a963d934ae3b76d1364d92d234c9a76 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 30 Dec 2020 16:52:57 +0800 Subject: [PATCH 06/44] update the checkpoint when exec ddl --- cdc/changefeed.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 36d3bf6cc25..4a3ba2622b0 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -883,16 +883,18 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { tsUpdated = true } + updateCheckpointTs := minCheckpointTs if minCheckpointTs > c.status.CheckpointTs { c.status.CheckpointTs = minCheckpointTs // when the `c.ddlState` is `model.ChangeFeedWaitToExecDDL`, // some DDL is waiting to executed, we can't ensure whether the DDL has been executed. // so we can't emit checkpoint to sink - if c.ddlState != model.ChangeFeedWaitToExecDDL { - err := c.sink.EmitCheckpointTs(ctx, minCheckpointTs) - if err != nil { - return errors.Trace(err) - } + if c.ddlState == model.ChangeFeedWaitToExecDDL { + updateCheckpointTs-- + } + err := c.sink.EmitCheckpointTs(ctx, updateCheckpointTs) + if err != nil { + return errors.Trace(err) } tsUpdated = true } @@ -900,7 +902,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if tsUpdated { log.Debug("update changefeed", zap.String("id", c.id), - zap.Uint64("checkpoint ts", c.status.CheckpointTs), + zap.Uint64("checkpoint ts", updateCheckpointTs), zap.Uint64("resolved ts", c.status.ResolvedTs)) } return nil From ee598c58bf283fbe3c8be45a379a7747cafb71ce Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 5 Jan 2021 19:44:31 +0800 Subject: [PATCH 07/44] add a middleware before sink --- cdc/processor.go | 474 +++++++----------- cdc/puller/rectifier.go | 114 ----- cdc/puller/rectifier_test.go | 198 -------- cdc/sink/manager.go | 121 +++++ cdc/sink/manager_test.go | 1 + .../jenkins_ci/integration_test_common.groovy | 44 +- tests/owner_remove_table_error/run.sh | 2 +- 7 files changed, 325 insertions(+), 629 deletions(-) delete mode 100644 cdc/puller/rectifier.go delete mode 100644 cdc/puller/rectifier_test.go create mode 100644 cdc/sink/manager.go create mode 100644 cdc/sink/manager_test.go diff --git a/cdc/processor.go b/cdc/processor.go index 6f44d7c2329..3cf866b7194 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "io" + "math" "os" "strconv" "sync" @@ -79,9 +80,8 @@ type processor struct { etcdCli kv.CDCEtcdClient session *concurrency.Session - sink sink.Sink + sinkManager *sink.Manager - sinkEmittedResolvedTs uint64 globalResolvedTs uint64 localResolvedTs uint64 checkpointTs uint64 @@ -92,7 +92,6 @@ type processor struct { ddlPullerCancel context.CancelFunc schemaStorage *entry.SchemaStorage - output chan *model.PolymorphicEvent mounter entry.Mounter stateMu sync.Mutex @@ -102,12 +101,11 @@ type processor struct { markTableIDs map[int64]struct{} statusModRevision int64 - sinkEmittedResolvedNotifier *notify.Notifier - sinkEmittedResolvedReceiver *notify.Receiver - localResolvedNotifier *notify.Notifier - localResolvedReceiver *notify.Receiver - localCheckpointTsNotifier *notify.Notifier - localCheckpointTsReceiver *notify.Receiver + globalResolvedTsNotifier *notify.Notifier + localResolvedNotifier *notify.Notifier + localResolvedReceiver *notify.Receiver + localCheckpointTsNotifier *notify.Notifier + localCheckpointTsReceiver *notify.Receiver wg *errgroup.Group errCh chan<- error @@ -115,18 +113,16 @@ type processor struct { } type tableInfo struct { - id int64 - name string // quoted schema and table, used in metircs only - resolvedTs uint64 - markTableID int64 - mResolvedTs uint64 - sorter *puller.Rectifier - workload model.WorkloadInfo - cancel context.CancelFunc - // isDying shows that the table is being removed. - // In the case the same table is added back before safe removal is finished, - // this flag is used to tell whether it's safe to kill the table. - isDying uint32 + id int64 + name string // quoted schema and table, used in metircs only + resolvedTs uint64 + checkpointTs uint64 + + markTableID int64 + mResolvedTs uint64 + mCheckpointTs uint64 + workload model.WorkloadInfo + cancel context.CancelFunc } func (t *tableInfo) loadResolvedTs() uint64 { @@ -140,15 +136,15 @@ func (t *tableInfo) loadResolvedTs() uint64 { return tableRts } -// safeStop will stop the table change feed safety -func (t *tableInfo) safeStop() (stopped bool, checkpointTs model.Ts) { - atomic.StoreUint32(&t.isDying, 1) - t.sorter.SafeStop() - status := t.sorter.GetStatus() - if status != model.SorterStatusStopped && status != model.SorterStatusFinished { - return false, 0 +func (t *tableInfo) loadCheckpointTs() uint64 { + tableCkpt := atomic.LoadUint64(&t.checkpointTs) + if t.markTableID != 0 { + mTableCkpt := atomic.LoadUint64(&t.mCheckpointTs) + if mTableCkpt < tableCkpt { + return mTableCkpt + } } - return true, t.sorter.GetMaxResolvedTs() + return tableCkpt } // newProcessor creates and returns a processor for the specified change feed @@ -158,7 +154,7 @@ func newProcessor( credential *security.Credential, session *concurrency.Session, changefeed model.ChangeFeedInfo, - sink sink.Sink, + sinkManager *sink.Manager, changefeedID string, captureInfo model.CaptureInfo, checkpointTs uint64, @@ -186,13 +182,9 @@ func newProcessor( return nil, errors.Trace(err) } - sinkEmittedResolvedNotifier := new(notify.Notifier) localResolvedNotifier := new(notify.Notifier) localCheckpointTsNotifier := new(notify.Notifier) - sinkEmittedResolvedReceiver, err := sinkEmittedResolvedNotifier.NewReceiver(50 * time.Millisecond) - if err != nil { - return nil, err - } + globalResolvedTsNotifier := new(notify.Notifier) localResolvedReceiver, err := localResolvedNotifier.NewReceiver(50 * time.Millisecond) if err != nil { return nil, err @@ -212,7 +204,7 @@ func newProcessor( credential: credential, etcdCli: cdcEtcdCli, session: session, - sink: sink, + sinkManager: sinkManager, ddlPuller: ddlPuller, mounter: entry.NewMounter(schemaStorage, changefeed.Config.Mounter.WorkerNum, changefeed.Config.EnableOldValue), schemaStorage: schemaStorage, @@ -221,13 +213,10 @@ func newProcessor( flushCheckpointInterval: flushCheckpointInterval, position: &model.TaskPosition{CheckPointTs: checkpointTs}, - output: make(chan *model.PolymorphicEvent, defaultOutputChanSize), - sinkEmittedResolvedNotifier: sinkEmittedResolvedNotifier, - sinkEmittedResolvedReceiver: sinkEmittedResolvedReceiver, - - localResolvedNotifier: localResolvedNotifier, - localResolvedReceiver: localResolvedReceiver, + globalResolvedTsNotifier: globalResolvedTsNotifier, + localResolvedNotifier: localResolvedNotifier, + localResolvedReceiver: localResolvedReceiver, checkpointTs: checkpointTs, localCheckpointTsNotifier: localCheckpointTsNotifier, @@ -266,18 +255,6 @@ func (p *processor) Run(ctx context.Context) { return p.globalStatusWorker(cctx) }) - wg.Go(func() error { - return p.sinkDriver(cctx) - }) - - wg.Go(func() error { - return p.syncResolved(cctx) - }) - - wg.Go(func() error { - return p.collectMetrics(cctx) - }) - wg.Go(func() error { return p.ddlPuller.Run(ddlPullerCtx) }) @@ -413,11 +390,23 @@ func (p *processor) positionWorker(ctx context.Context) error { } } case <-p.localCheckpointTsReceiver.C: - checkpointTs := atomic.LoadUint64(&p.checkpointTs) + checkpointTs := uint64(math.MaxUint64) + p.stateMu.Lock() + for _, table := range p.tables { + ts := table.loadCheckpointTs() + if ts < checkpointTs { + checkpointTs = ts + } + } + if len(p.tables) == 0 { + checkpointTs = 0 + } + p.stateMu.Unlock() if checkpointTs == 0 { log.Warn("0 is not a valid checkpointTs", util.ZapFieldChangefeed(ctx)) continue } + atomic.StoreUint64(&p.checkpointTs, checkpointTs) phyTs := oracle.ExtractPhysical(checkpointTs) // It is more accurate to get tso from PD, but in most cases we have // deployed NTP service, a little bias is acceptable here. @@ -584,9 +573,6 @@ func (p *processor) removeTable(tableID int64) { return } - if atomic.SwapUint32(&table.isDying, 0) == 0 { - return - } table.cancel() delete(p.tables, tableID) if table.markTableID != 0 { @@ -613,19 +599,18 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) status.Dirty = true continue } - stopped, checkpointTs := table.safeStop() - log.Debug("safeStop table", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), zap.Bool("stopped", stopped), + table.cancel() + checkpointTs := table.loadCheckpointTs() + log.Debug("stop table", zap.Int64("tableID", tableID), + util.ZapFieldChangefeed(ctx), zap.Uint64("checkpointTs", checkpointTs)) - if stopped { - opt.BoundaryTs = checkpointTs - if checkpointTs <= p.position.CheckPointTs { - tablesToRemove = append(tablesToRemove, tableID) - opt.Done = true - opt.Status = model.OperFinished - } - status.Dirty = true + opt.BoundaryTs = checkpointTs + if checkpointTs <= p.position.CheckPointTs { + tablesToRemove = append(tablesToRemove, tableID) + opt.Done = true + opt.Status = model.OperFinished } + status.Dirty = true } } else { replicaInfo, exist := status.Tables[tableID] @@ -677,18 +662,12 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { log.Info("Global status worker started", util.ZapFieldChangefeed(ctx)) var ( - changefeedStatus *model.ChangeFeedStatus - statusRev int64 - lastCheckPointTs uint64 - lastResolvedTs uint64 - watchKey = kv.GetEtcdKeyJob(p.changefeedID) - globalResolvedTsNotifier = new(notify.Notifier) + changefeedStatus *model.ChangeFeedStatus + statusRev int64 + lastCheckPointTs uint64 + lastResolvedTs uint64 + watchKey = kv.GetEtcdKeyJob(p.changefeedID) ) - defer globalResolvedTsNotifier.Close() - globalResolvedTsReceiver, err := globalResolvedTsNotifier.NewReceiver(1 * time.Second) - if err != nil { - return err - } updateStatus := func(changefeedStatus *model.ChangeFeedStatus) { atomic.StoreUint64(&p.globalcheckpointTs, changefeedStatus.CheckpointTs) @@ -709,34 +688,10 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { atomic.StoreUint64(&p.globalResolvedTs, lastResolvedTs) log.Debug("Update globalResolvedTs", zap.Uint64("globalResolvedTs", lastResolvedTs), util.ZapFieldChangefeed(ctx)) - globalResolvedTsNotifier.Notify() + p.globalResolvedTsNotifier.Notify() } } - go func() { - for { - select { - case <-ctx.Done(): - return - case <-globalResolvedTsReceiver.C: - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - if globalResolvedTs > localResolvedTs { - log.Warn("globalResolvedTs too large", zap.Uint64("globalResolvedTs", globalResolvedTs), - zap.Uint64("localResolvedTs", localResolvedTs), util.ZapFieldChangefeed(ctx)) - // we do not issue resolved events if globalResolvedTs > localResolvedTs. - continue - } - select { - case <-ctx.Done(): - return - case p.output <- model.NewResolvedPolymorphicEvent(0, globalResolvedTs): - // regionID = 0 means the event is produced by TiCDC - } - } - } - }() - retryCfg := backoff.WithMaxRetries( backoff.WithContext( backoff.NewExponentialBackOff(), ctx), @@ -787,153 +742,6 @@ func (p *processor) globalStatusWorker(ctx context.Context) error { } } -func (p *processor) sinkDriver(ctx context.Context) error { - metricFlushDuration := sinkFlushRowChangedDuration.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-p.sinkEmittedResolvedReceiver.C: - sinkEmittedResolvedTs := atomic.LoadUint64(&p.sinkEmittedResolvedTs) - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - var minTs uint64 - if sinkEmittedResolvedTs < globalResolvedTs { - minTs = sinkEmittedResolvedTs - } else { - minTs = globalResolvedTs - } - if minTs == 0 || atomic.LoadUint64(&p.checkpointTs) == minTs { - continue - } - start := time.Now() - - checkpointTs, err := p.sink.FlushRowChangedEvents(ctx, minTs) - if err != nil { - return errors.Trace(err) - } - if checkpointTs != 0 { - atomic.StoreUint64(&p.checkpointTs, checkpointTs) - p.localCheckpointTsNotifier.Notify() - } - - dur := time.Since(start) - metricFlushDuration.Observe(dur.Seconds()) - if dur > 3*time.Second { - log.Warn("flush row changed events too slow", - zap.Duration("duration", dur), util.ZapFieldChangefeed(ctx)) - } - } - } -} - -// syncResolved handle `p.ddlJobsCh` and `p.resolvedTxns` -func (p *processor) syncResolved(ctx context.Context) error { - defer func() { - p.sinkEmittedResolvedReceiver.Stop() - log.Info("syncResolved stopped", util.ZapFieldChangefeed(ctx)) - }() - - events := make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) - rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) - - flushRowChangedEvents := func() error { - for _, ev := range events { - err := ev.WaitPrepare(ctx) - if err != nil { - return errors.Trace(err) - } - if ev.Row == nil { - continue - } - rows = append(rows, ev.Row) - } - failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { - log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") - time.Sleep(10 * time.Second) - panic("ProcessorSyncResolvedPreEmit") - }) - err := p.sink.EmitRowChangedEvents(ctx, rows...) - if err != nil { - return errors.Trace(err) - } - events = events[:0] - rows = rows[:0] - return nil - } - - processRowChangedEvent := func(row *model.PolymorphicEvent) error { - events = append(events, row) - - if len(events) >= defaultSyncResolvedBatch { - err := flushRowChangedEvents() - if err != nil { - return errors.Trace(err) - } - } - return nil - } - - var resolvedTs uint64 - for { - select { - case <-ctx.Done(): - return ctx.Err() - case row := <-p.output: - if row == nil { - continue - } - failpoint.Inject("ProcessorSyncResolvedError", func() { - failpoint.Return(errors.New("processor sync resolved injected error")) - }) - if row.RawKV != nil && row.RawKV.OpType == model.OpTypeResolved { - err := flushRowChangedEvents() - if err != nil { - return errors.Trace(err) - } - resolvedTs = row.CRTs - atomic.StoreUint64(&p.sinkEmittedResolvedTs, row.CRTs) - p.sinkEmittedResolvedNotifier.Notify() - continue - } - // Global resolved ts should fallback in some table rebalance cases, - // since the start-ts(from checkpoint ts) or a rebalanced table could - // be less then the global resolved ts. - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - if resolvedTs > localResolvedTs { - log.Info("global resolved ts fallback", - zap.String("changefeed", p.changefeedID), - zap.Uint64("localResolvedTs", localResolvedTs), - zap.Uint64("resolvedTs", resolvedTs), - ) - resolvedTs = localResolvedTs - } - if row.CRTs <= resolvedTs { - _ = row.WaitPrepare(ctx) - log.Panic("The CRTs must be greater than the resolvedTs", - zap.String("model", "processor"), - zap.String("changefeed", p.changefeedID), - zap.Uint64("resolvedTs", resolvedTs), - zap.Any("row", row)) - } - err := processRowChangedEvent(row) - if err != nil { - return errors.Trace(err) - } - } - } -} - -func (p *processor) collectMetrics(ctx context.Context) error { - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-time.After(defaultMetricInterval): - tableOutputChanSizeGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Set(float64(len(p.output))) - } - } -} - func createSchemaStorage( kvStorage tidbkv.Storage, checkpointTs uint64, @@ -964,17 +772,9 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo tableName = strconv.Itoa(int(tableID)) } - if table, ok := p.tables[tableID]; ok { - if atomic.SwapUint32(&table.isDying, 0) == 1 { - failpoint.Inject("ProcessorPanicWhenDyingTableAdded", func() { - log.Panic("The same table exists but is dying.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) - }) - log.Warn("The same table exists but is dying. Cancel it and continue.", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) - table.cancel() - } else { - log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) - return - } + if _, ok := p.tables[tableID]; ok { + log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) + return } globalcheckpointTs := atomic.LoadUint64(&p.globalcheckpointTs) @@ -987,7 +787,7 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo zap.Uint64("startTs", replicaInfo.StartTs)) } - globalResolvedTs := atomic.LoadUint64(&p.sinkEmittedResolvedTs) + globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) log.Debug("Add table", zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx), zap.String("name", tableName), @@ -1000,13 +800,12 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo id: tableID, name: tableName, resolvedTs: replicaInfo.StartTs, - cancel: cancel, } // TODO(leoppro) calculate the workload of this table // We temporarily set the value to constant 1 table.workload = model.WorkloadInfo{Workload: 1} - startPuller := func(tableID model.TableID, pResolvedTs *uint64) *puller.Rectifier { + startPuller := func(tableID model.TableID, pResolvedTs *uint64, pCheckpointTs *uint64) sink.Sink { // start table puller enableOldValue := p.changefeed.Config.EnableOldValue span := regionspan.GetTableSpan(tableID, enableOldValue) @@ -1023,10 +822,10 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo } }() - var sorterImpl puller.EventSorter + var sorter puller.EventSorter switch p.changefeed.Engine { case model.SortInMemory: - sorterImpl = puller.NewEntrySorter() + sorter = puller.NewEntrySorter() case model.SortInFile, model.SortUnified: err := util.IsDirAndWritable(p.changefeed.SortDir) if err != nil { @@ -1043,17 +842,15 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo } if p.changefeed.Engine == model.SortInFile { - sorterImpl = puller.NewFileSorter(p.changefeed.SortDir) + sorter = puller.NewFileSorter(p.changefeed.SortDir) } else { // Unified Sorter - sorterImpl = psorter.NewUnifiedSorter(p.changefeed.SortDir, tableName, util.CaptureAddrFromCtx(ctx)) + sorter = psorter.NewUnifiedSorter(p.changefeed.SortDir, tableName, util.CaptureAddrFromCtx(ctx)) } default: p.errCh <- cerror.ErrUnknownSortEngine.GenWithStackByArgs(p.changefeed.Engine) return nil } - sorter := puller.NewRectifier(sorterImpl, p.changefeed.GetTargetTs()) - go func() { err := sorter.Run(ctx) if errors.Cause(err) != context.Canceled { @@ -1065,13 +862,13 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo p.pullerConsume(ctx, plr, sorter) }() + tableSink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) go func() { - p.sorterConsume(ctx, tableID, tableName, sorter, pResolvedTs, replicaInfo) + p.sorterConsume(ctx, tableID, tableName, sorter, pResolvedTs, pCheckpointTs, replicaInfo, tableSink) }() - - return sorter + return tableSink } - + var tableSink, mTableSink sink.Sink if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID != 0 { mTableID := replicaInfo.MarkTableID // we should to make sure a mark table is only listened once. @@ -1080,7 +877,7 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo table.markTableID = mTableID table.mResolvedTs = replicaInfo.StartTs - startPuller(mTableID, &table.mResolvedTs) + mTableSink = startPuller(mTableID, &table.mResolvedTs, &table.mCheckpointTs) } } @@ -1093,8 +890,14 @@ func (p *processor) addTable(ctx context.Context, tableID int64, replicaInfo *mo } atomic.StoreUint64(&p.localResolvedTs, p.position.ResolvedTs) - table.sorter = startPuller(tableID, &table.resolvedTs) - + tableSink = startPuller(tableID, &table.resolvedTs, &table.checkpointTs) + table.cancel = func() { + cancel() + tableSink.Close() + if mTableSink != nil { + mTableSink.Close() + } + } syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Inc() } @@ -1104,13 +907,16 @@ func (p *processor) sorterConsume( ctx context.Context, tableID int64, tableName string, - sorter *puller.Rectifier, + sorter puller.EventSorter, pResolvedTs *uint64, + pCheckpointTs *uint64, replicaInfo *model.TableReplicaInfo, + sink sink.Sink, ) { var lastResolvedTs uint64 opDone := false resolvedTsGauge := tableResolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr, tableName) + metricFlushDuration := sinkFlushRowChangedDuration.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) checkDoneTicker := time.NewTicker(1 * time.Second) checkDone := func() { localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) @@ -1141,6 +947,53 @@ func (p *processor) sorterConsume( } } + events := make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) + rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) + + flushRowChangedEvents := func() error { + for _, ev := range events { + err := ev.WaitPrepare(ctx) + if err != nil { + return errors.Trace(err) + } + if ev.Row == nil { + continue + } + rows = append(rows, ev.Row) + } + failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { + log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") + time.Sleep(10 * time.Second) + panic("ProcessorSyncResolvedPreEmit") + }) + err := sink.EmitRowChangedEvents(ctx, rows...) + if err != nil { + return errors.Trace(err) + } + events = events[:0] + rows = rows[:0] + return nil + } + + processRowChangedEvent := func(row *model.PolymorphicEvent) error { + events = append(events, row) + + if len(events) >= defaultSyncResolvedBatch { + err := flushRowChangedEvents() + if err != nil { + return errors.Trace(err) + } + } + return nil + } + + globalResolvedTsReceiver, err := p.globalResolvedTsNotifier.NewReceiver(1 * time.Second) + if err != nil { + p.errCh <- errors.Trace(err) + return + } + defer globalResolvedTsReceiver.Stop() + for { select { case <-ctx.Done(): @@ -1164,6 +1017,11 @@ func (p *processor) sorterConsume( } if pEvent.RawKV != nil && pEvent.RawKV.OpType == model.OpTypeResolved { + err := flushRowChangedEvents() + if err != nil { + p.errCh <- errors.Trace(err) + return + } atomic.StoreUint64(pResolvedTs, pEvent.CRTs) lastResolvedTs = pEvent.CRTs p.localResolvedNotifier.Notify() @@ -1173,24 +1031,52 @@ func (p *processor) sorterConsume( } continue } - sinkResolvedTs := atomic.LoadUint64(&p.sinkEmittedResolvedTs) if pEvent.CRTs <= lastResolvedTs || pEvent.CRTs < replicaInfo.StartTs { log.Panic("The CRTs of event is not expected, please report a bug", util.ZapFieldChangefeed(ctx), zap.String("model", "sorter"), - zap.Uint64("globalResolvedTs", sinkResolvedTs), zap.Uint64("resolvedTs", lastResolvedTs), zap.Int64("tableID", tableID), zap.Any("replicaInfo", replicaInfo), zap.Any("row", pEvent)) } - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.errCh <- ctx.Err() - } + failpoint.Inject("ProcessorSyncResolvedError", func() { + failpoint.Return(errors.New("processor sync resolved injected error")) + }) + err := processRowChangedEvent(pEvent) + if err != nil { + p.errCh <- errors.Trace(err) + return + } + case <-globalResolvedTsReceiver.C: + localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) + globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) + var minTs uint64 + if localResolvedTs < globalResolvedTs { + minTs = localResolvedTs + } else { + minTs = globalResolvedTs + } + if minTs == 0 || atomic.LoadUint64(&p.checkpointTs) == minTs { + continue + } + start := time.Now() + + checkpointTs, err := sink.FlushRowChangedEvents(ctx, minTs) + if err != nil { + p.errCh <- errors.Trace(err) return - case p.output <- pEvent: + } + if checkpointTs != 0 { + atomic.StoreUint64(pCheckpointTs, checkpointTs) + p.localCheckpointTsNotifier.Notify() + } + + dur := time.Since(start) + metricFlushDuration.Observe(dur.Seconds()) + if dur > 3*time.Second { + log.Warn("flush row changed events too slow", + zap.Duration("duration", dur), util.ZapFieldChangefeed(ctx)) } case <-checkDoneTicker.C: if !opDone { @@ -1205,7 +1091,7 @@ func (p *processor) sorterConsume( func (p *processor) pullerConsume( ctx context.Context, plr puller.Puller, - sorter *puller.Rectifier, + sorter puller.EventSorter, ) { for { select { @@ -1233,6 +1119,9 @@ func (p *processor) stop(ctx context.Context) error { p.ddlPullerCancel() // mark tables share the same context with its original table, don't need to cancel p.stateMu.Unlock() + p.globalResolvedTsNotifier.Close() + p.localCheckpointTsNotifier.Close() + p.localResolvedNotifier.Close() failpoint.Inject("processorStopDelay", nil) atomic.StoreInt32(&p.stopped, 1) if err := p.etcdCli.DeleteTaskPosition(ctx, p.changefeedID, p.captureInfo.ID); err != nil { @@ -1244,7 +1133,7 @@ func (p *processor) stop(ctx context.Context) error { if err := p.etcdCli.DeleteTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID); err != nil { return err } - return p.sink.Close() + return p.sinkManager.Close() } func (p *processor) isStopped() bool { @@ -1278,12 +1167,13 @@ func runProcessor( } ctx, cancel := context.WithCancel(ctx) errCh := make(chan error, 1) - sink, err := sink.NewSink(ctx, changefeedID, info.SinkURI, filter, info.Config, opts, errCh) + s, err := sink.NewSink(ctx, changefeedID, info.SinkURI, filter, info.Config, opts, errCh) if err != nil { cancel() return nil, errors.Trace(err) } - processor, err := newProcessor(ctx, pdCli, credential, session, info, sink, + sinkManager := sink.NewManager(s, checkpointTs) + processor, err := newProcessor(ctx, pdCli, credential, session, info, sinkManager, changefeedID, captureInfo, checkpointTs, errCh, flushCheckpointInterval) if err != nil { cancel() diff --git a/cdc/puller/rectifier.go b/cdc/puller/rectifier.go deleted file mode 100644 index efbfc849492..00000000000 --- a/cdc/puller/rectifier.go +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package puller - -import ( - "context" - "sync/atomic" - - "go.uber.org/zap" - - "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "golang.org/x/sync/errgroup" -) - -// Rectifier filters and collates the output stream from the sorter -type Rectifier struct { - EventSorter - - status model.SorterStatus - maxSentResolvedTs model.Ts - targetTs model.Ts - - outputCh chan *model.PolymorphicEvent -} - -// NewRectifier creates a new Rectifier -func NewRectifier(s EventSorter, targetTs model.Ts) *Rectifier { - return &Rectifier{ - EventSorter: s, - targetTs: targetTs, - outputCh: make(chan *model.PolymorphicEvent), - } -} - -// GetStatus returns the state of the Rectifier -func (r *Rectifier) GetStatus() model.SorterStatus { - return atomic.LoadInt32(&r.status) -} - -// GetMaxResolvedTs returns the maximum resolved ts sent from the Rectifier -func (r *Rectifier) GetMaxResolvedTs() model.Ts { - return atomic.LoadUint64(&r.maxSentResolvedTs) -} - -// SafeStop stops the Rectifier and Sorter safety -func (r *Rectifier) SafeStop() { - atomic.CompareAndSwapInt32(&r.status, - model.SorterStatusWorking, - model.SorterStatusStopping) -} - -// Run running the Rectifier -func (r *Rectifier) Run(ctx context.Context) error { - output := func(event *model.PolymorphicEvent) { - select { - case <-ctx.Done(): - log.Warn("failed to send to output channel", zap.Error(ctx.Err())) - case r.outputCh <- event: - } - } - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - return r.EventSorter.Run(ctx) - }) - errg.Go(func() error { - defer close(r.outputCh) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-r.EventSorter.Output(): - if event == nil { - return nil - } - if event.CRTs > r.targetTs { - output(model.NewResolvedPolymorphicEvent(event.RegionID(), r.targetTs)) - atomic.StoreUint64(&r.maxSentResolvedTs, r.targetTs) - atomic.StoreInt32(&r.status, model.SorterStatusFinished) - return nil - } - output(event) - if event.RawKV.OpType == model.OpTypeResolved { - atomic.StoreUint64(&r.maxSentResolvedTs, event.CRTs) - switch atomic.LoadInt32(&r.status) { - case model.SorterStatusStopping: - atomic.StoreInt32(&r.status, model.SorterStatusStopped) - return nil - case model.SorterStatusStopped: - return nil - case model.SorterStatusWorking: - } - } - } - } - }) - return errg.Wait() -} - -// Output returns the output streams -func (r *Rectifier) Output() <-chan *model.PolymorphicEvent { - return r.outputCh -} diff --git a/cdc/puller/rectifier_test.go b/cdc/puller/rectifier_test.go deleted file mode 100644 index 91f41e7a161..00000000000 --- a/cdc/puller/rectifier_test.go +++ /dev/null @@ -1,198 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package puller - -import ( - "context" - "math" - "sync/atomic" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" - "golang.org/x/sync/errgroup" -) - -type rectifierSuite struct{} - -var _ = check.Suite(&rectifierSuite{}) - -type mockSorter struct { - outputCh chan *model.PolymorphicEvent -} - -func newMockSorter() *mockSorter { - return &mockSorter{ - outputCh: make(chan *model.PolymorphicEvent, 16), - } -} - -func (m *mockSorter) Run(ctx context.Context) error { - <-ctx.Done() - return nil -} - -func (m *mockSorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) { - select { - case <-ctx.Done(): - return - case m.outputCh <- entry: - } -} - -func (m *mockSorter) Output() <-chan *model.PolymorphicEvent { - return m.outputCh -} - -func waitEntriesReceived(ctx context.Context, currentNum *int32, expectedNum int32) { - for { - select { - case <-ctx.Done(): - return - default: - num := atomic.LoadInt32(currentNum) - if num >= expectedNum { - return - } - } - time.Sleep(50 * time.Millisecond) - } -} - -func (s *rectifierSuite) TestRectifierSafeStop(c *check.C) { - defer testleak.AfterTest(c)() - mockSorter := newMockSorter() - r := NewRectifier(mockSorter, math.MaxUint64) - ctx, cancel := context.WithCancel(context.Background()) - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - return r.Run(ctx) - }) - entriesReceivedNum := int32(0) - errg.Go(func() error { - expected := []*model.PolymorphicEvent{ - {CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 2, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 3, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 4, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 5, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 6, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 7, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 8, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 9, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - } - i := int32(0) - for e := range r.Output() { - c.Assert(e, check.DeepEquals, expected[i]) - i++ - atomic.StoreInt32(&entriesReceivedNum, i) - } - - return nil - }) - entries := []*model.PolymorphicEvent{ - {CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 2, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 3, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 4, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 5, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 6, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - } - for _, e := range entries { - mockSorter.AddEntry(ctx, e) - } - waitEntriesReceived(ctx, &entriesReceivedNum, 6) - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusWorking) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(5)) - r.SafeStop() - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusStopping) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(5)) - - entries = []*model.PolymorphicEvent{ - {CRTs: 7, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 8, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 9, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 10, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 11, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - } - for _, e := range entries { - mockSorter.AddEntry(ctx, e) - } - waitEntriesReceived(ctx, &entriesReceivedNum, 9) - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusStopped) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(9)) - r.SafeStop() - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusStopped) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(9)) - cancel() - c.Assert(errg.Wait(), check.IsNil) -} - -func (s *rectifierSuite) TestRectifierFinished(c *check.C) { - defer testleak.AfterTest(c)() - mockSorter := newMockSorter() - r := NewRectifier(mockSorter, 7) - ctx, cancel := context.WithCancel(context.Background()) - errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - return r.Run(ctx) - }) - entriesReceivedNum := int32(0) - errg.Go(func() error { - expected := []*model.PolymorphicEvent{ - {CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 2, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 3, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 4, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 5, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 6, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 7, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 7, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved, CRTs: 7}}, - } - i := int32(0) - for e := range r.Output() { - c.Assert(e, check.DeepEquals, expected[i]) - i++ - atomic.StoreInt32(&entriesReceivedNum, i) - } - - return nil - }) - entries := []*model.PolymorphicEvent{ - {CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 2, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 3, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 4, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 5, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 6, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 7, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 8, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 9, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - {CRTs: 10, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}, - {CRTs: 11, RawKV: &model.RawKVEntry{OpType: model.OpTypeResolved}}, - } - for _, e := range entries { - mockSorter.AddEntry(ctx, e) - } - waitEntriesReceived(ctx, &entriesReceivedNum, 8) - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusFinished) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(7)) - r.SafeStop() - c.Assert(r.GetStatus(), check.Equals, model.SorterStatusFinished) - c.Assert(r.GetMaxResolvedTs(), check.Equals, model.Ts(7)) - - cancel() - c.Assert(errg.Wait(), check.IsNil) -} diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go new file mode 100644 index 00000000000..8474921ca8a --- /dev/null +++ b/cdc/sink/manager.go @@ -0,0 +1,121 @@ +package sink + +import ( + "context" + "math" + "sort" + + "github.com/pingcap/log" + "go.uber.org/zap" + + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" +) + +type Manager struct { + backendSink Sink + checkpointTs model.Ts + tableSinks map[model.TableID]*tableSink +} + +func NewManager(backendSink Sink, checkpointTs model.Ts) *Manager { + return &Manager{ + backendSink: backendSink, + checkpointTs: checkpointTs, + tableSinks: make(map[model.TableID]*tableSink), + } +} + +func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts) Sink { + if _, exist := m.tableSinks[tableID]; exist { + log.Panic("the table sink already exists", zap.Uint64("tableID", uint64(tableID))) + } + sink := &tableSink{ + tableID: tableID, + manager: m, + buffer: make([]*model.RowChangedEvent, 0, 128), + emittedTs: checkpointTs, + } + m.tableSinks[tableID] = sink + return sink +} + +func (m *Manager) Close() error { + return m.backendSink.Close() +} + +func (m *Manager) getMinEmittedTs() model.Ts { + if len(m.tableSinks) == 0 { + return m.checkpointTs + } + minTs := model.Ts(math.MaxUint64) + for _, tableSink := range m.tableSinks { + if minTs > tableSink.emittedTs { + minTs = tableSink.emittedTs + } + } + return minTs +} + +func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { + checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, m.getMinEmittedTs()) + if err != nil { + return m.checkpointTs, errors.Trace(err) + } + m.checkpointTs = checkpointTs + return checkpointTs, nil +} + +func (m *Manager) destroyTableSink(tableID model.TableID) { + delete(m.tableSinks, tableID) +} + +type tableSink struct { + tableID model.TableID + manager *Manager + buffer []*model.RowChangedEvent + emittedTs model.Ts +} + +func (t *tableSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + // do nothing + return nil +} + +func (t *tableSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + t.buffer = append(t.buffer, rows...) + return nil +} + +func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + // the table sink doesn't receive the DDL event + return nil +} + +func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + i := sort.Search(len(t.buffer), func(i int) bool { + return t.buffer[i].CommitTs > resolvedTs + }) + if i == 0 { + t.emittedTs = resolvedTs + return t.manager.flushBackendSink(ctx) + } + resolvedRows := t.buffer[:i] + t.buffer = t.buffer[i:] + err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...) + if err != nil { + return t.manager.checkpointTs, errors.Trace(err) + } + t.emittedTs = resolvedTs + return t.manager.flushBackendSink(ctx) +} + +func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + // the table sink doesn't receive the checkpoint event + return nil +} + +func (t *tableSink) Close() error { + t.manager.destroyTableSink(t.tableID) + return nil +} diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go new file mode 100644 index 00000000000..b9d240359de --- /dev/null +++ b/cdc/sink/manager_test.go @@ -0,0 +1 @@ +package sink diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 8ae312e34d7..6072d63128f 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -114,31 +114,27 @@ def tests(sink_type, node_label) { } -// unstash 'cases_name' -// def cases_name = sh ( -// script: 'cat go/src/github.com/pingcap/ticdc/tests/CASES', -// returnStdout: true -// ).trim().split() -// -// def step_cases = [] -// def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) -// for(int i in 1..CONCURRENT_NUMBER) { -// def end = i*step_length-1 -// if (i == CONCURRENT_NUMBER){ -// end = cases_name.size()-1 -// } -// step_cases.add(cases_name[(i-1)*step_length..end]) -// } -// step_cases.eachWithIndex{ case_names, index -> -// def step_name = "step_${index}" -// test_cases["integration test ${step_name}"] = { -// run_integration_test(step_name, case_names.join(" ")) -// } -// } - test_cases["integration test owner_remove_table_error"] = { - run_integration_test("step_owner_remove_table_error", "owner_remove_table_error") + unstash 'cases_name' + def cases_name = sh ( + script: 'cat go/src/github.com/pingcap/ticdc/tests/CASES', + returnStdout: true + ).trim().split() + + def step_cases = [] + def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) + for(int i in 1..CONCURRENT_NUMBER) { + def end = i*step_length-1 + if (i == CONCURRENT_NUMBER){ + end = cases_name.size()-1 + } + step_cases.add(cases_name[(i-1)*step_length..end]) + } + step_cases.eachWithIndex{ case_names, index -> + def step_name = "step_${index}" + test_cases["integration test ${step_name}"] = { + run_integration_test(step_name, case_names.join(" ")) + } } - parallel test_cases } } diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh index e485d8eb356..8a587493cab 100644 --- a/tests/owner_remove_table_error/run.sh +++ b/tests/owner_remove_table_error/run.sh @@ -26,7 +26,7 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1"; - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true);github.com/pingcap/ticdc/cdc/ProcessorPanicWhenDyingTableAdded=return(true)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') From 59d0ab5023438bf858a4d9816ee16b757b09caa6 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 11:10:46 +0800 Subject: [PATCH 08/44] add a middleware before sink --- cdc/sink/manager.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 8474921ca8a..6cef1e72e57 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -4,6 +4,8 @@ import ( "context" "math" "sort" + "sync" + "sync/atomic" "github.com/pingcap/log" "go.uber.org/zap" @@ -16,6 +18,8 @@ type Manager struct { backendSink Sink checkpointTs model.Ts tableSinks map[model.TableID]*tableSink + tableSinksMu sync.Mutex + flushMu sync.Mutex } func NewManager(backendSink Sink, checkpointTs model.Ts) *Manager { @@ -36,6 +40,8 @@ func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts) buffer: make([]*model.RowChangedEvent, 0, 128), emittedTs: checkpointTs, } + m.tableSinksMu.Lock() + defer m.tableSinksMu.Unlock() m.tableSinks[tableID] = sink return sink } @@ -49,6 +55,8 @@ func (m *Manager) getMinEmittedTs() model.Ts { return m.checkpointTs } minTs := model.Ts(math.MaxUint64) + m.tableSinksMu.Lock() + defer m.tableSinksMu.Unlock() for _, tableSink := range m.tableSinks { if minTs > tableSink.emittedTs { minTs = tableSink.emittedTs @@ -58,15 +66,20 @@ func (m *Manager) getMinEmittedTs() model.Ts { } func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { - checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, m.getMinEmittedTs()) + minEmittedTs := m.getMinEmittedTs() + m.flushMu.Lock() + defer m.flushMu.Unlock() + checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs) if err != nil { return m.checkpointTs, errors.Trace(err) } - m.checkpointTs = checkpointTs + atomic.StoreUint64(&m.checkpointTs, checkpointTs) return checkpointTs, nil } func (m *Manager) destroyTableSink(tableID model.TableID) { + m.tableSinksMu.Lock() + defer m.tableSinksMu.Unlock() delete(m.tableSinks, tableID) } From 8c1eb0b20047b2e91a21751bed78513fea21b599 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 11:23:29 +0800 Subject: [PATCH 09/44] fix build --- cdc/processor.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cdc/processor.go b/cdc/processor.go index 3cf866b7194..d3da1204d24 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1041,7 +1041,8 @@ func (p *processor) sorterConsume( zap.Any("row", pEvent)) } failpoint.Inject("ProcessorSyncResolvedError", func() { - failpoint.Return(errors.New("processor sync resolved injected error")) + p.errCh <- errors.New("processor sync resolved injected error") + failpoint.Return() }) err := processRowChangedEvent(pEvent) if err != nil { From 9804ed51fc34e34ed0dace7046dca764a2a15aa2 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 14:51:27 +0800 Subject: [PATCH 10/44] fix build --- cdc/processor.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cdc/processor.go b/cdc/processor.go index d3da1204d24..fc5c75ee451 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1024,6 +1024,7 @@ func (p *processor) sorterConsume( } atomic.StoreUint64(pResolvedTs, pEvent.CRTs) lastResolvedTs = pEvent.CRTs + log.Info("[LEOPPRO] show table resolvedts", zap.Int64("tableID", tableID), zap.Uint64("resolvedTs", lastResolvedTs)) p.localResolvedNotifier.Notify() resolvedTsGauge.Set(float64(oracle.ExtractPhysical(pEvent.CRTs))) if !opDone { From 6598b81232ee3e5cf67612a37d97b89832de0ff9 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 15:34:05 +0800 Subject: [PATCH 11/44] fix test --- cdc/sink/manager_test.go | 65 +++++++++++++++++++++++++++ tests/owner_remove_table_error/run.sh | 2 +- 2 files changed, 66 insertions(+), 1 deletion(-) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index b9d240359de..ff51f911c84 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -1 +1,66 @@ package sink + +import ( + "context" + "math/rand" + + "github.com/pingcap/check" + "github.com/pingcap/ticdc/cdc/model" +) + +type managerSuite struct{} + +var _ = check.Suite(&testCausalitySuite{}) + +type checkSink struct { +} + +func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + panic("implement me") +} + +func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + panic("implement me") +} + +func (c *checkSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + panic("implement me") +} + +func (c *checkSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + panic("implement me") +} + +func (c *checkSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + panic("implement me") +} + +func (c *checkSink) Close() error { + panic("implement me") +} + +func (s *testCausalitySuite) TestManagerRandom(c *check.C) { + manager := NewManager(&checkSink{}, 0) + for i := 0; i < 10; i++ { + i := i + go func() { + ctx := context.Background() + tableSink := manager.CreateTableSink(model.TableID(i), 0) + var lastResolvedTs uint64 + for j := 0; j < 100; j++ { + if rand.Intn(10) == 0 { + resolvedTs := lastResolvedTs + uint64(rand.Intn(j-int(lastResolvedTs))) + _, err := tableSink.FlushRowChangedEvents(ctx, resolvedTs) + c.Assert(err, check.IsNil) + lastResolvedTs = resolvedTs + } else { + err := tableSink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + Table: &model.TableName{TableID: int64(i)}, + CommitTs: uint64(j), + }) + c.Assert(err, check.IsNil) + } + } + }() + } +} diff --git a/tests/owner_remove_table_error/run.sh b/tests/owner_remove_table_error/run.sh index 8a587493cab..d78907c44e8 100644 --- a/tests/owner_remove_table_error/run.sh +++ b/tests/owner_remove_table_error/run.sh @@ -26,7 +26,7 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1"; - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=return(true)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/OwnerRemoveTableError=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1|tail -n2|head -n1|awk '{print $2}') From 315be4a8e563dac2398cd525dedee6f1ff3f2398 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 17:05:54 +0800 Subject: [PATCH 12/44] add unit tests --- cdc/sink/manager.go | 2 +- cdc/sink/manager_test.go | 57 ++++++++++++++++++++++++++++++++++------ 2 files changed, 50 insertions(+), 9 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 6cef1e72e57..c144f12515e 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -66,9 +66,9 @@ func (m *Manager) getMinEmittedTs() model.Ts { } func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { - minEmittedTs := m.getMinEmittedTs() m.flushMu.Lock() defer m.flushMu.Unlock() + minEmittedTs := m.getMinEmittedTs() checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs) if err != nil { return m.checkpointTs, errors.Trace(err) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index ff51f911c84..6a806c63fc2 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -3,6 +3,10 @@ package sink import ( "context" "math/rand" + "sync" + + "github.com/pingcap/log" + "go.uber.org/zap" "github.com/pingcap/check" "github.com/pingcap/ticdc/cdc/model" @@ -10,9 +14,13 @@ import ( type managerSuite struct{} -var _ = check.Suite(&testCausalitySuite{}) +var _ = check.Suite(&managerSuite{}) type checkSink struct { + *check.C + rows []*model.RowChangedEvent + rowsMu sync.Mutex + lastResolvedTs uint64 } func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { @@ -20,7 +28,13 @@ func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTab } func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { - panic("implement me") + c.rowsMu.Lock() + defer c.rowsMu.Unlock() + for _, row := range rows { + log.Info("rows in check sink", zap.Reflect("row", row)) + } + c.rows = append(c.rows, rows...) + return nil } func (c *checkSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { @@ -28,7 +42,22 @@ func (c *checkSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error } func (c *checkSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { - panic("implement me") + c.rowsMu.Lock() + defer c.rowsMu.Unlock() + log.Info("flush in check sink", zap.Uint64("resolved", resolvedTs)) + var newRows []*model.RowChangedEvent + for _, row := range c.rows { + c.Assert(row.CommitTs, check.Greater, c.lastResolvedTs) + if row.CommitTs > resolvedTs { + newRows = append(newRows, row) + } + } + + c.Assert(c.lastResolvedTs, check.LessEqual, resolvedTs) + c.lastResolvedTs = resolvedTs + c.rows = newRows + + return c.lastResolvedTs, nil } func (c *checkSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -39,15 +68,24 @@ func (c *checkSink) Close() error { panic("implement me") } -func (s *testCausalitySuite) TestManagerRandom(c *check.C) { - manager := NewManager(&checkSink{}, 0) - for i := 0; i < 10; i++ { +func (s *managerSuite) TestManagerRandom(c *check.C) { + manager := NewManager(&checkSink{C: c}, 0) + goroutineNum := 10 + rowNum := 100 + var wg sync.WaitGroup + tableSinks := make([]Sink, goroutineNum) + for i := 0; i < goroutineNum; i++ { + tableSinks[i] = manager.CreateTableSink(model.TableID(i), 0) + } + for i := 0; i < goroutineNum; i++ { i := i + tableSink := tableSinks[i] + wg.Add(1) go func() { + defer wg.Done() ctx := context.Background() - tableSink := manager.CreateTableSink(model.TableID(i), 0) var lastResolvedTs uint64 - for j := 0; j < 100; j++ { + for j := 1; j < rowNum; j++ { if rand.Intn(10) == 0 { resolvedTs := lastResolvedTs + uint64(rand.Intn(j-int(lastResolvedTs))) _, err := tableSink.FlushRowChangedEvents(ctx, resolvedTs) @@ -61,6 +99,9 @@ func (s *testCausalitySuite) TestManagerRandom(c *check.C) { c.Assert(err, check.IsNil) } } + _, err := tableSink.FlushRowChangedEvents(ctx, uint64(rowNum)) + c.Assert(err, check.IsNil) }() } + wg.Wait() } From 0bfac0cf31f1bb25ed80bcad3745afdfd4dae864 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 6 Jan 2021 17:47:01 +0800 Subject: [PATCH 13/44] fix race --- cdc/sink/manager.go | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index c144f12515e..98e279812c0 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -52,14 +52,15 @@ func (m *Manager) Close() error { func (m *Manager) getMinEmittedTs() model.Ts { if len(m.tableSinks) == 0 { - return m.checkpointTs + return m.getCheckpointTs() } minTs := model.Ts(math.MaxUint64) m.tableSinksMu.Lock() defer m.tableSinksMu.Unlock() for _, tableSink := range m.tableSinks { - if minTs > tableSink.emittedTs { - minTs = tableSink.emittedTs + emittedTs := tableSink.getEmittedTs() + if minTs > emittedTs { + minTs = emittedTs } } return minTs @@ -71,7 +72,7 @@ func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { minEmittedTs := m.getMinEmittedTs() checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs) if err != nil { - return m.checkpointTs, errors.Trace(err) + return m.getCheckpointTs(), errors.Trace(err) } atomic.StoreUint64(&m.checkpointTs, checkpointTs) return checkpointTs, nil @@ -83,6 +84,10 @@ func (m *Manager) destroyTableSink(tableID model.TableID) { delete(m.tableSinks, tableID) } +func (m *Manager) getCheckpointTs() uint64 { + return atomic.LoadUint64(&m.checkpointTs) +} + type tableSink struct { tableID model.TableID manager *Manager @@ -110,19 +115,23 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 return t.buffer[i].CommitTs > resolvedTs }) if i == 0 { - t.emittedTs = resolvedTs + atomic.StoreUint64(&t.emittedTs, resolvedTs) return t.manager.flushBackendSink(ctx) } resolvedRows := t.buffer[:i] t.buffer = t.buffer[i:] err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...) if err != nil { - return t.manager.checkpointTs, errors.Trace(err) + return t.manager.getCheckpointTs(), errors.Trace(err) } - t.emittedTs = resolvedTs + atomic.StoreUint64(&t.emittedTs, resolvedTs) return t.manager.flushBackendSink(ctx) } +func (t *tableSink) getEmittedTs() uint64 { + return atomic.LoadUint64(&t.emittedTs) +} + func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { // the table sink doesn't receive the checkpoint event return nil From fb6627da288d1b3f1d86280f7d8bb4c8b3a7cde3 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 11:18:08 +0800 Subject: [PATCH 14/44] add debug code --- cdc/processor.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cdc/processor.go b/cdc/processor.go index fc5c75ee451..760964d36a9 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1069,6 +1069,7 @@ func (p *processor) sorterConsume( p.errCh <- errors.Trace(err) return } + log.Info("LEOPPRO: show checkpointTs after sink", zap.Uint64("resolved", minTs), zap.Uint64("checkpointTs", checkpointTs)) if checkpointTs != 0 { atomic.StoreUint64(pCheckpointTs, checkpointTs) p.localCheckpointTsNotifier.Notify() From 3326decbfdc7aa5a48ae4790f52b12cc0d9e2a64 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 11:33:47 +0800 Subject: [PATCH 15/44] fix some bug --- cdc/processor.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index 760964d36a9..ccf997211de 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -17,7 +17,6 @@ import ( "context" "fmt" "io" - "math" "os" "strconv" "sync" @@ -390,7 +389,7 @@ func (p *processor) positionWorker(ctx context.Context) error { } } case <-p.localCheckpointTsReceiver.C: - checkpointTs := uint64(math.MaxUint64) + checkpointTs := atomic.LoadUint64(&p.globalResolvedTs) p.stateMu.Lock() for _, table := range p.tables { ts := table.loadCheckpointTs() @@ -398,9 +397,6 @@ func (p *processor) positionWorker(ctx context.Context) error { checkpointTs = ts } } - if len(p.tables) == 0 { - checkpointTs = 0 - } p.stateMu.Unlock() if checkpointTs == 0 { log.Warn("0 is not a valid checkpointTs", util.ZapFieldChangefeed(ctx)) From 7cc0bec083b9b3f7f4393871c264db2037d0a395 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 12:03:46 +0800 Subject: [PATCH 16/44] fix check --- cdc/changefeed.go | 3 +-- cdc/metrics.go | 6 ------ cdc/processor.go | 6 ------ cdc/sink/manager.go | 17 +++++++++++++++++ cdc/sink/manager_test.go | 13 +++++++++++++ .../jenkins_ci/integration_test_common.groovy | 1 + 6 files changed, 32 insertions(+), 14 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 4a3ba2622b0..bfb4ec80d29 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -20,9 +20,8 @@ import ( "sync" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/ticdc/cdc/entry" diff --git a/cdc/metrics.go b/cdc/metrics.go index d7212ca2c02..df8a1c38615 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -14,8 +14,6 @@ package cdc import ( - "time" - "github.com/pingcap/ticdc/cdc/puller/sorter" "github.com/pingcap/ticdc/cdc/entry" @@ -25,10 +23,6 @@ import ( "github.com/prometheus/client_golang/prometheus" ) -const ( - defaultMetricInterval = time.Second * 15 -) - var registry = prometheus.NewRegistry() func init() { diff --git a/cdc/processor.go b/cdc/processor.go index ccf997211de..c2daa42d938 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -52,12 +52,6 @@ import ( ) const ( - // TODO: processor output chan size, the accumulated data is determined by - // the count of sorted data and unmounted data. In current benchmark a single - // processor can reach 50k-100k QPS, and accumulated data is around - // 200k-400k in most cases. We need a better chan cache mechanism. - defaultOutputChanSize = 1280000 - // defaultMemBufferCapacity is the default memory buffer per change feed. defaultMemBufferCapacity int64 = 10 * 1024 * 1024 * 1024 // 10G diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 98e279812c0..754ecbb0c0d 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -1,3 +1,16 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package sink import ( @@ -14,6 +27,7 @@ import ( "github.com/pingcap/ticdc/cdc/model" ) +// Manager manages table sinks, maintains the relationship between table sinks and backendSink type Manager struct { backendSink Sink checkpointTs model.Ts @@ -22,6 +36,7 @@ type Manager struct { flushMu sync.Mutex } +// NewManager creates a new Sink manager func NewManager(backendSink Sink, checkpointTs model.Ts) *Manager { return &Manager{ backendSink: backendSink, @@ -30,6 +45,7 @@ func NewManager(backendSink Sink, checkpointTs model.Ts) *Manager { } } +// CreateTableSink creates a table sink func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts) Sink { if _, exist := m.tableSinks[tableID]; exist { log.Panic("the table sink already exists", zap.Uint64("tableID", uint64(tableID))) @@ -46,6 +62,7 @@ func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts) return sink } +// Close closes the Sink manager and backend Sink func (m *Manager) Close() error { return m.backendSink.Close() } diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 6a806c63fc2..f2ed50cee97 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -1,3 +1,16 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package sink import ( diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 6072d63128f..1fc01f4b935 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -135,6 +135,7 @@ def tests(sink_type, node_label) { run_integration_test(step_name, case_names.join(" ")) } } + parallel test_cases } } From 0b2b11bf2d2d3f34a3488ee2956e50cdc2c8003f Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 12:08:39 +0800 Subject: [PATCH 17/44] fix check --- cdc/sink/manager_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index f2ed50cee97..55d1d6aebbc 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -18,11 +18,11 @@ import ( "math/rand" "sync" - "github.com/pingcap/log" - "go.uber.org/zap" - "github.com/pingcap/check" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/util/testleak" + "go.uber.org/zap" ) type managerSuite struct{} @@ -82,6 +82,7 @@ func (c *checkSink) Close() error { } func (s *managerSuite) TestManagerRandom(c *check.C) { + defer testleak.AfterTest(c)() manager := NewManager(&checkSink{C: c}, 0) goroutineNum := 10 rowNum := 100 From 5d5900f559b640cecb65f12045caa64ce06f92e4 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 12:22:24 +0800 Subject: [PATCH 18/44] refine fmt --- cdc/processor.go | 8 +++----- cdc/sink/manager_test.go | 9 +++++---- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index c2daa42d938..2f7bd70e379 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -595,11 +595,9 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) util.ZapFieldChangefeed(ctx), zap.Uint64("checkpointTs", checkpointTs)) opt.BoundaryTs = checkpointTs - if checkpointTs <= p.position.CheckPointTs { - tablesToRemove = append(tablesToRemove, tableID) - opt.Done = true - opt.Status = model.OperFinished - } + tablesToRemove = append(tablesToRemove, tableID) + opt.Done = true + opt.Status = model.OperFinished status.Dirty = true } } else { diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 55d1d6aebbc..586ac4dca1b 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -37,7 +37,7 @@ type checkSink struct { } func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - panic("implement me") + panic("unreachable") } func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { @@ -51,7 +51,7 @@ func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row } func (c *checkSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { - panic("implement me") + panic("unreachable") } func (c *checkSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { @@ -74,16 +74,17 @@ func (c *checkSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 } func (c *checkSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { - panic("implement me") + panic("unreachable") } func (c *checkSink) Close() error { - panic("implement me") + return nil } func (s *managerSuite) TestManagerRandom(c *check.C) { defer testleak.AfterTest(c)() manager := NewManager(&checkSink{C: c}, 0) + defer manager.Close() goroutineNum := 10 rowNum := 100 var wg sync.WaitGroup From 9f4b935790c2fe550505e875982173a05e87d8bf Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 15:02:20 +0800 Subject: [PATCH 19/44] fix test --- cdc/processor.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cdc/processor.go b/cdc/processor.go index 2f7bd70e379..0116ac64554 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1005,6 +1005,9 @@ func (p *processor) sorterConsume( } if pEvent.RawKV != nil && pEvent.RawKV.OpType == model.OpTypeResolved { + if pEvent.CRTs == 0 { + continue + } err := flushRowChangedEvents() if err != nil { p.errCh <- errors.Trace(err) From 3d03fb54bd7858d0deac41dd06290ad3fb2befad Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 15:45:41 +0800 Subject: [PATCH 20/44] fix test --- cdc/processor.go | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index 0116ac64554..8ff0e9662de 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -977,7 +977,9 @@ func (p *processor) sorterConsume( globalResolvedTsReceiver, err := p.globalResolvedTsNotifier.NewReceiver(1 * time.Second) if err != nil { - p.errCh <- errors.Trace(err) + if errors.Cause(err) != context.Canceled { + p.errCh <- errors.Trace(err) + } return } defer globalResolvedTsReceiver.Stop() @@ -1010,7 +1012,9 @@ func (p *processor) sorterConsume( } err := flushRowChangedEvents() if err != nil { - p.errCh <- errors.Trace(err) + if errors.Cause(err) != context.Canceled { + p.errCh <- errors.Trace(err) + } return } atomic.StoreUint64(pResolvedTs, pEvent.CRTs) @@ -1038,7 +1042,9 @@ func (p *processor) sorterConsume( }) err := processRowChangedEvent(pEvent) if err != nil { - p.errCh <- errors.Trace(err) + if errors.Cause(err) != context.Canceled { + p.errCh <- errors.Trace(err) + } return } case <-globalResolvedTsReceiver.C: @@ -1057,7 +1063,9 @@ func (p *processor) sorterConsume( checkpointTs, err := sink.FlushRowChangedEvents(ctx, minTs) if err != nil { - p.errCh <- errors.Trace(err) + if errors.Cause(err) != context.Canceled { + p.errCh <- errors.Trace(err) + } return } log.Info("LEOPPRO: show checkpointTs after sink", zap.Uint64("resolved", minTs), zap.Uint64("checkpointTs", checkpointTs)) From 00f59c6bbf82df513f53335ad69ebef132a99d5b Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 17:12:45 +0800 Subject: [PATCH 21/44] add debug code --- cdc/sink/mq.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 42027d9550f..a6cef3fce93 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -166,6 +166,7 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha continue } partition := k.dispatcher.Dispatch(row) + log.Debug("MQ Sink: received row changed event", zap.Int32("partition", partition), zap.Reflect("row", row)) select { case <-ctx.Done(): return ctx.Err() From 2ccdc94825cbb2abeee7260c5af109d4d22f91a9 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 7 Jan 2021 17:27:24 +0800 Subject: [PATCH 22/44] add debug code --- cdc/processor.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cdc/processor.go b/cdc/processor.go index 8ff0e9662de..681608accd0 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -593,7 +593,11 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) checkpointTs := table.loadCheckpointTs() log.Debug("stop table", zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx), + zap.Any("opt", opt), zap.Uint64("checkpointTs", checkpointTs)) + if opt.BoundaryTs != p.position.CheckPointTs { + log.Panic("panic") + } opt.BoundaryTs = checkpointTs tablesToRemove = append(tablesToRemove, tableID) opt.Done = true From 7d684de7a9836a5cbcf742e18d24e57672d699ab Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 11:00:15 +0800 Subject: [PATCH 23/44] update test --- .../jenkins_ci/integration_test_common.groovy | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 1fc01f4b935..563116a9fb3 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -120,22 +120,24 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() - def step_cases = [] - def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) - for(int i in 1..CONCURRENT_NUMBER) { - def end = i*step_length-1 - if (i == CONCURRENT_NUMBER){ - end = cases_name.size()-1 +// def step_cases = [] +// def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) +// for(int i in 1..CONCURRENT_NUMBER) { +// def end = i*step_length-1 +// if (i == CONCURRENT_NUMBER){ +// end = cases_name.size()-1 +// } +// step_cases.add(cases_name[(i-1)*step_length..end]) +// } +// step_cases.eachWithIndex{ case_names, index -> +// def step_name = "step_${index}" +// test_cases["integration test ${step_name}"] = { +// run_integration_test(step_name, case_names.join(" ")) +// } +// } +test_cases["integration test 1"] = { + run_integration_test("step 1", "move_table") } - step_cases.add(cases_name[(i-1)*step_length..end]) - } - step_cases.eachWithIndex{ case_names, index -> - def step_name = "step_${index}" - test_cases["integration test ${step_name}"] = { - run_integration_test(step_name, case_names.join(" ")) - } - } - parallel test_cases } } From c49ea60d6b0f47b0a3c7e797e6cfa25b55cff10c Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 11:40:22 +0800 Subject: [PATCH 24/44] update test --- .../jenkins_ci/integration_test_common.groovy | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 563116a9fb3..6072d63128f 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -120,24 +120,21 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() -// def step_cases = [] -// def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) -// for(int i in 1..CONCURRENT_NUMBER) { -// def end = i*step_length-1 -// if (i == CONCURRENT_NUMBER){ -// end = cases_name.size()-1 -// } -// step_cases.add(cases_name[(i-1)*step_length..end]) -// } -// step_cases.eachWithIndex{ case_names, index -> -// def step_name = "step_${index}" -// test_cases["integration test ${step_name}"] = { -// run_integration_test(step_name, case_names.join(" ")) -// } -// } -test_cases["integration test 1"] = { - run_integration_test("step 1", "move_table") + def step_cases = [] + def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) + for(int i in 1..CONCURRENT_NUMBER) { + def end = i*step_length-1 + if (i == CONCURRENT_NUMBER){ + end = cases_name.size()-1 } + step_cases.add(cases_name[(i-1)*step_length..end]) + } + step_cases.eachWithIndex{ case_names, index -> + def step_name = "step_${index}" + test_cases["integration test ${step_name}"] = { + run_integration_test(step_name, case_names.join(" ")) + } + } parallel test_cases } } From 1d51a223e55907d22bf795697c836388818aacd4 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 12:21:09 +0800 Subject: [PATCH 25/44] fix data race --- cdc/sink/manager.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 754ecbb0c0d..ef8fbcb35d8 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -33,7 +33,6 @@ type Manager struct { checkpointTs model.Ts tableSinks map[model.TableID]*tableSink tableSinksMu sync.Mutex - flushMu sync.Mutex } // NewManager creates a new Sink manager @@ -72,8 +71,6 @@ func (m *Manager) getMinEmittedTs() model.Ts { return m.getCheckpointTs() } minTs := model.Ts(math.MaxUint64) - m.tableSinksMu.Lock() - defer m.tableSinksMu.Unlock() for _, tableSink := range m.tableSinks { emittedTs := tableSink.getEmittedTs() if minTs > emittedTs { @@ -84,8 +81,6 @@ func (m *Manager) getMinEmittedTs() model.Ts { } func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { - m.flushMu.Lock() - defer m.flushMu.Unlock() minEmittedTs := m.getMinEmittedTs() checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs) if err != nil { @@ -133,10 +128,15 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 }) if i == 0 { atomic.StoreUint64(&t.emittedTs, resolvedTs) + t.manager.tableSinksMu.Lock() + defer t.manager.tableSinksMu.Unlock() return t.manager.flushBackendSink(ctx) } resolvedRows := t.buffer[:i] t.buffer = t.buffer[i:] + + t.manager.tableSinksMu.Lock() + defer t.manager.tableSinksMu.Unlock() err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...) if err != nil { return t.manager.getCheckpointTs(), errors.Trace(err) From 8ff15f17762d95cfbdb46e7a2ce4044b6129890d Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 12:47:15 +0800 Subject: [PATCH 26/44] remove debug code --- cdc/changefeed.go | 14 ++++++-------- cdc/processor.go | 2 -- cdc/sink/mq.go | 1 - scripts/jenkins_ci/integration_test_common.groovy | 1 + 4 files changed, 7 insertions(+), 11 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index bfb4ec80d29..465ff78ac45 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -882,18 +882,16 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { tsUpdated = true } - updateCheckpointTs := minCheckpointTs if minCheckpointTs > c.status.CheckpointTs { c.status.CheckpointTs = minCheckpointTs // when the `c.ddlState` is `model.ChangeFeedWaitToExecDDL`, // some DDL is waiting to executed, we can't ensure whether the DDL has been executed. // so we can't emit checkpoint to sink - if c.ddlState == model.ChangeFeedWaitToExecDDL { - updateCheckpointTs-- - } - err := c.sink.EmitCheckpointTs(ctx, updateCheckpointTs) - if err != nil { - return errors.Trace(err) + if c.ddlState != model.ChangeFeedWaitToExecDDL { + err := c.sink.EmitCheckpointTs(ctx, minCheckpointTs) + if err != nil { + return errors.Trace(err) + } } tsUpdated = true } @@ -901,7 +899,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if tsUpdated { log.Debug("update changefeed", zap.String("id", c.id), - zap.Uint64("checkpoint ts", updateCheckpointTs), + zap.Uint64("checkpoint ts", c.status.CheckpointTs), zap.Uint64("resolved ts", c.status.ResolvedTs)) } return nil diff --git a/cdc/processor.go b/cdc/processor.go index 0d5fe24e3ca..77e416672cf 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1023,7 +1023,6 @@ func (p *processor) sorterConsume( } atomic.StoreUint64(pResolvedTs, pEvent.CRTs) lastResolvedTs = pEvent.CRTs - log.Info("[LEOPPRO] show table resolvedts", zap.Int64("tableID", tableID), zap.Uint64("resolvedTs", lastResolvedTs)) p.localResolvedNotifier.Notify() resolvedTsGauge.Set(float64(oracle.ExtractPhysical(pEvent.CRTs))) if !opDone { @@ -1072,7 +1071,6 @@ func (p *processor) sorterConsume( } return } - log.Info("LEOPPRO: show checkpointTs after sink", zap.Uint64("resolved", minTs), zap.Uint64("checkpointTs", checkpointTs)) if checkpointTs != 0 { atomic.StoreUint64(pCheckpointTs, checkpointTs) p.localCheckpointTsNotifier.Notify() diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index a6cef3fce93..42027d9550f 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -166,7 +166,6 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha continue } partition := k.dispatcher.Dispatch(row) - log.Debug("MQ Sink: received row changed event", zap.Int32("partition", partition), zap.Reflect("row", row)) select { case <-ctx.Done(): return ctx.Err() diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 6072d63128f..1fc01f4b935 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -135,6 +135,7 @@ def tests(sink_type, node_label) { run_integration_test(step_name, case_names.join(" ")) } } + parallel test_cases } } From f337bcf0daa3481b03dff224afc938fcc5645cc5 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 14:09:49 +0800 Subject: [PATCH 27/44] update test script --- .../jenkins_ci/integration_test_common.groovy | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 1fc01f4b935..e369d2668ff 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -120,20 +120,20 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() - def step_cases = [] - def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) - for(int i in 1..CONCURRENT_NUMBER) { - def end = i*step_length-1 - if (i == CONCURRENT_NUMBER){ - end = cases_name.size()-1 - } - step_cases.add(cases_name[(i-1)*step_length..end]) + + test_cases["integration test step_1"] = { + run_integration_test("step_1", case_names.join("ddl_puller_lag processor_panic split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table")) } - step_cases.eachWithIndex{ case_names, index -> - def step_name = "step_${index}" - test_cases["integration test ${step_name}"] = { - run_integration_test(step_name, case_names.join(" ")) - } + + test_cases["integration test step_2"] = { + run_integration_test("step_2", case_names.join("split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table")) + } + + test_cases["integration test step_3"] = { + run_integration_test("step_3", case_names.join("changefeed_pause_resume kafka_messages move_table")) + } + test_cases["integration test step_4"] = { + run_integration_test("step_4", case_names.join("move_table")) } parallel test_cases From f17c6822fe2363cb8535484ecf6dd00e73d81661 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 14:30:29 +0800 Subject: [PATCH 28/44] update test script --- scripts/jenkins_ci/integration_test_common.groovy | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index e369d2668ff..f22ddda62ab 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -122,18 +122,18 @@ def tests(sink_type, node_label) { test_cases["integration test step_1"] = { - run_integration_test("step_1", case_names.join("ddl_puller_lag processor_panic split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table")) + run_integration_test("step_1", "ddl_puller_lag processor_panic split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table") } test_cases["integration test step_2"] = { - run_integration_test("step_2", case_names.join("split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table")) + run_integration_test("step_2", "split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table") } test_cases["integration test step_3"] = { - run_integration_test("step_3", case_names.join("changefeed_pause_resume kafka_messages move_table")) + run_integration_test("step_3", "changefeed_pause_resume kafka_messages move_table") } test_cases["integration test step_4"] = { - run_integration_test("step_4", case_names.join("move_table")) + run_integration_test("step_4", "move_table") } parallel test_cases From 5077336e0187bbce4c29fe2514d2dce10bea1d94 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 15:02:54 +0800 Subject: [PATCH 29/44] update test script --- scripts/jenkins_ci/integration_test_common.groovy | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index f22ddda62ab..b38c3e8e871 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -120,20 +120,16 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() - - test_cases["integration test step_1"] = { - run_integration_test("step_1", "ddl_puller_lag processor_panic split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table") - } - test_cases["integration test step_2"] = { - run_integration_test("step_2", "split_region changefeed_auto_stop changefeed_pause_resume kafka_messages move_table") + run_integration_test("step_2", "processor_panic move_table") } test_cases["integration test step_3"] = { - run_integration_test("step_3", "changefeed_pause_resume kafka_messages move_table") + run_integration_test("step_3", "ddl_puller_lag move_table") } + test_cases["integration test step_4"] = { - run_integration_test("step_4", "move_table") + run_integration_test("step_4", "ddl_puller_lag processor_panic move_table") } parallel test_cases From b38f39026034d1277799416f1d51aaff476c6c0a Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 15:58:18 +0800 Subject: [PATCH 30/44] update test script --- cdc/sink/mq.go | 1 + .../jenkins_ci/integration_test_common.groovy | 23 +++---------------- 2 files changed, 4 insertions(+), 20 deletions(-) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 42027d9550f..4c335add7c5 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -166,6 +166,7 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha continue } partition := k.dispatcher.Dispatch(row) + log.Debug("LEOPPRO show row in mq sink", zap.Int32("partition", partition), zap.Reflect("row", row)) select { case <-ctx.Done(): return ctx.Err() diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index b38c3e8e871..f77b2e69dc5 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -96,15 +96,7 @@ def tests(sink_type, node_label) { tail /tmp/tidb_cdc_test/cov* || true """ } catch (Exception e) { - sh """ - echo "print all log" - for log in `ls /tmp/tidb_cdc_test/*/*.log`; do - echo "____________________________________" - echo "\$log" - cat "\$log" - echo "____________________________________" - done - """ + archiveArtifacts artifacts: '/tmp/tidb_cdc_test/*/*.log', fingerprint: true throw e; } } @@ -120,18 +112,9 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() - test_cases["integration test step_2"] = { - run_integration_test("step_2", "processor_panic move_table") - } - - test_cases["integration test step_3"] = { - run_integration_test("step_3", "ddl_puller_lag move_table") + test_cases["integration test step_1"] = { + run_integration_test("step_1", "processor_panic move_table") } - - test_cases["integration test step_4"] = { - run_integration_test("step_4", "ddl_puller_lag processor_panic move_table") - } - parallel test_cases } } From b1c0209cb10094347e5f63d6a57b435fea2fd2b0 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 16:15:58 +0800 Subject: [PATCH 31/44] update test script --- scripts/jenkins_ci/integration_test_common.groovy | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index f77b2e69dc5..025c3b4d37f 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -96,7 +96,9 @@ def tests(sink_type, node_label) { tail /tmp/tidb_cdc_test/cov* || true """ } catch (Exception e) { - archiveArtifacts artifacts: '/tmp/tidb_cdc_test/*/*.log', fingerprint: true + dir("/tmp/tidb_cdc_test/") { + archiveArtifacts artifacts: '**/*.log' + } throw e; } } From bea1477a0f6d19f908654c9f46b177ff3cdbc6d9 Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 16:54:48 +0800 Subject: [PATCH 32/44] fix move table --- cdc/changefeed.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 465ff78ac45..dff08a7ed67 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -518,13 +518,13 @@ func (c *changeFeed) handleMoveTableJobs(ctx context.Context, captures map[model log.Warn("ignored the move job, the source capture is not found", zap.Reflect("job", job)) continue } - replicaInfo, exist := status.RemoveTable(tableID, c.status.CheckpointTs) + replicaInfo, exist := status.RemoveTable(tableID, c.status.ResolvedTs) if !exist { delete(c.moveTableJobs, tableID) log.Warn("ignored the move job, the table is not exist in the source capture", zap.Reflect("job", job)) continue } - replicaInfo.StartTs = c.status.CheckpointTs + replicaInfo.StartTs = c.status.ResolvedTs job.TableReplicaInfo = replicaInfo job.Status = model.MoveTableStatusDeleted log.Info("handle the move job, remove table from the source capture", zap.Reflect("job", job)) @@ -532,14 +532,13 @@ func (c *changeFeed) handleMoveTableJobs(ctx context.Context, captures map[model // add table to target capture status, exist := cloneStatus(job.To) replicaInfo := job.TableReplicaInfo.Clone() - replicaInfo.StartTs = c.status.CheckpointTs if !exist { // the target capture is not exist, add table to orphanTables. c.orphanTables[tableID] = replicaInfo.StartTs log.Warn("the target capture is not exist, sent the table to orphanTables", zap.Reflect("job", job)) continue } - status.AddTable(tableID, replicaInfo, c.status.CheckpointTs) + status.AddTable(tableID, replicaInfo, replicaInfo.StartTs) job.Status = model.MoveTableStatusFinished delete(c.moveTableJobs, tableID) log.Info("handle the move job, add table to the target capture", zap.Reflect("job", job)) From c750ee30e5769f8d6a492242ea5a1a7d684fcd1c Mon Sep 17 00:00:00 2001 From: leoppro Date: Fri, 8 Jan 2021 17:18:23 +0800 Subject: [PATCH 33/44] fix move table --- cdc/processor.go | 9 ++++-- .../jenkins_ci/integration_test_common.groovy | 29 +++++++++++++++---- 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index 77e416672cf..4b6ab6f7331 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -580,6 +580,10 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) } if opt.Delete { if opt.BoundaryTs <= p.position.CheckPointTs { + if opt.BoundaryTs != p.position.CheckPointTs { + log.Warn("the replication progresses beyond the BoundaryTs and duplicate data may be received by downstream", + zap.Uint64("local resolved TS", p.position.ResolvedTs), zap.Any("opt", opt)) + } table, exist := p.tables[tableID] if !exist { log.Warn("table which will be deleted is not found", @@ -595,9 +599,6 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) util.ZapFieldChangefeed(ctx), zap.Any("opt", opt), zap.Uint64("checkpointTs", checkpointTs)) - if opt.BoundaryTs != p.position.CheckPointTs { - log.Panic("panic") - } opt.BoundaryTs = checkpointTs tablesToRemove = append(tablesToRemove, tableID) opt.Done = true @@ -1056,6 +1057,8 @@ func (p *processor) sorterConsume( var minTs uint64 if localResolvedTs < globalResolvedTs { minTs = localResolvedTs + log.Warn("the local resolved ts is less than the global resolved ts", + zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) } else { minTs = globalResolvedTs } diff --git a/scripts/jenkins_ci/integration_test_common.groovy b/scripts/jenkins_ci/integration_test_common.groovy index 025c3b4d37f..1fc01f4b935 100644 --- a/scripts/jenkins_ci/integration_test_common.groovy +++ b/scripts/jenkins_ci/integration_test_common.groovy @@ -96,9 +96,15 @@ def tests(sink_type, node_label) { tail /tmp/tidb_cdc_test/cov* || true """ } catch (Exception e) { - dir("/tmp/tidb_cdc_test/") { - archiveArtifacts artifacts: '**/*.log' - } + sh """ + echo "print all log" + for log in `ls /tmp/tidb_cdc_test/*/*.log`; do + echo "____________________________________" + echo "\$log" + cat "\$log" + echo "____________________________________" + done + """ throw e; } } @@ -114,9 +120,22 @@ def tests(sink_type, node_label) { returnStdout: true ).trim().split() - test_cases["integration test step_1"] = { - run_integration_test("step_1", "processor_panic move_table") + def step_cases = [] + def step_length = (int)(cases_name.size() / CONCURRENT_NUMBER + 0.5) + for(int i in 1..CONCURRENT_NUMBER) { + def end = i*step_length-1 + if (i == CONCURRENT_NUMBER){ + end = cases_name.size()-1 + } + step_cases.add(cases_name[(i-1)*step_length..end]) } + step_cases.eachWithIndex{ case_names, index -> + def step_name = "step_${index}" + test_cases["integration test ${step_name}"] = { + run_integration_test(step_name, case_names.join(" ")) + } + } + parallel test_cases } } From 79cef8785275e8a7dce9031f16591c1891083903 Mon Sep 17 00:00:00 2001 From: leoppro Date: Mon, 11 Jan 2021 16:36:40 +0800 Subject: [PATCH 34/44] accepting comments --- cdc/changefeed.go | 3 ++ cdc/processor.go | 2 +- cdc/sink/manager.go | 94 +++++++++++++++++++++++++++++++++++++--- cdc/sink/manager_test.go | 12 ++++- 4 files changed, 103 insertions(+), 8 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index dff08a7ed67..16d5aec0ef8 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -518,6 +518,9 @@ func (c *changeFeed) handleMoveTableJobs(ctx context.Context, captures map[model log.Warn("ignored the move job, the source capture is not found", zap.Reflect("job", job)) continue } + // To ensure that the replication pipeline stops exactly at the boundary TS, + // The boundary TS specified by Remove Table Operation MUST greater or equal to the checkpoint TS of this table. + // So the global resolved TS is a reasonable values. replicaInfo, exist := status.RemoveTable(tableID, c.status.ResolvedTs) if !exist { delete(c.moveTableJobs, tableID) diff --git a/cdc/processor.go b/cdc/processor.go index 4b6ab6f7331..1906427dcd9 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1179,7 +1179,7 @@ func runProcessor( cancel() return nil, errors.Trace(err) } - sinkManager := sink.NewManager(s, checkpointTs) + sinkManager := sink.NewManager(ctx, s, errCh, checkpointTs) processor, err := newProcessor(ctx, pdCli, credential, session, info, sinkManager, changefeedID, captureInfo, checkpointTs, errCh, flushCheckpointInterval) if err != nil { diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index ef8fbcb35d8..8ba0d17298f 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -27,6 +27,14 @@ import ( "github.com/pingcap/ticdc/cdc/model" ) +const ( + // TODO: buffer chan size, the accumulated data is determined by + // the count of sorted data and unmounted data. In current benchmark a single + // processor can reach 50k-100k QPS, and accumulated data is around + // 200k-400k in most cases. We need a better chan cache mechanism. + defaultBufferChanSize = 1280000 +) + // Manager manages table sinks, maintains the relationship between table sinks and backendSink type Manager struct { backendSink Sink @@ -36,9 +44,9 @@ type Manager struct { } // NewManager creates a new Sink manager -func NewManager(backendSink Sink, checkpointTs model.Ts) *Manager { +func NewManager(ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts) *Manager { return &Manager{ - backendSink: backendSink, + backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs), checkpointTs: checkpointTs, tableSinks: make(map[model.TableID]*tableSink), } @@ -67,6 +75,8 @@ func (m *Manager) Close() error { } func (m *Manager) getMinEmittedTs() model.Ts { + m.tableSinksMu.Lock() + defer m.tableSinksMu.Unlock() if len(m.tableSinks) == 0 { return m.getCheckpointTs() } @@ -128,15 +138,11 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 }) if i == 0 { atomic.StoreUint64(&t.emittedTs, resolvedTs) - t.manager.tableSinksMu.Lock() - defer t.manager.tableSinksMu.Unlock() return t.manager.flushBackendSink(ctx) } resolvedRows := t.buffer[:i] t.buffer = t.buffer[i:] - t.manager.tableSinksMu.Lock() - defer t.manager.tableSinksMu.Unlock() err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...) if err != nil { return t.manager.getCheckpointTs(), errors.Trace(err) @@ -158,3 +164,79 @@ func (t *tableSink) Close() error { t.manager.destroyTableSink(t.tableID) return nil } + +type bufferSink struct { + Sink + buffer chan struct { + rows []*model.RowChangedEvent + resolved bool + resolvedTs model.Ts + } + checkpointTs uint64 +} + +func newBufferSink(ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts) Sink { + sink := &bufferSink{ + Sink: backendSink, + buffer: make(chan struct { + rows []*model.RowChangedEvent + resolved bool + resolvedTs model.Ts + }, defaultBufferChanSize), + checkpointTs: checkpointTs, + } + go sink.run(ctx, errCh) + return sink +} + +func (b *bufferSink) run(ctx context.Context, errCh chan error) { + for { + select { + case <-ctx.Done(): + err := ctx.Err() + if errors.Cause(err) != context.Canceled { + errCh <- err + } + return + case e := <-b.buffer: + if e.resolved { + checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, e.resolvedTs) + if errors.Cause(err) != context.Canceled { + errCh <- err + } + atomic.StoreUint64(&b.checkpointTs, checkpointTs) + continue + } + err := b.Sink.EmitRowChangedEvents(ctx, e.rows...) + if errors.Cause(err) != context.Canceled { + errCh <- err + } + } + } +} + +func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + select { + case <-ctx.Done(): + return ctx.Err() + case b.buffer <- struct { + rows []*model.RowChangedEvent + resolved bool + resolvedTs model.Ts + }{rows: rows}: + } + return nil +} + +func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + select { + case <-ctx.Done(): + return atomic.LoadUint64(&b.checkpointTs), ctx.Err() + case b.buffer <- struct { + rows []*model.RowChangedEvent + resolved bool + resolvedTs model.Ts + }{resolved: true, resolvedTs: resolvedTs}: + } + return atomic.LoadUint64(&b.checkpointTs), nil +} diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 586ac4dca1b..3418404621f 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -17,6 +17,7 @@ import ( "context" "math/rand" "sync" + "time" "github.com/pingcap/check" "github.com/pingcap/log" @@ -83,7 +84,10 @@ func (c *checkSink) Close() error { func (s *managerSuite) TestManagerRandom(c *check.C) { defer testleak.AfterTest(c)() - manager := NewManager(&checkSink{C: c}, 0) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + errCh := make(chan error, 16) + manager := NewManager(ctx, &checkSink{C: c}, errCh, 0) defer manager.Close() goroutineNum := 10 rowNum := 100 @@ -119,4 +123,10 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { }() } wg.Wait() + cancel() + time.Sleep(1 * time.Second) + close(errCh) + for err := range errCh { + c.Assert(err, check.IsNil) + } } From e4574769671b85556cb7e5996151c72cc4a47200 Mon Sep 17 00:00:00 2001 From: leoppro Date: Mon, 11 Jan 2021 17:09:25 +0800 Subject: [PATCH 35/44] add debug log --- cdc/processor.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cdc/processor.go b/cdc/processor.go index 1906427dcd9..eb879317368 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1193,6 +1193,7 @@ func runProcessor( go func() { err := <-errCh + log.Debug("processor exited by error", zap.Error(err)) cause := errors.Cause(err) if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Inc() From 76e64979d3b52e59071d607b9005adf876120597 Mon Sep 17 00:00:00 2001 From: leoppro Date: Mon, 11 Jan 2021 17:30:34 +0800 Subject: [PATCH 36/44] fix cancel --- cdc/sink/manager.go | 23 +++++++++++++++-------- cdc/sink/mq.go | 1 - 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 8ba0d17298f..0c96eb5fc3f 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -111,9 +111,10 @@ func (m *Manager) getCheckpointTs() uint64 { } type tableSink struct { - tableID model.TableID - manager *Manager - buffer []*model.RowChangedEvent + tableID model.TableID + manager *Manager + buffer []*model.RowChangedEvent + // emittedTs means all of events which of commitTs less than or equal to emittedTs is sent to backendSink emittedTs model.Ts } @@ -194,22 +195,28 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { select { case <-ctx.Done(): err := ctx.Err() - if errors.Cause(err) != context.Canceled { + if err != nil && errors.Cause(err) != context.Canceled { errCh <- err } return case e := <-b.buffer: if e.resolved { checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, e.resolvedTs) - if errors.Cause(err) != context.Canceled { - errCh <- err + if err != nil { + if errors.Cause(err) != context.Canceled { + errCh <- err + } + return } atomic.StoreUint64(&b.checkpointTs, checkpointTs) continue } err := b.Sink.EmitRowChangedEvents(ctx, e.rows...) - if errors.Cause(err) != context.Canceled { - errCh <- err + if err != nil { + if errors.Cause(err) != context.Canceled { + errCh <- err + } + return } } } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 4c335add7c5..42027d9550f 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -166,7 +166,6 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha continue } partition := k.dispatcher.Dispatch(row) - log.Debug("LEOPPRO show row in mq sink", zap.Int32("partition", partition), zap.Reflect("row", row)) select { case <-ctx.Done(): return ctx.Err() From f50b314e7bdceac5a9e61ee66c7c4626a7984bd6 Mon Sep 17 00:00:00 2001 From: leoppro Date: Mon, 11 Jan 2021 21:49:25 +0800 Subject: [PATCH 37/44] fix build --- cdc/changefeed.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 5ba267370d1..b3930947ba3 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -20,8 +20,6 @@ import ( "sync" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" From ad6ad57bedeac47f3d11b1ae46bb08ade1a648d0 Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 12 Jan 2021 11:33:16 +0800 Subject: [PATCH 38/44] update data-flow --- docs/data-flow.dot | 39 +++-- docs/data-flow.svg | 425 +++++++++++++++++++++++++-------------------- 2 files changed, 261 insertions(+), 203 deletions(-) diff --git a/docs/data-flow.dot b/docs/data-flow.dot index f2afa7bcc22..9a32651ad5c 100644 --- a/docs/data-flow.dot +++ b/docs/data-flow.dot @@ -104,10 +104,9 @@ digraph Dataflow { processor_sorter_consume [label = "sorterConsume"] processor_puller_consume [label = "pullerConsume"] - processor_sync_resolved [label = "syncResolved"] - processor_sorter_consume -> processor_sync_resolved [label = "P.output"] } + processor_sorter_consume -> table_sink_run [label = "P.output"] subgraph cluster_puller { label = "Puller"; @@ -132,18 +131,11 @@ digraph Dataflow { sorter_run [label = "Run"] } - sorter_run -> rectifier_run [label = "E.Output"] - - subgraph cluster_rectifier { - label = "Rectifier"; - - rectifier_run [label = "Run"] - } } - rectifier_run -> processor_sorter_consume [label = "R.Ouput"] + sorter_run -> processor_sorter_consume [label = "S.Ouput"] processor_sorter_consume -> mounter_run [label = "M.Input"] - mounter_run -> processor_sync_resolved [ + mounter_run -> table_sink_run [ label = "wait unmarshal", style = "dashed", dir = back, @@ -156,13 +148,30 @@ digraph Dataflow { mounter_run [label = "Run"] } - processor_sync_resolved -> sink_run [label = "S.EmitEvents"] - subgraph cluster_sink { - label = "Sink"; + label = "Sink Manager"; style = filled; - sink_run [label = "FlushEvents"] + subgraph table_sink { + label = "Table Sink"; + + table_sink_run [label = "Table Sink\nFlushEvents"] + } + + subgraph buffer_sink { + label = "Buffer Sink"; + + buffer_sink_run [label = "Buffer Sink\nFlushEvents"] + } + + subgraph backend_sink { + label = "Backend Sink"; + + backend_sink_run [label = "Backend Sink\nFlushEvents"] + } + + table_sink_run -> buffer_sink_run [label = "S.EmitEvents"] + buffer_sink_run -> backend_sink_run [label = "S.EmitEvents"] } } diff --git a/docs/data-flow.svg b/docs/data-flow.svg index 5cb424b7bbd..2989e28ca3f 100644 --- a/docs/data-flow.svg +++ b/docs/data-flow.svg @@ -1,246 +1,295 @@ - - - + + Dataflow - -cluster_legends - -Legend - -cluster_tikv - -TiKV - -cluster_ticdc - -TiCDC - -cluster_kvclient - -KV client - -cluster_processor - -Processor - -cluster_puller - -Puller - -cluster_sorter - -Sorter - -cluster_entry_sorter - -Entry sorter - -cluster_rectifier - -Rectifier - -cluster_mounter - -Mounter - -cluster_sink - -Sink + + +cluster_legends + +Legend + + +cluster_tikv + +TiKV + + +cluster_ticdc + +TiCDC + + +cluster_kvclient + +KV client + + +cluster_processor + +Processor + + +cluster_puller + +Puller + + +cluster_sorter + +Sorter + + +cluster_entry_sorter + +Entry sorter + + +cluster_mounter + +Mounter + + +cluster_sink + +Sink Manager -flow - -Go routine #1 + +flow + +Go routine #1 -flow_ - -Go routine #2 + +flow_ + +Go routine #2 -flow->flow_ - - -Flow -direction -Channel -/Buffer + +flow->flow_ + + +Flow +direction +Channel +/Buffer -wait - -Go routine #1 + +wait + +Go routine #1 -wait_ - -Go routine #2 + +wait_ + +Go routine #2 -wait->wait_ - - -#1 wiats #2 + +wait->wait_ + + +#1 wiats #2 -Raftstore - -Raftstore + +Raftstore + +Raftstore -CDC - -CDC + +CDC + +CDC -Raftstore->CDC - - -channel + +Raftstore->CDC + + +channel -kv_client - -kv_client + +kv_client + +kv_client -CDC->kv_client - - -gRPC + +CDC->kv_client + + +gRPC -puller_run_step1 - -Run #1 -(Add to memory buffer) + +puller_run_step1 + +Run #1 +(Add to memory buffer) -kv_client->puller_run_step1 - - -eventCh + +kv_client->puller_run_step1 + + +eventCh -puller_run_step2 - -Run #2 -(Output to output channel) + +puller_run_step2 + +Run #2 +(Output to output channel) -puller_run_step1->puller_run_step2 - - -P.buffer + +puller_run_step1->puller_run_step2 + + +P.buffer -processor_sorter_consume - -sorterConsume - - -processor_sync_resolved - -syncResolved - - -processor_sorter_consume->processor_sync_resolved - - -P.output + +processor_sorter_consume + +sorterConsume + + + +table_sink_run + +Table Sink +FlushEvents + + + +processor_sorter_consume->table_sink_run + + +P.output -mounter_run - -Run + +mounter_run + +Run -processor_sorter_consume->mounter_run - - -M.Input + +processor_sorter_consume->mounter_run + + +M.Input -processor_puller_consume - -pullerConsume + +processor_puller_consume + +pullerConsume -sorter_run - -Run + +sorter_run + +Run -processor_puller_consume->sorter_run - - -E.AddEntry + +processor_puller_consume->sorter_run + + +E.AddEntry + + + +buffer_sink_run + +Buffer Sink +FlushEvents + + + +table_sink_run->buffer_sink_run + + +S.EmitEvents - -sink_run - -FlushEvents + + +puller_run_step2->processor_puller_consume + + +P.Output + + + +sorter_run->processor_sorter_consume + + +S.Ouput + + + +mounter_run->table_sink_run + + +wait unmarshal + + + +backend_sink_run + +Backend Sink +FlushEvents + + + +buffer_sink_run->backend_sink_run + + +S.EmitEvents - -processor_sync_resolved->sink_run - - -S.EmitEvents + + +data_in + +Data in - -puller_run_step2->processor_puller_consume - - -P.Output - - -rectifier_run - -Run - - -sorter_run->rectifier_run - - -E.Output - - -rectifier_run->processor_sorter_consume - - -R.Ouput - - -mounter_run->processor_sync_resolved - - -wait unmarshal + + +data_in->Raftstore + + -data_out - -Data out + +data_out + +Data out - -sink_run->data_out - - - - -data_in - -Data in + + +sink_run + +sink_run - -data_in->Raftstore - - + + +sink_run->data_out + + From 90de5ff0011a7a90c0fe259e4e797cc36159cb1d Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 12 Jan 2021 13:25:06 +0800 Subject: [PATCH 39/44] update metrics --- cdc/metrics_processor.go | 17 - cdc/processor.go | 9 - cdc/sink/manager.go | 18 + cdc/sink/metrics.go | 17 + metrics/grafana/ticdc.json | 9456 ++++++++++++++++++------------------ 5 files changed, 4761 insertions(+), 4756 deletions(-) diff --git a/cdc/metrics_processor.go b/cdc/metrics_processor.go index 480fa4f0c82..7cc422b0219 100644 --- a/cdc/metrics_processor.go +++ b/cdc/metrics_processor.go @@ -75,13 +75,6 @@ var ( Help: "The time it took to update sub changefeed info.", Buckets: prometheus.ExponentialBuckets(0.001 /* 1 ms */, 2, 18), }, []string{"capture"}) - tableOutputChanSizeGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "txn_output_chan_size", - Help: "size of row changed event output channel from table to processor", - }, []string{"changefeed", "capture"}) processorErrorCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "ticdc", @@ -89,14 +82,6 @@ var ( Name: "exit_with_error_count", Help: "counter for processor exits with error", }, []string{"changefeed", "capture"}) - sinkFlushRowChangedDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "flush_event_duration_seconds", - Help: "Bucketed histogram of processing time (s) of flushing events in processor", - Buckets: prometheus.ExponentialBuckets(0.002 /* 2ms */, 2, 20), - }, []string{"changefeed", "capture"}) ) // initProcessorMetrics registers all metrics used in processor @@ -109,7 +94,5 @@ func initProcessorMetrics(registry *prometheus.Registry) { registry.MustRegister(syncTableNumGauge) registry.MustRegister(txnCounter) registry.MustRegister(updateInfoDuration) - registry.MustRegister(tableOutputChanSizeGauge) registry.MustRegister(processorErrorCounter) - registry.MustRegister(sinkFlushRowChangedDuration) } diff --git a/cdc/processor.go b/cdc/processor.go index eb879317368..55c1f586679 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -909,7 +909,6 @@ func (p *processor) sorterConsume( var lastResolvedTs uint64 opDone := false resolvedTsGauge := tableResolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr, tableName) - metricFlushDuration := sinkFlushRowChangedDuration.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) checkDoneTicker := time.NewTicker(1 * time.Second) checkDone := func() { localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) @@ -1065,7 +1064,6 @@ func (p *processor) sorterConsume( if minTs == 0 || atomic.LoadUint64(&p.checkpointTs) == minTs { continue } - start := time.Now() checkpointTs, err := sink.FlushRowChangedEvents(ctx, minTs) if err != nil { @@ -1078,13 +1076,6 @@ func (p *processor) sorterConsume( atomic.StoreUint64(pCheckpointTs, checkpointTs) p.localCheckpointTsNotifier.Notify() } - - dur := time.Since(start) - metricFlushDuration.Observe(dur.Seconds()) - if dur > 3*time.Second { - log.Warn("flush row changed events too slow", - zap.Duration("duration", dur), util.ZapFieldChangefeed(ctx)) - } case <-checkDoneTicker.C: if !opDone { checkDone() diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 0c96eb5fc3f..ac8e3e26533 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -19,6 +19,9 @@ import ( "sort" "sync" "sync/atomic" + "time" + + "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/log" "go.uber.org/zap" @@ -33,6 +36,7 @@ const ( // processor can reach 50k-100k QPS, and accumulated data is around // 200k-400k in most cases. We need a better chan cache mechanism. defaultBufferChanSize = 1280000 + defaultMetricInterval = time.Second * 15 ) // Manager manages table sinks, maintains the relationship between table sinks and backendSink @@ -191,6 +195,10 @@ func newBufferSink(ctx context.Context, backendSink Sink, errCh chan error, chec } func (b *bufferSink) run(ctx context.Context, errCh chan error) { + changefeedID := util.ChangefeedIDFromCtx(ctx) + advertiseAddr := util.CaptureAddrFromCtx(ctx) + metricFlushDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID) + metricBufferSize := bufferChanSizeGauge.WithLabelValues(advertiseAddr, changefeedID) for { select { case <-ctx.Done(): @@ -201,6 +209,7 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { return case e := <-b.buffer: if e.resolved { + start := time.Now() checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, e.resolvedTs) if err != nil { if errors.Cause(err) != context.Canceled { @@ -209,6 +218,13 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { return } atomic.StoreUint64(&b.checkpointTs, checkpointTs) + + dur := time.Since(start) + metricFlushDuration.Observe(dur.Seconds()) + if dur > 3*time.Second { + log.Warn("flush row changed events too slow", + zap.Duration("duration", dur), util.ZapFieldChangefeed(ctx)) + } continue } err := b.Sink.EmitRowChangedEvents(ctx, e.rows...) @@ -218,6 +234,8 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { } return } + case <-time.After(defaultMetricInterval): + metricBufferSize.Set(float64(len(b.buffer))) } } } diff --git a/cdc/sink/metrics.go b/cdc/sink/metrics.go index 7aa91469e10..4d7e7a33954 100644 --- a/cdc/sink/metrics.go +++ b/cdc/sink/metrics.go @@ -70,6 +70,21 @@ var ( Name: "total_flushed_rows_count", Help: "totla count of flushed rows", }, []string{"capture", "changefeed"}) + flushRowChangedDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "flush_event_duration_seconds", + Help: "Bucketed histogram of processing time (s) of flushing events in processor", + Buckets: prometheus.ExponentialBuckets(0.002 /* 2ms */, 2, 20), + }, []string{"capture", "changefeed"}) + bufferChanSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "buffer_chan_size", + Help: "size of row changed event buffer channel in sink manager", + }, []string{"capture", "changefeed"}) ) // InitMetrics registers all metrics in this file @@ -81,4 +96,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(bucketSizeCounter) registry.MustRegister(totalRowsCountGauge) registry.MustRegister(totalFlushedRowsCountGauge) + registry.MustRegister(flushRowChangedDuration) + registry.MustRegister(bufferChanSizeGauge) } diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index bb818f69250..0f90ba79aa7 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -109,11 +109,11 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1608013159669, + "iteration": 1610428254319, "links": [], "panels": [ { - "collapsed": true, + "collapsed": false, "gridPos": { "h": 1, "w": 24, @@ -121,4842 +121,4838 @@ "y": 0 }, "id": 21, - "panels": [ + "panels": [], + "title": "Server", + "type": "row" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Uptime of TiCDC and TiKV", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 1 + }, + "hiddenSeries": false, + "id": 19, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Uptime of TiCDC and TiKV", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 1 - }, - "hiddenSeries": false, - "id": 19, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "(time() - process_start_time_seconds{job=\"ticdc\"})", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "TiCDC - {{instance}}", - "refId": "A" - }, - { - "expr": "(time() - process_start_time_seconds{job=\"tikv\"})", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "TiKV - {{instance}}", - "refId": "B" - } + "expr": "(time() - process_start_time_seconds{job=\"ticdc\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "TiCDC - {{instance}}", + "refId": "A" + }, + { + "expr": "(time() - process_start_time_seconds{job=\"tikv\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "TiKV - {{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Uptime", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dtdurations", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "Goroutine count of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 8, + "y": 1 + }, + "hiddenSeries": false, + "id": 26, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": " go_goroutines{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Goroutine count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The count of open FD count of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 14, + "y": 1 + }, + "hiddenSeries": false, + "id": 27, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "process_open_fds{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Open FD count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC cluster ownership status", + "fontSize": "100%", + "gridPos": { + "h": 7, + "w": 4, + "x": 20, + "y": 1 + }, + "hideTimeOverride": true, + "id": 113, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 2, + "desc": false + }, + "styles": [ + { + "alias": "Instance", + "colorMode": null, + "colors": [ + "#8AB8FF", + "#73BF69", + "rgba(245, 54, 54, 0.9)" ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Uptime", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dtdurations", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "link": false, + "mappingType": 2, + "pattern": "Current", + "preserveFormat": false, + "rangeMaps": [ + { + "from": "0.1", + "text": "Owner", + "to": "2" }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "from": "0", + "text": "Worker", + "to": "0.1" } ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Goroutine count of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 8, - "y": 1 - }, - "hiddenSeries": false, - "id": 26, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": " go_goroutines{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } + "sanitize": false, + "thresholds": [ + "0.1", + "2" ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Goroutine count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, + "type": "string", + "unit": "short", + "valueMaps": [ { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "text": "Owner", + "value": "1" } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + ] + } + ], + "targets": [ + { + "expr": "rate(ticdc_owner_ownership_counter[30s])", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "timeFrom": "1s", + "title": "Ownership", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "CPU usage of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 8 + }, + "hiddenSeries": false, + "id": 24, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "rate(process_cpu_seconds_total{job=\"ticdc\"}[1m])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CPU usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The count of open FD count of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 14, - "y": 1 - }, - "hiddenSeries": false, - "id": 27, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "process_open_fds{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } + "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_TEST-CLUSTER}", + "description": "Memory usage of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 8 + }, + "hiddenSeries": false, + "id": 23, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "process-{{instance}}", + "refId": "A" + }, + { + "expr": "go_memstats_heap_alloc_bytes{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "heap-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The history of TiCDC cluster ownership, owner node has a value that is great than 0", + "fill": 1, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 8 + }, + "id": 110, + "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", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": true, + "targets": [ + { + "expr": "sum(rate(ticdc_owner_ownership_counter[30s])) by (instance) > 0", + "format": "time_series", + "interval": "30s", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Ownership history", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 8 + }, + "hiddenSeries": false, + "id": 114, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p999-{{instance}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p99-{{instance}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p95-{{instance}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd health check duration", + "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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 15 + }, + "id": 11, + "panels": [], + "title": "Changefeed", + "type": "row" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of captured table of TiCDC nodes ", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 0, + "y": 16 + }, + "id": 4, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" ], + "decimals": 2, + "pattern": "/.*/", "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Open FD count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Changefeed table count", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "Internal resolved ts of TiCDC nodes", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 7, + "y": 16 + }, + "id": 90, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" ], - "yaxis": { - "align": false, - "alignLevel": null - } + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" }, { - "columns": [ - { - "text": "Current", - "value": "current" - } + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC cluster ownership status", - "fontSize": "100%", - "gridPos": { - "h": 7, - "w": 4, - "x": 20, - "y": 1 - }, - "hideTimeOverride": true, - "id": 113, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 2, - "desc": false - }, - "styles": [ - { - "alias": "Instance", - "colorMode": null, - "colors": [ - "#8AB8FF", - "#73BF69", - "rgba(245, 54, 54, 0.9)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "mappingType": 2, - "pattern": "Current", - "preserveFormat": false, - "rangeMaps": [ - { - "from": "0.1", - "text": "Owner", - "to": "2" - }, - { - "from": "0", - "text": "Worker", - "to": "0.1" - } - ], - "sanitize": false, - "thresholds": [ - "0.1", - "2" - ], - "type": "string", - "unit": "short", - "valueMaps": [ - { - "text": "Owner", - "value": "1" - } - ] - } + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + }, + { + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Processor resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "Internal resolved ts of captured tables", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 10, + "x": 14, + "y": 16 + }, + "id": 30, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" ], - "targets": [ - { - "expr": "rate(ticdc_owner_ownership_counter[30s])", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" - } + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" ], - "timeFrom": "1s", - "title": "Ownership", - "transform": "timeseries_aggregations", - "type": "table" + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "CPU usage of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 8 - }, - "hiddenSeries": false, - "id": 24, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "rate(process_cpu_seconds_total{job=\"ticdc\"}[1m])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CPU usage", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percentunit", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Table resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The checkpoint ts of changefeeds.", + "fill": 0, + "gridPos": { + "h": 7, + "w": 9, + "x": 0, + "y": 21 + }, + "id": 86, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/approximate current time.*/", + "bars": false + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_owner_checkpoint_ts{changefeed=~\"$changefeed\"}) by (changefeed) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Memory usage of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 8 - }, - "hiddenSeries": false, - "id": 23, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "process_resident_memory_bytes{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "process-{{instance}}", - "refId": "A" - }, - { - "expr": "go_memstats_heap_alloc_bytes{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "heap-{{instance}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Memory usage", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "max(pd_cluster_tso) * 1000", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "approximate current time (s)", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed checkpoint", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "max": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The history of TiCDC cluster ownership, owner node has a value that is great than 0", - "fill": 1, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 8 - }, - "id": 110, - "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", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": true, - "targets": [ - { - "expr": "sum(rate(ticdc_owner_ownership_counter[30s])) by (instance) > 0", - "format": "time_series", - "interval": "30s", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Ownership history", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Request count of etcd operation per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 9, + "y": 21 + }, + "hiddenSeries": false, + "id": 102, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (capture, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD etcd requests/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 8 - }, - "hiddenSeries": false, - "id": 114, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p999-{{instance}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p99-{{instance}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p95-{{instance}}", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Etcd health check duration", - "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": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "title": "Server", - "type": "row" + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "collapsed": true, + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of errors that interrupt changefeed per minute ", + "fill": 1, "gridPos": { - "h": 1, - "w": 24, + "h": 7, + "w": 7, + "x": 17, + "y": 21 + }, + "id": 82, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(delta(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Exit error count/m", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, "x": 0, - "y": 1 + "y": 28 }, - "id": 11, - "panels": [ + "hiddenSeries": false, + "id": 3, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of captured table of TiCDC nodes ", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 0, - "y": 2 - }, - "id": 4, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" - }, - { - "alias": "", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], - "targets": [ - { - "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Changefeed table count", - "transform": "timeseries_aggregations", - "type": "table" + "expr": "max(ticdc_owner_checkpoint_ts_lag{changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed 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": null, + "show": true }, { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of TiCDC nodes", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 7, - "y": 2 - }, - "id": 90, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "max(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - }, - { - "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "changefeed checkpoint", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Processor resolved ts", - "transform": "timeseries_aggregations", - "type": "table" + "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_TEST-CLUSTER}", + "description": "The lag between internal resolved ts and the latest ts of upstream TiDB.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 28 + }, + "hiddenSeries": false, + "id": 2, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor 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": null, + "show": true }, { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of captured tables", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 10, - "x": 14, - "y": 2 - }, - "id": 30, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "max(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", - "refId": "A" - }, - { - "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "changefeed checkpoint", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Table resolved ts", - "transform": "timeseries_aggregations", - "type": "table" + "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, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Sink write duration of changefeeds", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 35 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 94, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "repeat": null, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Sink write 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_TEST-CLUSTER}", + "description": "Percentiles of sink write duration of changefeeds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 35 + }, + "hiddenSeries": false, + "id": 35, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" }, { - "aliasColors": {}, - "bars": true, - "cacheTimeout": null, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The checkpoint ts of changefeeds.", - "fill": 0, - "gridPos": { - "h": 7, - "w": 9, - "x": 0, - "y": 7 - }, - "id": 86, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": null, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 2, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/approximate current time.*/", - "bars": false - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_owner_checkpoint_ts{changefeed=~\"$changefeed\"}) by (changefeed) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "max(pd_cluster_tso) * 1000", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "approximate current time (s)", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Changefeed checkpoint", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "max": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Request count of etcd operation per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 9, - "y": 7 - }, - "hiddenSeries": false, - "id": 102, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (capture, type)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "PD etcd requests/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt changefeed per minute ", - "fill": 1, - "gridPos": { - "h": 7, - "w": 7, - "x": 17, - "y": 7 - }, - "id": 82, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(delta(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Exit error count/m", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "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_TEST-CLUSTER}", + "description": "The number of changed rows that are written to downstream per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 42 + }, + "hiddenSeries": false, + "id": 34, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 14 - }, - "hiddenSeries": false, - "id": 3, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_owner_checkpoint_ts_lag{changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Changefeed 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": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write rows count/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The lag between internal resolved ts and the latest ts of upstream TiDB.", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 14 - }, - "hiddenSeries": false, - "id": 2, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Processor 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": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "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_TEST-CLUSTER}", + "description": "Percentiles of sink batch size", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 42 + }, + "hiddenSeries": false, + "id": 36, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p90", + "refId": "A" }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Sink write duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 21 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 94, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Sink write 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 + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "B" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 21 - }, - "hiddenSeries": false, - "id": 35, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write batch size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of changed rows that are written to downstream per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 28 - }, - "hiddenSeries": false, - "id": 34, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - }, - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write rows count/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "format": "none", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink batch size", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 28 - }, - "hiddenSeries": false, - "id": 36, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write batch size percentile", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "show": true - }, - { - "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, - "max": null, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Asynchronous flush sink duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 35 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 93, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_processor_flush_event_duration_seconds_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Flush sink 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_TEST-CLUSTER}", - "description": "Percentiles of asynchronous flush sink duration of changefeeds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 35 - }, - "hiddenSeries": false, - "id": 98, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Flush sink duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true - }, - { - "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, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of detecting and waiting conflict of MySQL sink", - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 42 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 103, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "MySQL sink conflict detect 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_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 42 - }, - "hiddenSeries": false, - "id": 83, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "MySQL sink conflict detect duration percentile", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 42 - }, - "hiddenSeries": false, - "id": 95, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": true, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", - "refId": "A" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total worker", - "refId": "B" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", - "refId": "C" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "2-10 row/s worker", - "refId": "D" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "10-100 row/s worker", - "refId": "E" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": ">100 row/s worker", - "refId": "F" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "MySQL sink worker load", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Changefeed", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 13, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of established Eventfeed RPC between TiCDC and TiKV", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 3 - }, - "hiddenSeries": false, - "id": 15, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "ticdc_kvclient_event_feed_count", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Eventfeed count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "decimals": null, - "description": "Percentiles of Eventfeed message size", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 6, - "y": 3 - }, - "hiddenSeries": false, - "id": 17, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Event size percentile", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, + "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, + "max": null, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Asynchronous flush sink duration of changefeeds", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 49 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 93, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ { - "aliasColors": {}, - "bars": true, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt Eventfeed RPC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 10, - "x": 14, - "y": 3 - }, - "hiddenSeries": false, - "id": 28, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{capture=~\"$capture\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" - }, - { - "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*leader\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "transfer-leader", - "refId": "B" - }, - { - "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*(peer|region)\"}[1m]))", - "format": "time_series", - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "move-region", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Eventfeed error/m", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "max(rate(ticdc_sink_flush_event_duration_seconds_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Flush sink 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_TEST-CLUSTER}", + "description": "Percentiles of asynchronous flush sink duration of changefeeds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 49 + }, + "hiddenSeries": false, + "id": 98, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client received events from TiKV per seconds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 10 - }, - "hiddenSeries": false, - "id": 29, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV client receive events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of puller received events from kv client per second\n", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 10 - }, - "hiddenSeries": false, - "id": 32, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller receive events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Flush sink duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that puller outputs to sorter \n per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 10 - }, - "hiddenSeries": false, - "id": 5, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller output events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "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, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of detecting and waiting conflict of MySQL sink", + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 56 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 103, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "MySQL sink conflict detect 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_TEST-CLUSTER}", + "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 56 + }, + "hiddenSeries": false, + "id": 83, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows that sink flushes to downstream per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 10 - }, - "hiddenSeries": false, - "id": 108, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink flush rows/s", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Puller's memory buffer and output channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 17 - }, - "hiddenSeries": false, - "id": 43, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*chan.*/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - output chan", - "refId": "A" - }, - { - "expr": "-sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) ", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - memory buffer", - "refId": "B" - }, - { - "expr": "-sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - input chan", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MySQL sink conflict detect duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "Distribution of MySQL worker loads", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 56 + }, + "hiddenSeries": false, + "id": 95, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{bucket}}", + "refId": "A" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total worker", + "refId": "B" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "0-2 row/s worker", + "refId": "C" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "2-10 row/s worker", + "refId": "D" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "10-100 row/s worker", + "refId": "E" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ">100 row/s worker", + "refId": "F" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MySQL sink worker load", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 63 + }, + "id": 13, + "panels": [], + "title": "Events", + "type": "row" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of established Eventfeed RPC between TiCDC and TiKV", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 64 + }, + "hiddenSeries": false, + "id": 15, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "ticdc_kvclient_event_feed_count", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Eventfeed count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "decimals": null, + "description": "Percentiles of Eventfeed message size", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 6, + "y": 64 + }, + "hiddenSeries": false, + "id": 17, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Sorter's unsorted events buffer and output channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 17 - }, - "hiddenSeries": false, - "id": 51, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*resolvedts/", - "yaxis": 2 - }, - { - "alias": "/.*chan/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-unsorted events", - "refId": "A" - }, - { - "expr": "-sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-resolvedts", - "refId": "B" - }, - { - "expr": "-sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-ouput chan", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Event size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 17 - }, - "hiddenSeries": false, - "id": 107, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*processor.*/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-mounter input chan", - "refId": "A" - }, - { - "expr": "-sum(ticdc_processor_txn_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-processor output chan", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Processor/Mounter buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of errors that interrupt Eventfeed RPC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 10, + "x": 14, + "y": 64 + }, + "hiddenSeries": false, + "id": 28, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{capture=~\"$capture\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*leader\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "transfer-leader", + "refId": "B" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*(peer|region)\"}[1m]))", + "format": "time_series", + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "move-region", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Eventfeed error/m", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The number of KV client received events from TiKV per seconds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 71 + }, + "hiddenSeries": false, + "id": 29, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KV client receive events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are buffered in Sink's pending flush rows buffer", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 17 - }, - "hiddenSeries": false, - "id": 96, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink rows buffer size", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "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_TEST-CLUSTER}", + "description": "The number of puller received events from kv client per second\n", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 71 + }, + "hiddenSeries": false, + "id": 32, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - {{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller receive events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of sorting unsorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 24 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 99, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter sort 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 + "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_TEST-CLUSTER}", + "description": "The number of events that puller outputs to sorter \n per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 71 + }, + "hiddenSeries": false, + "id": 5, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}} - {{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller output events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sorting events duration", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 24 - }, - "hiddenSeries": false, - "id": 53, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter sort duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "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_TEST-CLUSTER}", + "description": "The number of rows that sink flushes to downstream per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 71 + }, + "hiddenSeries": false, + "id": 108, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink flush rows/s", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of merging sorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 31 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 105, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter merge 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 + "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_TEST-CLUSTER}", + "description": "The number of events that are buffered in Puller's memory buffer and output channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 78 + }, + "hiddenSeries": false, + "id": 43, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*chan.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - output chan", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of merging sorted events duration", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 31 - }, - "hiddenSeries": false, - "id": 106, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter merge duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "-sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) ", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - memory buffer", + "refId": "B" }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of unmarshal events from kv to SQL row", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 38 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 101, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Mounter unmarshal 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 + "expr": "-sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - input chan", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The number of events that are buffered in Sorter's unsorted events buffer and output channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 78 + }, + "hiddenSeries": false, + "id": 51, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*resolvedts/", + "yaxis": 2 }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of unmarshal events from kv to SQL row duration", - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 38 - }, - "hiddenSeries": false, - "id": 55, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Mounter unmarshal duration percentile", - "tooltip": { - "shared": true, - "sort": 2, - "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": null, - "show": true - }, - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "alias": "/.*chan/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-unsorted events", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client dispatched event per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 45 - }, - "hiddenSeries": false, - "id": 31, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*batch-resolved/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (capture, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", - "refId": "A" - }, - { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, table)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-batch-resolved", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV client dispatch events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "-sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{capture}}-resolvedts", + "refId": "B" + }, + { + "expr": "-sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-ouput chan", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 78 + }, + "hiddenSeries": false, + "id": 107, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*processor.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-mounter input chan", + "refId": "A" + }, + { + "expr": "-sum(ticdc_sink_buffer_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-sink buffer chan", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink/Mounter buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "description": "The number of rows(events) that are buffered in Sink's pending flush rows buffer", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 78 + }, + "hiddenSeries": false, + "id": 96, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink rows buffer size", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "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_TEST-CLUSTER}", + "description": "Duration of sorting unsorted events", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 85 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 99, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter sort 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_TEST-CLUSTER}", + "description": "Percentiles of sorting events duration", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 85 + }, + "hiddenSeries": false, + "id": 53, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "A" }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The size of batch resolved ts message from TiKV", - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 45 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 97, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "KV client batch resolved size", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" } ], - "title": "Events", - "type": "row" + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter sort duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "collapsed": true, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of merging sorted events", "gridPos": { - "h": 1, - "w": 24, + "h": 7, + "w": 12, "x": 0, - "y": 3 + "y": 92 }, - "id": 130, - "panels": [ + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 105, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 1 - }, - "id": 131, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_consume_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter intake rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, + "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter merge 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_TEST-CLUSTER}", + "description": "Percentiles of merging sorted events duration", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 92 + }, + "hiddenSeries": false, + "id": 106, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 1 - }, - "id": 132, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter event output rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "A" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 9 - }, - "id": 133, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter on disk data size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "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 - } + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter merge duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 9 - }, - "id": 134, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter in-memory data size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "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 - } + "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_TEST-CLUSTER}", + "description": "Duration of unmarshal events from kv to SQL row", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 99 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 101, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Mounter unmarshal 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_TEST-CLUSTER}", + "description": "Percentiles of unmarshal events from kv to SQL row duration", + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 99 + }, + "hiddenSeries": false, + "id": 55, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Mounter unmarshal duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "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": null, + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of KV client dispatched event per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 106 + }, + "hiddenSeries": false, + "id": 31, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*batch-resolved/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (capture, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" + }, + { + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, table)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-batch-resolved", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KV client dispatch events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "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_TEST-CLUSTER}", + "description": "The size of batch resolved ts message from TiKV", + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 106 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 97, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "KV client batch resolved size", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 113 + }, + "id": 130, + "panels": [], + "title": "Unified Sorter", + "type": "row" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 114 + }, + "id": 131, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_consume_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter intake rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "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_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 114 + }, + "id": 132, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter event output rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateOranges", - "exponent": 0.5, - "max": null, - "min": null, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 17 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 135, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter flush sizes", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "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_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 122 + }, + "id": 133, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter on disk data size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateBlues", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 17 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 136, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter merge size", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "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_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 122 + }, + "id": 134, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter in-memory data size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 25 - }, - "id": 137, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "min(ticdc_sorter_resolved_ts_gauge{changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter resolved ts", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "title": "Unified Sorter", - "type": "row" + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateOranges", + "exponent": 0.5, + "max": null, + "min": null, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 130 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 135, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter flush sizes", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateBlues", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 130 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 136, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter merge size", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 138 + }, + "id": 137, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "min(ticdc_sorter_resolved_ts_gauge{changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter resolved ts", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "collapsed": true, @@ -4964,7 +4960,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 146 }, "id": 58, "panels": [ @@ -6052,5 +6048,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 12 + "version": 13 } From 0c1c2ee1bfbf8439f4bcc4b79bbef43df03274ba Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 12 Jan 2021 14:07:34 +0800 Subject: [PATCH 40/44] update dot --- docs/data-flow.dot | 2 +- docs/data-flow.svg | 250 ++++++++++++++++++++++----------------------- 2 files changed, 123 insertions(+), 129 deletions(-) diff --git a/docs/data-flow.dot b/docs/data-flow.dot index 9a32651ad5c..475fe7e56d1 100644 --- a/docs/data-flow.dot +++ b/docs/data-flow.dot @@ -179,5 +179,5 @@ digraph Dataflow { data_in -> Raftstore data_out [label = "Data out", shape = oval] - sink_run -> data_out + backend_sink_run -> data_out } diff --git a/docs/data-flow.svg b/docs/data-flow.svg index 2989e28ca3f..c3edfc5138b 100644 --- a/docs/data-flow.svg +++ b/docs/data-flow.svg @@ -4,292 +4,286 @@ - - + + Dataflow - + cluster_legends - -Legend + +Legend cluster_tikv - -TiKV + +TiKV cluster_ticdc - -TiCDC + +TiCDC cluster_kvclient - -KV client + +KV client cluster_processor - -Processor + +Processor cluster_puller - -Puller + +Puller cluster_sorter - -Sorter + +Sorter cluster_entry_sorter - -Entry sorter + +Entry sorter cluster_mounter - -Mounter + +Mounter cluster_sink - -Sink Manager + +Sink Manager flow - -Go routine #1 + +Go routine #1 flow_ - -Go routine #2 + +Go routine #2 flow->flow_ - - -Flow -direction -Channel -/Buffer + + +Flow +direction +Channel +/Buffer wait - -Go routine #1 + +Go routine #1 wait_ - -Go routine #2 + +Go routine #2 wait->wait_ - - -#1 wiats #2 + + +#1 wiats #2 Raftstore - -Raftstore + +Raftstore CDC - -CDC + +CDC Raftstore->CDC - - -channel + + +channel kv_client - -kv_client + +kv_client CDC->kv_client - - -gRPC + + +gRPC puller_run_step1 - -Run #1 -(Add to memory buffer) + +Run #1 +(Add to memory buffer) kv_client->puller_run_step1 - - -eventCh + + +eventCh puller_run_step2 - -Run #2 -(Output to output channel) + +Run #2 +(Output to output channel) puller_run_step1->puller_run_step2 - - -P.buffer + + +P.buffer processor_sorter_consume - -sorterConsume + +sorterConsume table_sink_run - -Table Sink -FlushEvents + +Table Sink +FlushEvents processor_sorter_consume->table_sink_run - - -P.output + + +P.output mounter_run - -Run + +Run processor_sorter_consume->mounter_run - - -M.Input + + +M.Input processor_puller_consume - -pullerConsume + +pullerConsume sorter_run - -Run + +Run processor_puller_consume->sorter_run - - -E.AddEntry + + +E.AddEntry buffer_sink_run - -Buffer Sink -FlushEvents + +Buffer Sink +FlushEvents table_sink_run->buffer_sink_run - - -S.EmitEvents + + +S.EmitEvents puller_run_step2->processor_puller_consume - - -P.Output + + +P.Output sorter_run->processor_sorter_consume - - -S.Ouput + + +S.Ouput mounter_run->table_sink_run - - -wait unmarshal + + +wait unmarshal backend_sink_run - -Backend Sink -FlushEvents + +Backend Sink +FlushEvents buffer_sink_run->backend_sink_run - - -S.EmitEvents + + +S.EmitEvents + + + +data_out + +Data out + + + +backend_sink_run->data_out + + data_in - -Data in + +Data in data_in->Raftstore - - - - - -data_out - -Data out - - - -sink_run - -sink_run - - - -sink_run->data_out - - + + From 59855edf3e72cc66cc808a7d9f88e7f3bce3eba5 Mon Sep 17 00:00:00 2001 From: leoppro Date: Tue, 12 Jan 2021 14:18:00 +0800 Subject: [PATCH 41/44] update docs --- docs/ticdc-grafana-dashboard.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ticdc-grafana-dashboard.md b/docs/ticdc-grafana-dashboard.md index d1e344d3de9..3a446c085a3 100644 --- a/docs/ticdc-grafana-dashboard.md +++ b/docs/ticdc-grafana-dashboard.md @@ -94,7 +94,7 @@ cdc cli changefeed create --pd=http://10.0.10.25:2379 --sink-uri="mysql://root:1 - Sink flush rows/s : TiCDC 节点每秒写到下游的数据变更的个数 - Puller buffer size : TiCDC 节点中缓存在 Puller 模块中的数据变更个数 - Entry sorter buffer size : TiCDC 节点中缓存在 Sorter 模块中的数据变更个数 -- Processor/Mounter buffer size : TiCDC 节点中缓存在 Processor 模块和 Mounter 模块中的数据变更个数 +- Sink/Mounter buffer size : TiCDC 节点中缓存在 Buffer Sink 模块和 Mounter 模块中的数据变更个数 - Sink row buffer size : TiCDC 节点中缓存在 Sink 模块中的数据变更个数 - Entry sorter sort duration : TiCDC 节点排序数据变更的耗时直方图 - Entry sorter sort duration percentile : 每秒钟中 95%,99% 和 99.9% 的情况下,TiCDC 排序数据变更所花费的时间 From de2729b2705f1d01cf35846b7406ad42d86dcd79 Mon Sep 17 00:00:00 2001 From: leoppro Date: Wed, 13 Jan 2021 13:35:06 +0800 Subject: [PATCH 42/44] update ticdc.json --- metrics/grafana/ticdc.json | 9414 ++++++++++++++++++------------------ 1 file changed, 4709 insertions(+), 4705 deletions(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 0f90ba79aa7..28ecf484831 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -109,11 +109,11 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1610428254319, + "iteration": 1610515993595, "links": [], "panels": [ { - "collapsed": false, + "collapsed": true, "gridPos": { "h": 1, "w": 24, @@ -121,4838 +121,4842 @@ "y": 0 }, "id": 21, - "panels": [], - "title": "Server", - "type": "row" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Uptime of TiCDC and TiKV", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 1 - }, - "hiddenSeries": false, - "id": 19, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "(time() - process_start_time_seconds{job=\"ticdc\"})", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "TiCDC - {{instance}}", - "refId": "A" - }, - { - "expr": "(time() - process_start_time_seconds{job=\"tikv\"})", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "TiKV - {{instance}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Uptime", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dtdurations", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "Goroutine count of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 8, - "y": 1 - }, - "hiddenSeries": false, - "id": 26, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": " go_goroutines{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Goroutine count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "The count of open FD count of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 14, - "y": 1 - }, - "hiddenSeries": false, - "id": 27, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "process_open_fds{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Open FD count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC cluster ownership status", - "fontSize": "100%", - "gridPos": { - "h": 7, - "w": 4, - "x": 20, - "y": 1 - }, - "hideTimeOverride": true, - "id": 113, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 2, - "desc": false - }, - "styles": [ + "panels": [ { - "alias": "Instance", - "colorMode": null, - "colors": [ - "#8AB8FF", - "#73BF69", - "rgba(245, 54, 54, 0.9)" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Uptime of TiCDC and TiKV", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 1 + }, + "hiddenSeries": false, + "id": 19, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "(time() - process_start_time_seconds{job=\"ticdc\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "TiCDC - {{instance}}", + "refId": "A" + }, + { + "expr": "(time() - process_start_time_seconds{job=\"tikv\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "TiKV - {{instance}}", + "refId": "B" + } ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "mappingType": 2, - "pattern": "Current", - "preserveFormat": false, - "rangeMaps": [ - { - "from": "0.1", - "text": "Owner", - "to": "2" + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Uptime", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dtdurations", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "from": "0", - "text": "Worker", - "to": "0.1" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "sanitize": false, - "thresholds": [ - "0.1", - "2" + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Goroutine count of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 8, + "y": 1 + }, + "hiddenSeries": false, + "id": 26, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": " go_goroutines{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } ], - "type": "string", - "unit": "short", - "valueMaps": [ + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Goroutine count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, { - "text": "Owner", - "value": "1" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } - ] - } - ], - "targets": [ - { - "expr": "rate(ticdc_owner_ownership_counter[30s])", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "timeFrom": "1s", - "title": "Ownership", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "CPU usage of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 8 - }, - "hiddenSeries": false, - "id": 24, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "rate(process_cpu_seconds_total{job=\"ticdc\"}[1m])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CPU usage", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percentunit", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "Memory usage of TiCDC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 8 - }, - "hiddenSeries": false, - "id": 23, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "process_resident_memory_bytes{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "process-{{instance}}", - "refId": "A" - }, - { - "expr": "go_memstats_heap_alloc_bytes{job=\"ticdc\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "heap-{{instance}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Memory usage", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "The history of TiCDC cluster ownership, owner node has a value that is great than 0", - "fill": 1, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 8 - }, - "id": 110, - "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", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": true, - "targets": [ - { - "expr": "sum(rate(ticdc_owner_ownership_counter[30s])) by (instance) > 0", - "format": "time_series", - "interval": "30s", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Ownership history", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 8 - }, - "hiddenSeries": false, - "id": 114, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p999-{{instance}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p99-{{instance}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p95-{{instance}}", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Etcd health check duration", - "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": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "collapsed": false, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 15 - }, - "id": 11, - "panels": [], - "title": "Changefeed", - "type": "row" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of captured table of TiCDC nodes ", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 0, - "y": 16 - }, - "id": 4, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" - }, - { - "alias": "", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The count of open FD count of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 14, + "y": 1 + }, + "hiddenSeries": false, + "id": 27, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "process_open_fds{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } ], - "decimals": 2, - "pattern": "/.*/", "thresholds": [], - "type": "number", - "unit": "short" - } - ], - "targets": [ - { - "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Changefeed table count", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of TiCDC nodes", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 7, - "y": 16 - }, - "id": 90, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Open FD count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "columns": [ + { + "text": "Current", + "value": "current" + } ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "max(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - }, - { - "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "changefeed checkpoint", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Processor resolved ts", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of captured tables", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 10, - "x": 14, - "y": 16 - }, - "id": 30, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC cluster ownership status", + "fontSize": "100%", + "gridPos": { + "h": 7, + "w": 4, + "x": 20, + "y": 1 + }, + "hideTimeOverride": true, + "id": 113, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 2, + "desc": false + }, + "styles": [ + { + "alias": "Instance", + "colorMode": null, + "colors": [ + "#8AB8FF", + "#73BF69", + "rgba(245, 54, 54, 0.9)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "link": false, + "mappingType": 2, + "pattern": "Current", + "preserveFormat": false, + "rangeMaps": [ + { + "from": "0.1", + "text": "Owner", + "to": "2" + }, + { + "from": "0", + "text": "Worker", + "to": "0.1" + } + ], + "sanitize": false, + "thresholds": [ + "0.1", + "2" + ], + "type": "string", + "unit": "short", + "valueMaps": [ + { + "text": "Owner", + "value": "1" + } + ] + } ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "targets": [ + { + "expr": "rate(ticdc_owner_ownership_counter[30s])", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "max(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", - "refId": "A" - }, - { - "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "changefeed checkpoint", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Table resolved ts", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "aliasColors": {}, - "bars": true, - "cacheTimeout": null, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The checkpoint ts of changefeeds.", - "fill": 0, - "gridPos": { - "h": 7, - "w": 9, - "x": 0, - "y": 21 - }, - "id": 86, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": null, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 2, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/approximate current time.*/", - "bars": false - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_owner_checkpoint_ts{changefeed=~\"$changefeed\"}) by (changefeed) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" + "timeFrom": "1s", + "title": "Ownership", + "transform": "timeseries_aggregations", + "type": "table" }, { - "expr": "max(pd_cluster_tso) * 1000", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "approximate current time (s)", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Changefeed checkpoint", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "max": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Request count of etcd operation per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 9, - "y": 21 - }, - "hiddenSeries": false, - "id": 102, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (capture, type)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "PD etcd requests/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "The number of errors that interrupt changefeed per minute ", - "fill": 1, - "gridPos": { - "h": 7, - "w": 7, - "x": 17, - "y": 21 - }, - "id": 82, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(delta(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Exit error count/m", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 28 - }, - "hiddenSeries": false, - "id": 3, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_owner_checkpoint_ts_lag{changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Changefeed 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": null, - "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_TEST-CLUSTER}", - "description": "The lag between internal resolved ts and the latest ts of upstream TiDB.", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 28 - }, - "hiddenSeries": false, - "id": 2, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Processor 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": null, - "show": true - }, - { - "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, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Sink write duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 35 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 94, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Sink write 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_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 35 - }, - "hiddenSeries": false, - "id": 35, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write duration percentile", - "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": 2, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "The number of changed rows that are written to downstream per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 42 - }, - "hiddenSeries": false, - "id": 34, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - }, - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write rows count/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "Percentiles of sink batch size", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 42 - }, - "hiddenSeries": false, - "id": 36, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink write batch size percentile", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "show": true - }, - { - "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, - "max": null, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Asynchronous flush sink duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 49 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 93, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_flush_event_duration_seconds_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Flush sink 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_TEST-CLUSTER}", - "description": "Percentiles of asynchronous flush sink duration of changefeeds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 49 - }, - "hiddenSeries": false, - "id": 98, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "CPU usage of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 8 + }, + "hiddenSeries": false, + "id": 24, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "rate(process_cpu_seconds_total{job=\"ticdc\"}[1m])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CPU usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Memory usage of TiCDC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 8 + }, + "hiddenSeries": false, + "id": 23, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "process-{{instance}}", + "refId": "A" + }, + { + "expr": "go_memstats_heap_alloc_bytes{job=\"ticdc\"}", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "heap-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Flush sink duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The history of TiCDC cluster ownership, owner node has a value that is great than 0", + "fill": 1, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 8 + }, + "id": 110, + "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", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": true, + "targets": [ + { + "expr": "sum(rate(ticdc_owner_ownership_counter[30s])) by (instance) > 0", + "format": "time_series", + "interval": "30s", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Ownership history", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 8 + }, + "hiddenSeries": false, + "id": 114, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p999-{{instance}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p99-{{instance}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p95-{{instance}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd health check duration", + "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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "Server", + "type": "row" }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of detecting and waiting conflict of MySQL sink", + "collapsed": true, "gridPos": { - "h": 7, - "w": 8, + "h": 1, + "w": 24, "x": 0, - "y": 56 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 103, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "MySQL sink conflict detect 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_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 56 - }, - "hiddenSeries": false, - "id": 83, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "MySQL sink conflict detect duration percentile", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "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_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 56 - }, - "hiddenSeries": false, - "id": 95, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] + "y": 1 }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": true, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", - "refId": "A" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total worker", - "refId": "B" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", - "refId": "C" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "2-10 row/s worker", - "refId": "D" - }, + "id": 11, + "panels": [ { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "10-100 row/s worker", - "refId": "E" + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of captured table of TiCDC nodes ", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 0, + "y": 2 + }, + "id": 4, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "expr": "sum(ticdc_processor_num_of_tables{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Changefeed table count", + "transform": "timeseries_aggregations", + "type": "table" }, { - "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": ">100 row/s worker", - "refId": "F" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "MySQL sink worker load", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "Internal resolved ts of TiCDC nodes", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 7, + "y": 2 + }, + "id": 90, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + }, + { + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Processor resolved ts", + "transform": "timeseries_aggregations", + "type": "table" }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "collapsed": false, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 63 - }, - "id": 13, - "panels": [], - "title": "Events", - "type": "row" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of established Eventfeed RPC between TiCDC and TiKV", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 64 - }, - "hiddenSeries": false, - "id": 15, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "ticdc_kvclient_event_feed_count", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Eventfeed count", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "Internal resolved ts of captured tables", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 10, + "x": 14, + "y": 2 + }, + "id": 30, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "max(ticdc_processor_table_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,table)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" + }, + { + "expr": "max(ticdc_processor_checkpoint_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "changefeed checkpoint", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Table resolved ts", + "transform": "timeseries_aggregations", + "type": "table" }, { - "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_TEST-CLUSTER}", - "decimals": null, - "description": "Percentiles of Eventfeed message size", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 8, - "x": 6, - "y": 64 - }, - "hiddenSeries": false, - "id": 17, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "A" + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The checkpoint ts of changefeeds.", + "fill": 0, + "gridPos": { + "h": 7, + "w": 9, + "x": 0, + "y": 7 + }, + "id": 86, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/approximate current time.*/", + "bars": false + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_owner_checkpoint_ts{changefeed=~\"$changefeed\"}) by (changefeed) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "max(pd_cluster_tso) * 1000", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "approximate current time (s)", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed checkpoint", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "max": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Event size percentile", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Request count of etcd operation per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 9, + "y": 7 + }, + "hiddenSeries": false, + "id": 102, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (capture, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD etcd requests/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of errors that interrupt changefeed per minute ", + "fill": 1, + "gridPos": { + "h": 7, + "w": 7, + "x": 17, + "y": 7 + }, + "id": 82, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(delta(ticdc_processor_exit_with_error_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Exit error count/m", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": true, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt Eventfeed RPC", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 10, - "x": 14, - "y": 64 - }, - "hiddenSeries": false, - "id": 28, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{capture=~\"$capture\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "hiddenSeries": false, + "id": 3, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_owner_checkpoint_ts_lag{changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed 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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*leader\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "transfer-leader", - "refId": "B" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The lag between internal resolved ts and the latest ts of upstream TiDB.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "hiddenSeries": false, + "id": 2, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_processor_resolved_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor 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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*(peer|region)\"}[1m]))", - "format": "time_series", - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "move-region", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Eventfeed error/m", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Sink write duration of changefeeds", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 21 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 94, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "repeat": null, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Sink write 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 }, { - "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_TEST-CLUSTER}", - "description": "The number of KV client received events from TiKV per seconds", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 71 - }, - "hiddenSeries": false, - "id": 29, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV client receive events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of sink write duration of changefeeds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 21 + }, + "hiddenSeries": false, + "id": 35, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The number of puller received events from kv client per second\n", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 71 - }, - "hiddenSeries": false, - "id": 32, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller receive events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of changed rows that are written to downstream per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 28 + }, + "hiddenSeries": false, + "id": 34, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + }, + { + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{changefeed=~\"$changefeed\"}[1m])) by (changefeed)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write rows count/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The number of events that puller outputs to sorter \n per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 71 - }, - "hiddenSeries": false, - "id": 5, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller output events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of sink batch size", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 28 + }, + "hiddenSeries": false, + "id": 36, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p90", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write batch size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The number of rows that sink flushes to downstream per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 71 - }, - "hiddenSeries": false, - "id": 108, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink flush rows/s", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Asynchronous flush sink duration of changefeeds", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 35 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 93, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_flush_event_duration_seconds_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Flush sink 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 }, { - "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_TEST-CLUSTER}", - "description": "The number of events that are buffered in Puller's memory buffer and output channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 0, - "y": 78 - }, - "hiddenSeries": false, - "id": 43, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*chan.*/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - output chan", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of asynchronous flush sink duration of changefeeds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 35 + }, + "hiddenSeries": false, + "id": 98, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Flush sink duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) ", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - memory buffer", - "refId": "B" + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of detecting and waiting conflict of MySQL sink", + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 42 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 103, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "MySQL sink conflict detect 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 }, { - "expr": "-sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - input chan", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Puller buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 42 + }, + "hiddenSeries": false, + "id": 83, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MySQL sink conflict detect duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Distribution of MySQL worker loads", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 42 + }, + "hiddenSeries": false, + "id": 95, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{bucket}}", + "refId": "A" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total worker", + "refId": "B" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "0-2 row/s worker", + "refId": "C" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "2-10 row/s worker", + "refId": "D" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "10-100 row/s worker", + "refId": "E" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ">100 row/s worker", + "refId": "F" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MySQL sink worker load", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "Changefeed", + "type": "row" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Sorter's unsorted events buffer and output channel", - "fill": 1, - "fillGradient": 0, + "collapsed": true, "gridPos": { - "h": 7, - "w": 6, - "x": 6, - "y": 78 - }, - "hiddenSeries": false, - "id": 51, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] + "h": 1, + "w": 24, + "x": 0, + "y": 2 }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ + "id": 13, + "panels": [ { - "alias": "/.*resolvedts/", - "yaxis": 2 + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of established Eventfeed RPC between TiCDC and TiKV", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 3 + }, + "hiddenSeries": false, + "id": 15, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "ticdc_kvclient_event_feed_count", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Eventfeed count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "alias": "/.*chan/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "Percentiles of Eventfeed message size", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 6, + "y": 3 + }, + "hiddenSeries": false, + "id": 17, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Event size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { - "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-unsorted events", - "refId": "A" + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of errors that interrupt Eventfeed RPC", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 10, + "x": 14, + "y": 3 + }, + "hiddenSeries": false, + "id": 28, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{capture=~\"$capture\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*leader\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "transfer-leader", + "refId": "B" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{event=\"create\", type=~\".*(peer|region)\"}[1m]))", + "format": "time_series", + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "move-region", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Eventfeed error/m", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-resolvedts", - "refId": "B" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of KV client received events from TiKV per seconds", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 10 + }, + "hiddenSeries": false, + "id": 29, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KV client receive events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-ouput chan", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of puller received events from kv client per second\n", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 10 + }, + "hiddenSeries": false, + "id": 32, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_puller_kv_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - {{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller receive events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 78 - }, - "hiddenSeries": false, - "id": 107, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*processor.*/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-mounter input chan", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events that puller outputs to sorter \n per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 10 + }, + "hiddenSeries": false, + "id": 5, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_puller_txn_collect_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}} - {{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller output events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "-sum(ticdc_sink_buffer_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-sink buffer chan", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink/Mounter buffer size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows that sink flushes to downstream per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 10 + }, + "hiddenSeries": false, + "id": 108, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink flush rows/s", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The number of rows(events) that are buffered in Sink's pending flush rows buffer", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 6, - "x": 18, - "y": 78 - }, - "hiddenSeries": false, - "id": 96, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": false, - "rightSide": false, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink rows buffer size", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events that are buffered in Puller's memory buffer and output channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 43, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*chan.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - output chan", + "refId": "A" + }, + { + "expr": "-sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) ", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - memory buffer", + "refId": "B" + }, + { + "expr": "-sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}} - input chan", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Puller buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "Duration of sorting unsorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 85 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 99, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter sort 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_TEST-CLUSTER}", - "description": "Percentiles of sorting events duration", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 85 - }, - "hiddenSeries": false, - "id": 53, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events that are buffered in Sorter's unsorted events buffer and output channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 6, + "y": 17 + }, + "hiddenSeries": false, + "id": 51, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*resolvedts/", + "yaxis": 2 + }, + { + "alias": "/.*chan/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-unsorted events", + "refId": "A" + }, + { + "expr": "-sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{capture}}-resolvedts", + "refId": "B" + }, + { + "expr": "-sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-ouput chan", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter sort duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 17 + }, + "hiddenSeries": false, + "id": 107, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*processor.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-mounter input chan", + "refId": "A" + }, + { + "expr": "-sum(ticdc_sink_buffer_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-sink buffer chan", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink/Mounter buffer size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "Duration of merging sorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 92 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 105, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter merge 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_TEST-CLUSTER}", - "description": "Percentiles of merging sorted events duration", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 92 - }, - "hiddenSeries": false, - "id": 106, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows(events) that are buffered in Sink's pending flush rows buffer", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 17 + }, + "hiddenSeries": false, + "id": 96, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_total_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink rows buffer size", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Entry sorter merge duration percentile", - "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": 2, - "max": null, - "min": null, - "show": true + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of sorting unsorted events", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 24 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 99, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter sort 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 }, { - "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_TEST-CLUSTER}", - "description": "Duration of unmarshal events from kv to SQL row", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 99 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 101, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Mounter unmarshal 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_TEST-CLUSTER}", - "description": "Percentiles of unmarshal events from kv to SQL row duration", - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 99 - }, - "hiddenSeries": false, - "id": 55, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of sorting events duration", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 24 + }, + "hiddenSeries": false, + "id": 53, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter sort duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "A" + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of merging sorted events", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 31 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 105, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter merge 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 }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Mounter unmarshal duration percentile", - "tooltip": { - "shared": true, - "sort": 2, - "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": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of merging sorted events duration", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 31 + }, + "hiddenSeries": false, + "id": 106, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Entry sorter merge duration percentile", + "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": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client dispatched event per second", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 106 - }, - "hiddenSeries": false, - "id": 31, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*batch-resolved/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of unmarshal events from kv to SQL row", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 38 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 101, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Mounter unmarshal 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 + }, { - "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (capture, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", - "refId": "A" + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of unmarshal events from kv to SQL row duration", + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 38 + }, + "hiddenSeries": false, + "id": 55, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Mounter unmarshal duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "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": null, + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, table)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-batch-resolved", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV client dispatch events/s", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of KV client dispatched event per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 45 + }, + "hiddenSeries": false, + "id": 31, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*batch-resolved/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (capture, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" + }, + { + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, table)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-batch-resolved", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KV client dispatch events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "description": "The size of batch resolved ts message from TiKV", - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 106 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 97, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The size of batch resolved ts message from TiKV", + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 45 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 97, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "KV client batch resolved size", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null } ], - "title": "KV client batch resolved size", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "title": "Events", + "type": "row" }, { - "collapsed": false, + "collapsed": true, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 113 + "y": 3 }, "id": 130, - "panels": [], - "title": "Unified Sorter", - "type": "row" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 114 - }, - "id": 131, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_consume_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter intake rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ + "panels": [ { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 4 + }, + "id": 131, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_consume_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter intake rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 114 - }, - "id": 132, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter event output rate", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 4 + }, + "id": 132, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_event_count{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter event output rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "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_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 122 - }, - "id": 133, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter on disk data size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 12 + }, + "id": 133, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter on disk data size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "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 + } + }, { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 12 + }, + "id": 134, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter in-memory data size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "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 + } }, { - "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_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 122 - }, - "id": 134, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter in-memory data size", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateOranges", + "exponent": 0.5, + "max": null, + "min": null, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 20 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 135, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter flush sizes", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateOranges", - "exponent": 0.5, - "max": null, - "min": null, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 130 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 135, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter flush sizes", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateBlues", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 130 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 136, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter merge size", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 138 - }, - "id": 137, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "min(ticdc_sorter_resolved_ts_gauge{changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Unified Sorter resolved ts", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateBlues", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 20 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 136, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter merge size", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 28 + }, + "id": 137, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "min(ticdc_sorter_resolved_ts_gauge{changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter resolved ts", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "title": "Unified Sorter", + "type": "row" }, { "collapsed": true, @@ -4960,7 +4964,7 @@ "h": 1, "w": 24, "x": 0, - "y": 146 + "y": 4 }, "id": 58, "panels": [ From 75fbb46389d503576f7d8eb5d6a907a913cd7acd Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 14 Jan 2021 15:01:24 +0800 Subject: [PATCH 43/44] add some test cases --- cdc/processor.go | 29 ++++++-- cdc/sink/manager.go | 15 +++-- cdc/sink/manager_test.go | 131 +++++++++++++++++++++++++++++++++++++ cdc/sink/metrics.go | 2 +- docs/data-flow.dot | 12 ++-- docs/data-flow.svg | 72 ++++++++++---------- metrics/grafana/ticdc.json | 14 ++-- 7 files changed, 212 insertions(+), 63 deletions(-) diff --git a/cdc/processor.go b/cdc/processor.go index 55c1f586679..9f0cd1243ef 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1164,7 +1164,7 @@ func runProcessor( return nil, errors.Trace(err) } ctx, cancel := context.WithCancel(ctx) - errCh := make(chan error, 1) + errCh := make(chan error, 16) s, err := sink.NewSink(ctx, changefeedID, info.SinkURI, filter, info.Config, opts, errCh) if err != nil { cancel() @@ -1183,16 +1183,33 @@ func runProcessor( processor.Run(ctx) go func() { + var errs []error + appendError := func(err error) { + log.Debug("processor received error", zap.Error(err)) + cause := errors.Cause(err) + if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { + errs = append(errs, err) + } + } err := <-errCh - log.Debug("processor exited by error", zap.Error(err)) - cause := errors.Cause(err) - if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { - processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Inc() + appendError(err) + // sleep 500ms to wait all the errors are sent to errCh + time.Sleep(500 * time.Millisecond) + ReceiveErr: + for { + select { + case err := <-errCh: + appendError(err) + default: + break ReceiveErr + } + } + if len(errs) > 0 { log.Error("error on running processor", util.ZapFieldCapture(ctx), zap.String("changefeed", changefeedID), zap.String("processor", processor.id), - zap.Error(err)) + zap.Errors("errors", errs)) // record error information in etcd var code string if terror, ok := err.(*errors.Error); ok { diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index ac8e3e26533..ccca7fc65b0 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -174,7 +174,6 @@ type bufferSink struct { Sink buffer chan struct { rows []*model.RowChangedEvent - resolved bool resolvedTs model.Ts } checkpointTs uint64 @@ -185,7 +184,6 @@ func newBufferSink(ctx context.Context, backendSink Sink, errCh chan error, chec Sink: backendSink, buffer: make(chan struct { rows []*model.RowChangedEvent - resolved bool resolvedTs model.Ts }, defaultBufferChanSize), checkpointTs: checkpointTs, @@ -197,7 +195,8 @@ func newBufferSink(ctx context.Context, backendSink Sink, errCh chan error, chec func (b *bufferSink) run(ctx context.Context, errCh chan error) { changefeedID := util.ChangefeedIDFromCtx(ctx) advertiseAddr := util.CaptureAddrFromCtx(ctx) - metricFlushDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID) + metricFlushDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "Flush") + metricEmitRowDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "EmitRow") metricBufferSize := bufferChanSizeGauge.WithLabelValues(advertiseAddr, changefeedID) for { select { @@ -208,7 +207,8 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { } return case e := <-b.buffer: - if e.resolved { + if e.rows == nil { + // A resolved event received start := time.Now() checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, e.resolvedTs) if err != nil { @@ -227,6 +227,7 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { } continue } + start := time.Now() err := b.Sink.EmitRowChangedEvents(ctx, e.rows...) if err != nil { if errors.Cause(err) != context.Canceled { @@ -234,6 +235,8 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { } return } + dur := time.Since(start) + metricEmitRowDuration.Observe(dur.Seconds()) case <-time.After(defaultMetricInterval): metricBufferSize.Set(float64(len(b.buffer))) } @@ -246,7 +249,6 @@ func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Ro return ctx.Err() case b.buffer <- struct { rows []*model.RowChangedEvent - resolved bool resolvedTs model.Ts }{rows: rows}: } @@ -259,9 +261,8 @@ func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint6 return atomic.LoadUint64(&b.checkpointTs), ctx.Err() case b.buffer <- struct { rows []*model.RowChangedEvent - resolved bool resolvedTs model.Ts - }{resolved: true, resolvedTs: resolvedTs}: + }{resolvedTs: resolvedTs, rows: nil}: } return atomic.LoadUint64(&b.checkpointTs), nil } diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 3418404621f..0d8c4e3cd1a 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -15,10 +15,14 @@ package sink import ( "context" + "math" "math/rand" "sync" + "sync/atomic" "time" + "github.com/pingcap/errors" + "github.com/pingcap/check" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" @@ -130,3 +134,130 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { c.Assert(err, check.IsNil) } } + +func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { + defer testleak.AfterTest(c)() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + errCh := make(chan error, 16) + manager := NewManager(ctx, &checkSink{C: c}, errCh, 0) + defer manager.Close() + goroutineNum := 10 + var wg sync.WaitGroup + const EXIT_SIGNAL = uint64(math.MaxUint64) + + var maxResolvedTs uint64 + tableSinks := make([]Sink, 0, goroutineNum) + closeChs := make([]chan struct{}, 0, goroutineNum) + runTableSink := func(index int64, sink Sink, startTs uint64, close chan struct{}) { + defer wg.Done() + ctx := context.Background() + lastResolvedTs := startTs + for { + select { + case <-close: + return + default: + } + resolvedTs := atomic.LoadUint64(&maxResolvedTs) + if resolvedTs == EXIT_SIGNAL { + return + } + if resolvedTs == lastResolvedTs { + time.Sleep(10 * time.Millisecond) + continue + } + for i := lastResolvedTs + 1; i <= resolvedTs; i++ { + err := sink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + Table: &model.TableName{TableID: index}, + CommitTs: i, + }) + c.Assert(err, check.IsNil) + } + _, err := sink.FlushRowChangedEvents(ctx, resolvedTs) + c.Assert(err, check.IsNil) + lastResolvedTs = resolvedTs + } + } + + wg.Add(1) + go func() { + defer wg.Done() + // add three table and then remote one table + for i := 0; i < 10; i++ { + if i%4 != 3 { + // add table + table := manager.CreateTableSink(model.TableID(i), maxResolvedTs+1) + close := make(chan struct{}) + tableSinks = append(tableSinks, table) + closeChs = append(closeChs, close) + atomic.AddUint64(&maxResolvedTs, 20) + wg.Add(1) + go runTableSink(int64(i), table, maxResolvedTs, close) + } else { + //remove table + table := tableSinks[0] + close(closeChs[0]) + c.Assert(table.Close(), check.IsNil) + tableSinks = tableSinks[1:] + closeChs = closeChs[1:] + } + time.Sleep(100 * time.Millisecond) + } + atomic.StoreUint64(&maxResolvedTs, EXIT_SIGNAL) + }() + + wg.Wait() + cancel() + time.Sleep(1 * time.Second) + close(errCh) + for err := range errCh { + c.Assert(err, check.IsNil) + } +} + +type errorSink struct { + *check.C +} + +func (e *errorSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + panic("unreachable") +} + +func (e *errorSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { + return errors.New("error in emit row changed events") +} + +func (e *errorSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + panic("unreachable") +} + +func (e *errorSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + return 0, errors.New("error in flush row changed events") +} + +func (e *errorSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + panic("unreachable") +} + +func (e *errorSink) Close() error { + return nil +} + +func (s *managerSuite) TestManagerError(c *check.C) { + defer testleak.AfterTest(c)() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + errCh := make(chan error, 16) + manager := NewManager(ctx, &errorSink{C: c}, errCh, 0) + defer manager.Close() + sink := manager.CreateTableSink(1, 0) + err := sink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ + CommitTs: 1, + }) + c.Assert(err, check.IsNil) + _, err = sink.FlushRowChangedEvents(ctx, 2) + c.Assert(err, check.IsNil) + err = <-errCh + c.Assert(err.Error(), check.Equals, "error in emit row changed events") +} diff --git a/cdc/sink/metrics.go b/cdc/sink/metrics.go index 4d7e7a33954..77e5f0a21fa 100644 --- a/cdc/sink/metrics.go +++ b/cdc/sink/metrics.go @@ -77,7 +77,7 @@ var ( Name: "flush_event_duration_seconds", Help: "Bucketed histogram of processing time (s) of flushing events in processor", Buckets: prometheus.ExponentialBuckets(0.002 /* 2ms */, 2, 20), - }, []string{"capture", "changefeed"}) + }, []string{"capture", "changefeed", "type"}) bufferChanSizeGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", diff --git a/docs/data-flow.dot b/docs/data-flow.dot index 475fe7e56d1..c7f94e07815 100644 --- a/docs/data-flow.dot +++ b/docs/data-flow.dot @@ -153,21 +153,21 @@ digraph Dataflow { style = filled; subgraph table_sink { - label = "Table Sink"; + label = "Table Sink (N)"; - table_sink_run [label = "Table Sink\nFlushEvents"] + table_sink_run [label = "Table Sink (N)\nFlushEvents"] } subgraph buffer_sink { - label = "Buffer Sink"; + label = "Buffer Sink (1)"; - buffer_sink_run [label = "Buffer Sink\nFlushEvents"] + buffer_sink_run [label = "Buffer Sink (1)\nFlushEvents"] } subgraph backend_sink { - label = "Backend Sink"; + label = "Backend Sink (1)"; - backend_sink_run [label = "Backend Sink\nFlushEvents"] + backend_sink_run [label = "Backend Sink (1)\nFlushEvents"] } table_sink_run -> buffer_sink_run [label = "S.EmitEvents"] diff --git a/docs/data-flow.svg b/docs/data-flow.svg index c3edfc5138b..c4684d16ff0 100644 --- a/docs/data-flow.svg +++ b/docs/data-flow.svg @@ -41,13 +41,13 @@ cluster_sorter - -Sorter + +Sorter cluster_entry_sorter - -Entry sorter + +Entry sorter cluster_mounter @@ -56,8 +56,8 @@ cluster_sink - -Sink Manager + +Sink Manager @@ -169,15 +169,15 @@ table_sink_run - -Table Sink -FlushEvents + +Table Sink (N) +FlushEvents processor_sorter_consume->table_sink_run - - + + P.output @@ -202,29 +202,29 @@ sorter_run - -Run + +Run processor_puller_consume->sorter_run - - -E.AddEntry + + +E.AddEntry buffer_sink_run - -Buffer Sink -FlushEvents + +Buffer Sink (1) +FlushEvents table_sink_run->buffer_sink_run - - -S.EmitEvents + + +S.EmitEvents @@ -236,42 +236,42 @@ sorter_run->processor_sorter_consume - + S.Ouput mounter_run->table_sink_run - - -wait unmarshal + + +wait unmarshal backend_sink_run - -Backend Sink -FlushEvents + +Backend Sink (1) +FlushEvents buffer_sink_run->backend_sink_run - - -S.EmitEvents + + +S.EmitEvents data_out - -Data out + +Data out backend_sink_run->data_out - - + + diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 28ecf484831..98f3faded1a 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -109,7 +109,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1610515993595, + "iteration": 1610607213210, "links": [], "panels": [ { @@ -2133,24 +2133,24 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{instance}}-{{type}}-p95", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", + "legendFormat": "{{instance}}-{{type}}-p99", "refId": "B" }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", + "legendFormat": "{{instance}}-{{type}}-p999", "refId": "C" } ], From 3cea12fff1008671c7ce50159f99ca6e50afcf47 Mon Sep 17 00:00:00 2001 From: leoppro Date: Thu, 14 Jan 2021 15:51:21 +0800 Subject: [PATCH 44/44] fix fmt --- cdc/sink/manager_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 0d8c4e3cd1a..c307ac7c93f 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -144,7 +144,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { defer manager.Close() goroutineNum := 10 var wg sync.WaitGroup - const EXIT_SIGNAL = uint64(math.MaxUint64) + const ExitSignal = uint64(math.MaxUint64) var maxResolvedTs uint64 tableSinks := make([]Sink, 0, goroutineNum) @@ -160,7 +160,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { default: } resolvedTs := atomic.LoadUint64(&maxResolvedTs) - if resolvedTs == EXIT_SIGNAL { + if resolvedTs == ExitSignal { return } if resolvedTs == lastResolvedTs { @@ -195,7 +195,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { wg.Add(1) go runTableSink(int64(i), table, maxResolvedTs, close) } else { - //remove table + // remove table table := tableSinks[0] close(closeChs[0]) c.Assert(table.Close(), check.IsNil) @@ -204,7 +204,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { } time.Sleep(100 * time.Millisecond) } - atomic.StoreUint64(&maxResolvedTs, EXIT_SIGNAL) + atomic.StoreUint64(&maxResolvedTs, ExitSignal) }() wg.Wait()