diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index 07e7bd9747b9b0..2bc61430fb881f 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//br/pkg/membuf", "//br/pkg/storage", "//pkg/kv", + "//pkg/metrics", "//pkg/sessionctx/variable", "//pkg/util/hack", "//pkg/util/logutil", diff --git a/br/pkg/lightning/backend/external/engine.go b/br/pkg/lightning/backend/external/engine.go index bf41cd9fd9af6b..55a8c41371ee34 100644 --- a/br/pkg/lightning/backend/external/engine.go +++ b/br/pkg/lightning/backend/external/engine.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/atomic" "go.uber.org/zap" @@ -197,12 +198,13 @@ func (e *Engine) loadIngestData( hex.EncodeToString(start)) } - now := time.Now() + startTs := time.Now() keys := make([][]byte, 0, 1024) values := make([][]byte, 0, 1024) memBuf := e.bufPool.NewBuffer() cnt := 0 size := 0 + totalSize := 0 largeRegion := e.regionSplitSize > 2*int64(config.SplitRegionSize) ret := make([]common.DataAndRange, 0, 1) curStart := start @@ -215,6 +217,7 @@ func (e *Engine) loadIngestData( values = append(values, memBuf.AddBytes(v)) cnt++ size += len(k) + len(v) + totalSize += len(k) + len(v) } for iter.Next() { @@ -241,13 +244,16 @@ func (e *Engine) loadIngestData( values = append(values, memBuf.AddBytes(v)) cnt++ size += len(k) + len(v) + totalSize += len(k) + len(v) } if iter.Error() != nil { return nil, errors.Trace(iter.Error()) } + metrics.GlobalSortReadFromCloudStorageRate.WithLabelValues("read_and_sort").Observe(float64(totalSize) / 1024.0 / 1024.0 / time.Since(startTs).Seconds()) + metrics.GlobalSortReadFromCloudStorageDuration.WithLabelValues("read_and_sort").Observe(time.Since(startTs).Seconds()) logutil.Logger(ctx).Info("load data from external storage", - zap.Duration("cost time", time.Since(now)), + zap.Duration("cost time", time.Since(startTs)), zap.Int("iterated count", cnt)) ret = append(ret, common.DataAndRange{ Data: e.buildIngestData(keys, values, memBuf), diff --git a/br/pkg/lightning/backend/external/writer.go b/br/pkg/lightning/backend/external/writer.go index fc324bb9ee4824..b651e01b19c591 100644 --- a/br/pkg/lightning/backend/external/writer.go +++ b/br/pkg/lightning/backend/external/writer.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/storage" tidbkv "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/size" @@ -385,7 +386,10 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { } ts := time.Now() - var savedBytes uint64 + savedBytes := w.batchSize + + startTs := time.Now() + var startTsForWrite time.Time defer func() { w.currentSeq++ @@ -407,6 +411,10 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { zap.Duration("time", time.Since(ts)), zap.Uint64("bytes", savedBytes), zap.Any("rate", float64(savedBytes)/1024.0/1024.0/time.Since(ts).Seconds())) + metrics.GlobalSortWriteToCloudStorageDuration.WithLabelValues("write").Observe(time.Since(startTsForWrite).Seconds()) + metrics.GlobalSortWriteToCloudStorageRate.WithLabelValues("write").Observe(float64(savedBytes) / 1024.0 / 1024.0 / time.Since(startTsForWrite).Seconds()) + metrics.GlobalSortWriteToCloudStorageDuration.WithLabelValues("sort_and_write").Observe(time.Since(startTs).Seconds()) + metrics.GlobalSortWriteToCloudStorageRate.WithLabelValues("sort_and_write").Observe(float64(savedBytes) / 1024.0 / 1024.0 / time.Since(startTs).Seconds()) }() sorty.MaxGor = min(8, uint64(variable.GetDDLReorgWorkerCounter())) @@ -420,6 +428,10 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { return false }) + startTsForWrite = time.Now() + metrics.GlobalSortWriteToCloudStorageDuration.WithLabelValues("sort").Observe(time.Since(startTs).Seconds()) + metrics.GlobalSortWriteToCloudStorageRate.WithLabelValues("sort").Observe(float64(savedBytes) / 1024.0 / 1024.0 / time.Since(startTs).Seconds()) + w.kvStore, err = NewKeyValueStore(ctx, dataWriter, w.rc) if err != nil { return err diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index 084d0500660909..34bfa91e5722f9 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -47,6 +47,7 @@ go_library( "//pkg/distsql", "//pkg/infoschema", "//pkg/kv", + "//pkg/metrics", "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/sessionctx/variable", diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 04fdd7fddf38ef..88819fa7d5f8a3 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/store/pdtypes" "github.com/pingcap/tidb/pkg/tablecodec" @@ -1322,6 +1323,7 @@ func (local *Backend) startWorker( jobInCh, jobOutCh chan *regionJob, jobWg *sync.WaitGroup, ) error { + metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Set(0) for { select { case <-ctx.Done(): @@ -1333,7 +1335,9 @@ func (local *Backend) startWorker( return nil } + metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Inc() err := local.executeJob(ctx, job) + metrics.GlobalSortIngestWorkerCnt.WithLabelValues("execute job").Dec() switch job.stage { case regionScanned, wrote, ingested: jobOutCh <- job diff --git a/pkg/ddl/backfilling_operators.go b/pkg/ddl/backfilling_operators.go index 6cfe8ee932f967..f900cc229dccb4 100644 --- a/pkg/ddl/backfilling_operators.go +++ b/pkg/ddl/backfilling_operators.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/copr" "github.com/pingcap/tidb/pkg/ddl/ingest" "github.com/pingcap/tidb/pkg/ddl/internal/session" + util2 "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/disttask/operator" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" @@ -442,7 +443,7 @@ func (w *tableScanWorker) scanRecords(task TableScanTask, sender func(IndexRecor for !done { srcChk := w.getChunk() done, err = fetchTableScanResult(w.ctx, w.copCtx.GetBase(), rs, srcChk) - if err != nil { + if err != nil || util2.IsContextDone(w.ctx) { w.recycleChunk(srcChk) terror.Call(rs.Close) return err diff --git a/pkg/ddl/index_cop.go b/pkg/ddl/index_cop.go index 9c231d54a34498..2e797591e806a0 100644 --- a/pkg/ddl/index_cop.go +++ b/pkg/ddl/index_cop.go @@ -18,6 +18,7 @@ import ( "context" "encoding/hex" "sync" + "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -146,6 +147,7 @@ func scanRecords(p *copReqSenderPool, task *reorgBackfillTask, se *sess.Session) p.checkpointMgr.Register(task.id, task.endKey) } var done bool + startTime := time.Now() for !done { srcChk := p.getChunk() done, err = fetchTableScanResult(p.ctx, p.copCtx.GetBase(), rs, srcChk) @@ -158,10 +160,13 @@ func scanRecords(p *copReqSenderPool, task *reorgBackfillTask, se *sess.Session) p.checkpointMgr.UpdateTotal(task.id, srcChk.NumRows(), done) } idxRs := IndexRecordChunk{ID: task.id, Chunk: srcChk, Done: done} + rate := float64(srcChk.MemoryUsage()) / 1024.0 / 1024.0 / time.Since(startTime).Seconds() + metrics.AddIndexScanRate.WithLabelValues(metrics.LblAddIndex).Observe(rate) failpoint.Inject("mockCopSenderError", func() { idxRs.Err = errors.New("mock cop error") }) p.chunkSender.AddTask(idxRs) + startTime = time.Now() } terror.Call(rs.Close) return nil diff --git a/pkg/metrics/BUILD.bazel b/pkg/metrics/BUILD.bazel index 5c515e7c9dab3b..d74c31f504dad6 100644 --- a/pkg/metrics/BUILD.bazel +++ b/pkg/metrics/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "domain.go", "executor.go", "gc_worker.go", + "globalsort.go", "import.go", "log_backup.go", "meta.go", diff --git a/pkg/metrics/ddl.go b/pkg/metrics/ddl.go index 3301bee8286e11..0a899fb3fc311a 100644 --- a/pkg/metrics/ddl.go +++ b/pkg/metrics/ddl.go @@ -60,6 +60,7 @@ var ( BackfillProgressGauge *prometheus.GaugeVec DDLJobTableDuration *prometheus.HistogramVec DDLRunningJobCount *prometheus.GaugeVec + AddIndexScanRate *prometheus.HistogramVec ) // InitDDLMetrics initializes defines DDL metrics. @@ -165,6 +166,14 @@ func InitDDLMetrics() { Name: "running_job_count", Help: "Running DDL jobs count", }, []string{LblType}) + + AddIndexScanRate = NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "ddl", + Name: "scan_rate", + Help: "scan rate", + Buckets: prometheus.ExponentialBuckets(0.05, 2, 20), + }, []string{LblType}) } // Label constants. diff --git a/pkg/metrics/globalsort.go b/pkg/metrics/globalsort.go new file mode 100644 index 00000000000000..51d7210fcb4400 --- /dev/null +++ b/pkg/metrics/globalsort.go @@ -0,0 +1,72 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import "github.com/prometheus/client_golang/prometheus" + +var ( + // GlobalSortWriteToCloudStorageDuration records the duration of writing to cloud storage. + GlobalSortWriteToCloudStorageDuration *prometheus.HistogramVec + // GlobalSortWriteToCloudStorageRate records the rate of writing to cloud storage. + GlobalSortWriteToCloudStorageRate *prometheus.HistogramVec + // GlobalSortReadFromCloudStorageDuration records the duration of reading from cloud storage. + GlobalSortReadFromCloudStorageDuration *prometheus.HistogramVec + // GlobalSortReadFromCloudStorageRate records the rate of reading from cloud storage. + GlobalSortReadFromCloudStorageRate *prometheus.HistogramVec + // GlobalSortIngestWorkerCnt records the working number of ingest workers. + GlobalSortIngestWorkerCnt *prometheus.GaugeVec +) + +// InitGlobalSortMetrics initializes defines global sort metrics. +func InitGlobalSortMetrics() { + GlobalSortWriteToCloudStorageDuration = NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "global_sort", + Name: "write_to_cloud_storage_duration", + Help: "write to cloud storage duration", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s + }, []string{LblType}) + + GlobalSortWriteToCloudStorageRate = NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "global_sort", + Name: "write_to_cloud_storage_rate", + Help: "write to cloud storage rate", + Buckets: prometheus.ExponentialBuckets(0.05, 2, 20), + }, []string{LblType}) + + GlobalSortReadFromCloudStorageDuration = NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "global_sort", + Name: "read_from_cloud_storage_duration", + Help: "read from cloud storage duration", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), + }, []string{LblType}) + + GlobalSortReadFromCloudStorageRate = NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "global_sort", + Name: "read_from_cloud_storage_rate", + Help: "read from cloud storage rate", + Buckets: prometheus.ExponentialBuckets(0.05, 2, 20), + }, []string{LblType}) + + GlobalSortIngestWorkerCnt = NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "global_sort", + Name: "ingest_worker_cnt", + Help: "ingest worker cnt", + }, []string{LblType}) +} diff --git a/pkg/metrics/grafana/tidb.json b/pkg/metrics/grafana/tidb.json index 1bbc20fa2b3d1e..698c31eeb06991 100644 --- a/pkg/metrics/grafana/tidb.json +++ b/pkg/metrics/grafana/tidb.json @@ -14027,6 +14027,120 @@ "align": false, "alignLevel": null } + }, + { + "id": 23763572001, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 48 + }, + "type": "graph", + "title": "Add Index Scan Rate", + "datasource": "${DS_TEST-CLUSTER}", + "thresholds": [], + "pluginVersion": "8.5.27", + "description": "Rate of scanning during adding index", + "links": [], + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "aliasColors": {}, + "dashLength": 10, + "fieldConfig": { + "defaults": { + "unit": "MiBs" + }, + "overrides": [] + }, + "fill": 1, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "pointradius": 5, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_ddl_scan_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-999", + "range": true, + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_ddl_scan_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-95", + "range": true, + "refId": "C" + }, + { + "expr": "histogram_quantile(0.50, sum(rate(tidb_ddl_scan_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-50", + "range": true, + "refId": "D" + } + ], + "timeRegions": [], + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "xaxis": { + "mode": "time", + "show": true, + "values": [], + "name": null, + "buckets": null + }, + "yaxes": [ + { + "$$hashKey": "object:598", + "format": "MiBs", + "logBase": 2, + "show": true + }, + { + "$$hashKey": "object:599", + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + }, + "bars": false, + "dashes": false, + "fillGradient": 0, + "hiddenSeries": false, + "percentage": false, + "points": false, + "stack": false, + "steppedLine": false, + "timeFrom": null, + "timeShift": null } ], "repeat": null, @@ -21717,6 +21831,538 @@ ], "title": "Import Into", "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 22 + }, + "id": 23763571995, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of writing to cloud storage", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 1 + }, + "hiddenSeries": false, + "id": 23763571996, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "8.5.27", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_global_sort_write_to_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-999", + "range": true, + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_global_sort_write_to_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-95", + "range": true, + "refId": "C" + }, + { + "expr": "histogram_quantile(0.50, sum(rate(tidb_global_sort_write_to_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-50", + "range": true, + "refId": "D" + } + ], + "thresholds": [], + "timeRegions": [], + "title": "Write To Cloud Storage Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "mode": "time", + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "logBase": 2, + "show": true + }, + { + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Rate of writing to cloud storage", + "fieldConfig": { + "defaults": { + "unit": "MiBs" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 1 + }, + "hiddenSeries": false, + "id": 23763571997, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "8.5.27", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_global_sort_write_to_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-999", + "range": true, + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_global_sort_write_to_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-95", + "range": true, + "refId": "C" + }, + { + "expr": "histogram_quantile(0.50, sum(rate(tidb_global_sort_write_to_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-50", + "range": true, + "refId": "D" + } + ], + "thresholds": [], + "timeRegions": [], + "title": "Write To Cloud Storage Rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "mode": "time", + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:598", + "format": "MiBs", + "logBase": 2, + "show": true + }, + { + "$$hashKey": "object:599", + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of reading from cloud storage", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 7 + }, + "hiddenSeries": false, + "id": 23763571998, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "8.5.27", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_global_sort_read_from_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-999", + "range": true, + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_global_sort_read_from_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-95", + "range": true, + "refId": "C" + }, + { + "expr": "histogram_quantile(0.50, sum(rate(tidb_global_sort_read_from_cloud_storage_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-50", + "range": true, + "refId": "D" + } + ], + "thresholds": [], + "timeRegions": [], + "title": "Read From Cloud Storage Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "mode": "time", + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "logBase": 2, + "show": true + }, + { + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Rate of reading from cloud storage", + "fieldConfig": { + "defaults": { + "unit": "MiBs" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 7 + }, + "hiddenSeries": false, + "id": 23763571999, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "8.5.27", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tidb_global_sort_read_from_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-999", + "range": true, + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_global_sort_read_from_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-95", + "range": true, + "refId": "C" + }, + { + "expr": "histogram_quantile(0.50, sum(rate(tidb_global_sort_read_from_cloud_storage_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-50", + "range": true, + "refId": "D" + } + ], + "thresholds": [], + "timeRegions": [], + "title": "Read From Cloud Storage Rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "mode": "time", + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:598", + "format": "MiBs", + "logBase": 2, + "show": true + }, + { + "$$hashKey": "object:599", + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Working worker count of ingest", + "fieldConfig": { + "defaults": { + "unit": "none" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 13 + }, + "hiddenSeries": false, + "id": 23763572000, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "8.5.27", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "tidb_global_sort_ingest_worker_cnt{k8s_cluster=\"$k8s_cluster\", tidb_cluster=~\"$tidb_cluster.*\", instance=~\"$instance\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "range": true, + "refId": "A" + } + ], + "thresholds": [], + "timeRegions": [], + "title": "Ingest Worker Count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "mode": "time", + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:598", + "format": "none", + "logBase": 2, + "show": true + }, + { + "$$hashKey": "object:599", + "format": "short", + "logBase": 1, + "show": true + } + ], + "yaxis": { + "align": false + } + } + ], + "title": "Global Sort", + "type": "row" } ], "refresh": "30s", diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index a90fc37697c3bf..a429dc130056f5 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -93,6 +93,7 @@ func InitMetrics() { InitTopSQLMetrics() InitTTLMetrics() InitDistTaskMetrics() + InitGlobalSortMetrics() timermetrics.InitTimerMetrics() PanicCounter = NewCounterVec( @@ -269,6 +270,13 @@ func RegisterMetrics() { prometheus.MustRegister(DistTaskSubTaskCntGauge) prometheus.MustRegister(DistTaskSubTaskStartTimeGauge) + prometheus.MustRegister(GlobalSortWriteToCloudStorageDuration) + prometheus.MustRegister(GlobalSortWriteToCloudStorageRate) + prometheus.MustRegister(GlobalSortReadFromCloudStorageDuration) + prometheus.MustRegister(GlobalSortReadFromCloudStorageRate) + prometheus.MustRegister(GlobalSortIngestWorkerCnt) + prometheus.MustRegister(AddIndexScanRate) + tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() tikvmetrics.TiKVPanicCounter = PanicCounter // reset tidb metrics for tikv metrics