From 840494dc1380bab667c0df72de7283c07c7c531e Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Mon, 7 Jun 2021 16:38:28 +0800 Subject: [PATCH] store/tikv: add and fix some metrics for the new retry logic (#25123) --- metrics/grafana/tidb.json | 236 ++++++++++++++++++++- store/tikv/2pc.go | 27 ++- store/tikv/metrics/metrics.go | 39 ++++ store/tikv/region_request.go | 10 +- store/tikv/retry/backoff.go | 12 +- store/tikv/txn.go | 8 +- store/tikv/util/execdetails.go | 8 +- util/execdetails/execdetails.go | 14 +- util/execdetails/execdetails_test.go | 39 ++-- util/stmtsummary/statement_summary.go | 10 +- util/stmtsummary/statement_summary_test.go | 65 +++--- 11 files changed, 385 insertions(+), 83 deletions(-) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 4053986976fcd..5c78605bb62c1 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -5063,7 +5063,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "This metric shows OPS of async commit transactions.", + "description": "This metric shows the OPS of different types of transactions.", "editable": true, "error": false, "fill": 1, @@ -5079,13 +5079,13 @@ "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -5103,6 +5103,13 @@ "stack": false, "steppedLine": false, "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_commit_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "2PC-{{type}}", + "refId": "C" + }, { "expr": "sum(rate(tidb_tikvclient_async_commit_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", @@ -5111,18 +5118,18 @@ "refId": "A" }, { - "refId": "B", "expr": "sum(rate(tidb_tikvclient_one_pc_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "intervalFactor": 1, "format": "time_series", - "legendFormat": "1PC-{{type}}" + "intervalFactor": 1, + "legendFormat": "1PC-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Async Commit Transaction Counter", + "title": "Transaction Types Per Second", "tooltip": { "msResolution": false, "shared": true, @@ -5159,6 +5166,123 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "99th percentile of backoff count and duration in a transaction commit", + "editable": true, + "error": false, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 46 + }, + "id": 224, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sort": "avg", + "sortDesc": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/count.*/", + "yaxis": 1 + }, + { + "alias": "/duration.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_count_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "count - {{instance}}", + "refId": "A", + "step": 40 + }, + { + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "duration - {{instance}}", + "refId": "B", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Transaction Commit .99 Backoff", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": "count", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": "duration", + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -7003,6 +7127,102 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "This metric shows the reasons of replica selector failure (which needs a backoff).", + "editable": true, + "error": false, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 21 + }, + "id": 223, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_replica_selector_failure_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Replica Selector Failure Per Second", + "tooltip": { + "msResolution": false, + "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": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 2f8a94422f2ef..49096254743dc 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1079,11 +1079,13 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitDetail := c.getDetail() commitDetail.PrewriteTime = time.Since(start) - // TODO(youjiali1995): Record the backoff time of the last finished batch. It doesn't make sense to aggregate all batches'. if bo.GetTotalSleep() > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) + boSleep := int64(bo.GetTotalSleep()) * int64(time.Millisecond) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, bo.GetTypes()...) + if boSleep > commitDetail.Mu.CommitBackoffTime { + commitDetail.Mu.CommitBackoffTime = boSleep + commitDetail.Mu.BackoffTypes = bo.GetTypes() + } commitDetail.Mu.Unlock() } @@ -1242,8 +1244,8 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co err := c.commitMutations(commitBo, c.mutations) commitDetail.CommitTime = time.Since(start) if commitBo.GetTotalSleep() > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() + commitDetail.Mu.CommitBackoffTime += int64(commitBo.GetTotalSleep()) * int64(time.Millisecond) commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.GetTypes()...) commitDetail.Mu.Unlock() } @@ -1636,17 +1638,20 @@ func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork() defer singleBatchCancel() } - beforeSleep := singleBatchBackoffer.GetTotalSleep() ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch) commitDetail := batchExe.committer.getDetail() - if commitDetail != nil { // lock operations of pessimistic-txn will let commitDetail be nil - if delta := singleBatchBackoffer.GetTotalSleep() - beforeSleep; delta > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.GetTotalSleep()-beforeSleep)*int64(time.Millisecond)) - commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.GetTypes()...) - commitDetail.Mu.Unlock() + // For prewrite, we record the max backoff time + if _, ok := batchExe.action.(actionPrewrite); ok { + commitDetail.Mu.Lock() + boSleep := int64(singleBatchBackoffer.GetTotalSleep()) * int64(time.Millisecond) + if boSleep > commitDetail.Mu.CommitBackoffTime { + commitDetail.Mu.CommitBackoffTime = boSleep + commitDetail.Mu.BackoffTypes = singleBatchBackoffer.GetTypes() } + commitDetail.Mu.Unlock() } + // Backoff time in the 2nd phase of a non-async-commit txn is added + // in the commitTxn method, so we don't add it here. }() } else { logutil.Logger(batchExe.backoffer.GetCtx()).Info("break startWorker", diff --git a/store/tikv/metrics/metrics.go b/store/tikv/metrics/metrics.go index 6b8ea32d456f7..7f6cb97b9dde8 100644 --- a/store/tikv/metrics/metrics.go +++ b/store/tikv/metrics/metrics.go @@ -61,6 +61,10 @@ var ( TiKVTSFutureWaitDuration prometheus.Histogram TiKVSafeTSUpdateCounter *prometheus.CounterVec TiKVSafeTSUpdateStats *prometheus.GaugeVec + TiKVReplicaSelectorFailureCounter *prometheus.CounterVec + TiKVRequestRetryTimesHistogram prometheus.Histogram + TiKVTxnCommitBackoffSeconds prometheus.Histogram + TiKVTxnCommitBackoffCount prometheus.Histogram ) // Label constants. @@ -431,6 +435,37 @@ func initMetrics(namespace, subsystem string) { Name: "safets_update_stats", Help: "stat of tikv updating safe_ts stats", }, []string{LblStore}) + TiKVReplicaSelectorFailureCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "replica_selector_failure_counter", + Help: "Counter of the reason why the replica selector cannot yield a potential leader.", + }, []string{LblType}) + TiKVRequestRetryTimesHistogram = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "request_retry_times", + Help: "Bucketed histogram of how many times a region request retries.", + Buckets: []float64{1, 2, 3, 4, 8, 16, 32, 64, 128, 256}, + }) + TiKVTxnCommitBackoffSeconds = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "txn_commit_backoff_seconds", + Help: "Bucketed histogram of the total backoff duration in committing a transaction.", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2097s + }) + TiKVTxnCommitBackoffCount = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "txn_commit_backoff_count", + Help: "Bucketed histogram of the backoff count in committing a transaction.", + Buckets: prometheus.ExponentialBuckets(1, 2, 12), // 1 ~ 2048 + }) initShortcuts() } @@ -488,6 +523,10 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVTSFutureWaitDuration) prometheus.MustRegister(TiKVSafeTSUpdateCounter) prometheus.MustRegister(TiKVSafeTSUpdateStats) + prometheus.MustRegister(TiKVReplicaSelectorFailureCounter) + prometheus.MustRegister(TiKVRequestRetryTimesHistogram) + prometheus.MustRegister(TiKVTxnCommitBackoffSeconds) + prometheus.MustRegister(TiKVTxnCommitBackoffCount) } // readCounter reads the value of a prometheus.Counter. diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 26a8f5380faf0..61c82feb58f94 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -262,13 +262,15 @@ func (s *replicaSelector) nextReplica() *replica { const maxReplicaAttempt = 10 // next creates the RPCContext of the current candidate replica. -// It returns a SendError if runs out of all replicas of the cached region is invalidated. +// It returns a SendError if runs out of all replicas or the cached region is invalidated. func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { for { if !s.region.isValid() { + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("invalid").Inc() return nil, nil } if s.isExhausted() { + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() s.invalidateRegion() return nil, nil } @@ -284,6 +286,7 @@ func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { storeFailEpoch := atomic.LoadUint32(&replica.store.epoch) if storeFailEpoch != replica.epoch { // TODO(youjiali1995): Is it necessary to invalidate the region? + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("stale_store").Inc() s.invalidateRegion() return nil, nil } @@ -481,6 +484,11 @@ func (s *RegionRequestSender) SendReqCtx( s.reset() tryTimes := 0 + defer func() { + if tryTimes > 0 { + metrics.TiKVRequestRetryTimesHistogram.Observe(float64(tryTimes)) + } + }() for { if (tryTimes > 0) && (tryTimes%100 == 0) { logutil.Logger(bo.GetCtx()).Warn("retry", zap.Uint64("region", regionID.GetID()), zap.Int("times", tryTimes)) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index ecb7608275de2..485a179ced6c9 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -47,6 +47,7 @@ type Backoffer struct { configs []*Config backoffSleepMS map[string]int backoffTimes map[string]int + parent *Backoffer } type txnStartCtxKeyType struct{} @@ -196,6 +197,7 @@ func (b *Backoffer) Clone() *Backoffer { totalSleep: b.totalSleep, errors: b.errors, vars: b.vars, + parent: b.parent, } } @@ -209,6 +211,7 @@ func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { totalSleep: b.totalSleep, errors: b.errors, vars: b.vars, + parent: b, }, cancel } @@ -222,11 +225,14 @@ func (b *Backoffer) GetTotalSleep() int { return b.totalSleep } -// GetTypes returns type list. +// GetTypes returns type list of this backoff and all its ancestors. func (b *Backoffer) GetTypes() []string { typs := make([]string, 0, len(b.configs)) - for _, cfg := range b.configs { - typs = append(typs, cfg.String()) + for b != nil { + for _, cfg := range b.configs { + typs = append(typs, cfg.String()) + } + b = b.parent } return typs } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index f2c5fc24449c3..3cde92561d8ee 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -346,13 +346,19 @@ func (txn *KVTxn) Commit(ctx context.Context) error { } defer func() { + detail := committer.getDetail() + detail.Mu.Lock() + metrics.TiKVTxnCommitBackoffSeconds.Observe(float64(detail.Mu.CommitBackoffTime) / float64(time.Second)) + metrics.TiKVTxnCommitBackoffCount.Observe(float64(len(detail.Mu.BackoffTypes))) + detail.Mu.Unlock() + ctxValue := ctx.Value(util.CommitDetailCtxKey) if ctxValue != nil { commitDetail := ctxValue.(**util.CommitDetails) if *commitDetail != nil { (*commitDetail).TxnRetry++ } else { - *commitDetail = committer.getDetail() + *commitDetail = detail } } }() diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index 97c88a3687319..f8a8364fbb645 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -46,10 +46,10 @@ type CommitDetails struct { WaitPrewriteBinlogTime time.Duration CommitTime time.Duration LocalLatchTime time.Duration - CommitBackoffTime int64 Mu struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string } ResolveLockTime int64 WriteKeys int @@ -65,12 +65,12 @@ func (cd *CommitDetails) Merge(other *CommitDetails) { cd.WaitPrewriteBinlogTime += other.WaitPrewriteBinlogTime cd.CommitTime += other.CommitTime cd.LocalLatchTime += other.LocalLatchTime - cd.CommitBackoffTime += other.CommitBackoffTime cd.ResolveLockTime += other.ResolveLockTime cd.WriteKeys += other.WriteKeys cd.WriteSize += other.WriteSize cd.PrewriteRegionNum += other.PrewriteRegionNum cd.TxnRetry += other.TxnRetry + cd.Mu.CommitBackoffTime += other.Mu.CommitBackoffTime cd.Mu.BackoffTypes = append(cd.Mu.BackoffTypes, other.Mu.BackoffTypes...) } @@ -82,7 +82,6 @@ func (cd *CommitDetails) Clone() *CommitDetails { WaitPrewriteBinlogTime: cd.WaitPrewriteBinlogTime, CommitTime: cd.CommitTime, LocalLatchTime: cd.LocalLatchTime, - CommitBackoffTime: cd.CommitBackoffTime, ResolveLockTime: cd.ResolveLockTime, WriteKeys: cd.WriteKeys, WriteSize: cd.WriteSize, @@ -90,6 +89,7 @@ func (cd *CommitDetails) Clone() *CommitDetails { TxnRetry: cd.TxnRetry, } commit.Mu.BackoffTypes = append([]string{}, cd.Mu.BackoffTypes...) + commit.Mu.CommitBackoffTime = cd.Mu.CommitBackoffTime return commit } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index faed7f8c9eef7..ac16e1fa13f09 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -142,11 +142,11 @@ func (d ExecDetails) String() string { if commitDetails.GetCommitTsTime > 0 { parts = append(parts, GetCommitTSTimeStr+": "+strconv.FormatFloat(commitDetails.GetCommitTsTime.Seconds(), 'f', -1, 64)) } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) + commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime if commitBackoffTime > 0 { parts = append(parts, CommitBackoffTimeStr+": "+strconv.FormatFloat(time.Duration(commitBackoffTime).Seconds(), 'f', -1, 64)) } - commitDetails.Mu.Lock() if len(commitDetails.Mu.BackoffTypes) > 0 { parts = append(parts, BackoffTypesStr+": "+fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes)) } @@ -234,11 +234,11 @@ func (d ExecDetails) ToZapFields() (fields []zap.Field) { if commitDetails.GetCommitTsTime > 0 { fields = append(fields, zap.String("get_commit_ts_time", fmt.Sprintf("%v", strconv.FormatFloat(commitDetails.GetCommitTsTime.Seconds(), 'f', -1, 64)+"s"))) } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) + commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime if commitBackoffTime > 0 { fields = append(fields, zap.String("commit_backoff_time", fmt.Sprintf("%v", strconv.FormatFloat(time.Duration(commitBackoffTime).Seconds(), 'f', -1, 64)+"s"))) } - commitDetails.Mu.Lock() if len(commitDetails.Mu.BackoffTypes) > 0 { fields = append(fields, zap.String("backoff_types", fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes))) } @@ -803,18 +803,18 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", commit:") buf.WriteString(FormatDuration(e.Commit.CommitTime)) } - commitBackoffTime := atomic.LoadInt64(&e.Commit.CommitBackoffTime) + e.Commit.Mu.Lock() + commitBackoffTime := e.Commit.Mu.CommitBackoffTime if commitBackoffTime > 0 { buf.WriteString(", backoff: {time: ") buf.WriteString(FormatDuration(time.Duration(commitBackoffTime))) - e.Commit.Mu.Lock() if len(e.Commit.Mu.BackoffTypes) > 0 { buf.WriteString(", type: ") buf.WriteString(e.formatBackoff(e.Commit.Mu.BackoffTypes)) } - e.Commit.Mu.Unlock() buf.WriteString("}") } + e.Commit.Mu.Unlock() if e.Commit.ResolveLockTime > 0 { buf.WriteString(", resolve_lock: ") buf.WriteString(FormatDuration(time.Duration(e.Commit.ResolveLockTime))) diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 3d3a70959c180..461814e1b7d3e 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -34,18 +34,22 @@ func TestString(t *testing.T) { BackoffTime: time.Second, RequestCount: 1, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: time.Second, - PrewriteTime: time.Second, - CommitTime: time.Second, - LocalLatchTime: time.Second, - CommitBackoffTime: int64(time.Second), + GetCommitTsTime: time.Second, + PrewriteTime: time.Second, + CommitTime: time.Second, + LocalLatchTime: time.Second, + Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{ - "backoff1", - "backoff2", - }}, + CommitBackoffTime int64 + BackoffTypes []string + }{ + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{ + "backoff1", + "backoff2", + }, + }, ResolveLockTime: 1000000000, // 10^9 ns = 1s WriteKeys: 1, WriteSize: 1, @@ -200,14 +204,17 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) { } func TestRuntimeStatsWithCommit(t *testing.T) { commitDetail := &util.CommitDetails{ - GetCommitTsTime: time.Second, - PrewriteTime: time.Second, - CommitTime: time.Second, - CommitBackoffTime: int64(time.Second), + GetCommitTsTime: time.Second, + PrewriteTime: time.Second, + CommitTime: time.Second, Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}}, + CommitBackoffTime int64 + BackoffTypes []string + }{ + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + }, ResolveLockTime: int64(time.Second), WriteKeys: 3, WriteSize: 66, diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index acfdaf405cd12..149196e0a4c31 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -783,11 +783,6 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco if commitDetails.GetCommitTsTime > ssElement.maxGetCommitTsTime { ssElement.maxGetCommitTsTime = commitDetails.GetCommitTsTime } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) - ssElement.sumCommitBackoffTime += commitBackoffTime - if commitBackoffTime > ssElement.maxCommitBackoffTime { - ssElement.maxCommitBackoffTime = commitBackoffTime - } resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime) ssElement.sumResolveLockTime += resolveLockTime if resolveLockTime > ssElement.maxResolveLockTime { @@ -815,6 +810,11 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco ssElement.maxTxnRetry = commitDetails.TxnRetry } commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime + ssElement.sumCommitBackoffTime += commitBackoffTime + if commitBackoffTime > ssElement.maxCommitBackoffTime { + ssElement.maxCommitBackoffTime = commitBackoffTime + } ssElement.sumBackoffTimes += int64(len(commitDetails.Mu.BackoffTypes)) for _, backoffType := range commitDetails.Mu.BackoffTypes { ssElement.backoffTypes[backoffType] += 1 diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 751a8e501f715..f09398df68423 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -83,6 +83,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { planDigest: stmtExecInfo1.PlanDigest, } samplePlan, _ := stmtExecInfo1.PlanGenerator() + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() expectedSummaryElement := stmtSummaryByDigestElement{ beginTime: now + 60, endTime: now + 1860, @@ -120,8 +121,8 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { maxCommitTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitTime, sumLocalLatchTime: stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime, maxLocalLatchTime: stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime, - sumCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, - maxCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, + sumCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, + maxCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, sumResolveLockTime: stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, maxResolveLockTime: stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, sumWriteKeys: int64(stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys), @@ -141,6 +142,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { firstSeen: stmtExecInfo1.StartTime, lastSeen: stmtExecInfo1.StartTime, } + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Unlock() history := list.New() history.PushBack(&expectedSummaryElement) expectedSummary := stmtSummaryByDigest{ @@ -186,16 +188,17 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { BackoffTime: 180, RequestCount: 20, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 500, - PrewriteTime: 50000, - CommitTime: 5000, - LocalLatchTime: 50, - CommitBackoffTime: 1000, + GetCommitTsTime: 500, + PrewriteTime: 50000, + CommitTime: 5000, + LocalLatchTime: 50, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 1000, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 10000, WriteKeys: 100000, @@ -258,8 +261,10 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.maxCommitTime = stmtExecInfo2.ExecDetail.CommitDetail.CommitTime expectedSummaryElement.sumLocalLatchTime += stmtExecInfo2.ExecDetail.CommitDetail.LocalLatchTime expectedSummaryElement.maxLocalLatchTime = stmtExecInfo2.ExecDetail.CommitDetail.LocalLatchTime - expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo2.ExecDetail.CommitDetail.CommitBackoffTime - expectedSummaryElement.maxCommitBackoffTime = stmtExecInfo2.ExecDetail.CommitDetail.CommitBackoffTime + stmtExecInfo2.ExecDetail.CommitDetail.Mu.Lock() + expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo2.ExecDetail.CommitDetail.Mu.CommitBackoffTime + expectedSummaryElement.maxCommitBackoffTime = stmtExecInfo2.ExecDetail.CommitDetail.Mu.CommitBackoffTime + stmtExecInfo2.ExecDetail.CommitDetail.Mu.Unlock() expectedSummaryElement.sumResolveLockTime += stmtExecInfo2.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.maxResolveLockTime = stmtExecInfo2.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.sumWriteKeys += int64(stmtExecInfo2.ExecDetail.CommitDetail.WriteKeys) @@ -313,16 +318,17 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { BackoffTime: 18, RequestCount: 2, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 50, - PrewriteTime: 5000, - CommitTime: 500, - LocalLatchTime: 5, - CommitBackoffTime: 100, + GetCommitTsTime: 50, + PrewriteTime: 5000, + CommitTime: 500, + LocalLatchTime: 5, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 100, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 1000, WriteKeys: 10000, @@ -370,7 +376,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumPrewriteTime += stmtExecInfo3.ExecDetail.CommitDetail.PrewriteTime expectedSummaryElement.sumCommitTime += stmtExecInfo3.ExecDetail.CommitDetail.CommitTime expectedSummaryElement.sumLocalLatchTime += stmtExecInfo3.ExecDetail.CommitDetail.LocalLatchTime - expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo3.ExecDetail.CommitDetail.CommitBackoffTime + stmtExecInfo3.ExecDetail.CommitDetail.Mu.Lock() + expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo3.ExecDetail.CommitDetail.Mu.CommitBackoffTime + stmtExecInfo3.ExecDetail.CommitDetail.Mu.Unlock() expectedSummaryElement.sumResolveLockTime += stmtExecInfo3.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.sumWriteKeys += int64(stmtExecInfo3.ExecDetail.CommitDetail.WriteKeys) expectedSummaryElement.sumWriteSize += int64(stmtExecInfo3.ExecDetail.CommitDetail.WriteSize) @@ -569,16 +577,17 @@ func generateAnyExecInfo() *StmtExecInfo { BackoffTime: 80, RequestCount: 10, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 100, - PrewriteTime: 10000, - CommitTime: 1000, - LocalLatchTime: 10, - CommitBackoffTime: 200, + GetCommitTsTime: 100, + PrewriteTime: 10000, + CommitTime: 1000, + LocalLatchTime: 10, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 200, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 2000, WriteKeys: 20000, @@ -628,6 +637,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { n := types.NewTime(types.FromGoTime(time.Unix(s.ssMap.beginTimeForCurInterval, 0)), mysql.TypeTimestamp, types.DefaultFsp) e := types.NewTime(types.FromGoTime(time.Unix(s.ssMap.beginTimeForCurInterval+1800, 0)), mysql.TypeTimestamp, types.DefaultFsp) t := types.NewTime(types.FromGoTime(stmtExecInfo1.StartTime), mysql.TypeTimestamp, types.DefaultFsp) + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() expectedDatum := []interface{}{n, e, "Select", stmtExecInfo1.SchemaName, stmtExecInfo1.Digest, stmtExecInfo1.NormalizedSQL, "db1.tb1,db2.tb2", "a", stmtExecInfo1.User, 1, 0, 0, int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), @@ -646,7 +656,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { int64(stmtExecInfo1.ExecDetail.CommitDetail.PrewriteTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.PrewriteTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.CommitTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.CommitTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.GetCommitTsTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.GetCommitTsTime), - stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, + stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, int64(stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime), stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, @@ -656,6 +666,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { fmt.Sprintf("%s:1", boTxnLockName), stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, 0, 0, 0, 0, 0, stmtExecInfo1.StmtCtx.AffectedRows(), t, t, 0, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Unlock() match(c, datums[0], expectedDatum...) datums = s.ssMap.ToHistoryDatum(nil, true) c.Assert(len(datums), Equals, 1)