diff --git a/DEPS.bzl b/DEPS.bzl index ff6f339cb0c0c..6bf103f525ef2 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2510,8 +2510,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:VKMmvYhtG28j1sCCBdq4s+V9UOYqNgQ6CQviQwOgTeg=", - version = "v0.0.0-20220705090230-a5d4ffd2ba33", + sum = "h1:PAXtUVMJnyQQS8t9GzihIFmh6FBXu0JziWbIVknLniA=", + version = "v0.0.0-20220711062932-08b02befd813", ) go_repository( name = "com_github_pingcap_log", @@ -3012,8 +3012,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:RpH/obpgyNKkXV4Wt8PqSdcUTnqWyExPcla+qdTVgi0=", - version = "v2.0.1-0.20220711061028-1c198aab9585", + sum = "h1:nbcwXbkilywhMoAseLPzg/VHdFNhMEWy6JeqL/Gmq7A=", + version = "v2.0.1-0.20220713085647-57c12f7c64f6", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/go.mod b/go.mod index bd2953a3d9019..1d1d316434a1a 100644 --- a/go.mod +++ b/go.mod @@ -46,7 +46,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33 + github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813 github.com/pingcap/log v1.1.0 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e @@ -62,7 +62,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585 + github.com/tikv/client-go/v2 v2.0.1-0.20220713085647-57c12f7c64f6 github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index d586c031e94e9..58772530df713 100644 --- a/go.sum +++ b/go.sum @@ -673,9 +673,9 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20220302110454-c696585a961b/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20220525022339-6aaebf466305/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33 h1:VKMmvYhtG28j1sCCBdq4s+V9UOYqNgQ6CQviQwOgTeg= -github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220705053936-aa9c2d20cd2a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813 h1:PAXtUVMJnyQQS8t9GzihIFmh6FBXu0JziWbIVknLniA= +github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -800,8 +800,8 @@ github.com/tdakkota/asciicheck v0.1.1 h1:PKzG7JUTUmVspQTDqtkX9eSiLGossXTybutHwTX github.com/tdakkota/asciicheck v0.1.1/go.mod h1:yHp0ai0Z9gUljN3o0xMhYJnH/IcvkdTBOX2fmJ93JEM= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585 h1:RpH/obpgyNKkXV4Wt8PqSdcUTnqWyExPcla+qdTVgi0= -github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585/go.mod h1:uoZHYWKB+PsDueEnZ0EvF5zvNJPEauEWN26Tgi7qvNI= +github.com/tikv/client-go/v2 v2.0.1-0.20220713085647-57c12f7c64f6 h1:nbcwXbkilywhMoAseLPzg/VHdFNhMEWy6JeqL/Gmq7A= +github.com/tikv/client-go/v2 v2.0.1-0.20220713085647-57c12f7c64f6/go.mod h1:XLUJ0GyLzGp4Qk80lZQf1egpm+lYvSWRtK74DjZy8PI= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710/go.mod h1:AtvppPwkiyUgQlR1W9qSqfTB+OsOIu19jDCOxOsPkmU= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 7b15d3d6bbf64..ad32c8280230b 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -9311,6 +9311,117 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Time spent on the RPC layer between TiDB and TiKV, including the part used in the TiDB batch client", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 22 + }, + "hiddenSeries": false, + "id": 263, + "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": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_rpc_net_latency_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "99-store{{store}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_rpc_net_latency_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store) / sum(rate(tidb_tikvclient_rpc_net_latency_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-store{{store}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "RPC Layer Latency", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:132", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:133", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 30efc150f2214..db54f2c05c74a 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -86,6 +86,8 @@ const ( CommitBackoffTimeStr = "Commit_backoff_time" // BackoffTypesStr means the backoff type. BackoffTypesStr = "Backoff_types" + // SlowestCommitRPCDetailStr means the details of the slowest RPC during the transaction commit process. + SlowestCommitRPCDetailStr = "Slowest_commit_rpc_detail" // ResolveLockTimeStr means the time of resolving lock. ResolveLockTimeStr = "Resolve_lock_time" // LocalLatchWaitTimeStr means the time of waiting in local latch. @@ -98,6 +100,8 @@ const ( PrewriteRegionStr = "Prewrite_region" // TxnRetryStr means the count of transaction retry. TxnRetryStr = "Txn_retry" + // GetSnapshotTimeStr means the time spent on getting an engine snapshot. + GetSnapshotTimeStr = "Get_snapshot_time" // RocksdbDeleteSkippedCountStr means the count of rocksdb delete skipped count. RocksdbDeleteSkippedCountStr = "Rocksdb_delete_skipped_count" // RocksdbKeySkippedCountStr means the count of rocksdb key skipped count. @@ -108,6 +112,8 @@ const ( RocksdbBlockReadCountStr = "Rocksdb_block_read_count" // RocksdbBlockReadByteStr means the bytes of rocksdb block read. RocksdbBlockReadByteStr = "Rocksdb_block_read_byte" + // RocksdbBlockReadTimeStr means the time spent on rocksdb block read. + RocksdbBlockReadTimeStr = "Rocksdb_block_read_time" ) // String implements the fmt.Stringer interface. @@ -156,6 +162,12 @@ func (d ExecDetails) String() string { if len(commitDetails.Mu.BackoffTypes) > 0 { parts = append(parts, BackoffTypesStr+": "+fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes)) } + if commitDetails.Mu.SlowestReqTotalTime > 0 { + parts = append(parts, SlowestCommitRPCDetailStr+": {total:"+strconv.FormatFloat(commitDetails.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)+ + "s, region_id: "+strconv.FormatUint(commitDetails.Mu.SlowestRegion, 10)+ + ", store: "+commitDetails.Mu.SlowestStoreAddr+ + ", "+commitDetails.Mu.SlowestExecDetails.String()+"}") + } commitDetails.Mu.Unlock() resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLock.ResolveLockTime) if resolveLockTime > 0 { @@ -186,6 +198,9 @@ func (d ExecDetails) String() string { if scanDetail.TotalKeys > 0 { parts = append(parts, TotalKeysStr+": "+strconv.FormatInt(scanDetail.TotalKeys, 10)) } + if scanDetail.GetSnapshotDuration > 0 { + parts = append(parts, GetSnapshotTimeStr+": "+strconv.FormatFloat(scanDetail.GetSnapshotDuration.Seconds(), 'f', 3, 64)) + } if scanDetail.RocksdbDeleteSkippedCount > 0 { parts = append(parts, RocksdbDeleteSkippedCountStr+": "+strconv.FormatUint(scanDetail.RocksdbDeleteSkippedCount, 10)) } @@ -201,6 +216,9 @@ func (d ExecDetails) String() string { if scanDetail.RocksdbBlockReadByte > 0 { parts = append(parts, RocksdbBlockReadByteStr+": "+strconv.FormatUint(scanDetail.RocksdbBlockReadByte, 10)) } + if scanDetail.RocksdbBlockReadDuration > 0 { + parts = append(parts, RocksdbBlockReadTimeStr+": "+strconv.FormatFloat(scanDetail.RocksdbBlockReadDuration.Seconds(), 'f', 3, 64)) + } } return strings.Join(parts, " ") } @@ -893,6 +911,17 @@ func (e *RuntimeStatsWithCommit) String() string { } buf.WriteString("}") } + if e.Commit.Mu.SlowestReqTotalTime > 0 { + buf.WriteString(", slowest_commit_rpc: {total: ") + buf.WriteString(strconv.FormatFloat(e.Commit.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)) + buf.WriteString("s, region_id: ") + buf.WriteString(strconv.FormatUint(e.Commit.Mu.SlowestRegion, 10)) + buf.WriteString(", store: ") + buf.WriteString(e.Commit.Mu.SlowestStoreAddr) + buf.WriteString(", ") + buf.WriteString(e.Commit.Mu.SlowestExecDetails.String()) + buf.WriteString("}") + } e.Commit.Mu.Unlock() if e.Commit.ResolveLock.ResolveLockTime > 0 { buf.WriteString(", resolve_lock: ") @@ -939,17 +968,28 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", resolve_lock:") buf.WriteString(FormatDuration(time.Duration(e.LockKeys.ResolveLock.ResolveLockTime))) } + e.LockKeys.Mu.Lock() if e.LockKeys.BackoffTime > 0 { buf.WriteString(", backoff: {time: ") buf.WriteString(FormatDuration(time.Duration(e.LockKeys.BackoffTime))) - e.LockKeys.Mu.Lock() if len(e.LockKeys.Mu.BackoffTypes) > 0 { buf.WriteString(", type: ") buf.WriteString(e.formatBackoff(e.LockKeys.Mu.BackoffTypes)) } - e.LockKeys.Mu.Unlock() buf.WriteString("}") } + if e.LockKeys.Mu.SlowestReqTotalTime > 0 { + buf.WriteString(", slowest_rpc: {total: ") + buf.WriteString(strconv.FormatFloat(e.LockKeys.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)) + buf.WriteString("s, region_id: ") + buf.WriteString(strconv.FormatUint(e.LockKeys.Mu.SlowestRegion, 10)) + buf.WriteString(", store: ") + buf.WriteString(e.LockKeys.Mu.SlowestStoreAddr) + buf.WriteString(", ") + buf.WriteString(e.LockKeys.Mu.SlowestExecDetails.String()) + buf.WriteString("}") + } + e.LockKeys.Mu.Unlock() if e.LockKeys.LockRPCTime > 0 { buf.WriteString(", lock_rpc:") buf.WriteString(time.Duration(e.LockKeys.LockRPCTime).String()) @@ -962,6 +1002,7 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", retry_count:") buf.WriteString(strconv.FormatInt(int64(e.LockKeys.RetryCount), 10)) } + buf.WriteString("}") } return buf.String() diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 8dc53473ee9f2..0495a833a13cd 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -39,14 +39,51 @@ func TestString(t *testing.T) { Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: int64(time.Second), BackoffTypes: []string{ "backoff1", "backoff2", }, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }, }, WriteKeys: 1, WriteSize: 1, @@ -64,6 +101,7 @@ func TestString(t *testing.T) { RocksdbBlockCacheHitCount: 1, RocksdbBlockReadCount: 1, RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: time.Millisecond, }, TimeDetail: util.TimeDetail{ ProcessTime: 2*time.Second + 5*time.Millisecond, @@ -71,8 +109,14 @@ func TestString(t *testing.T) { }, } expected := "Cop_time: 1.003 Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Prewrite_time: 1 Commit_time: 1 " + - "Get_commit_ts_time: 1 Get_latest_ts_time: 1 Commit_backoff_time: 1 Backoff_types: [backoff1 backoff2] Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1 " + - "Process_keys: 10 Total_keys: 100 Rocksdb_delete_skipped_count: 1 Rocksdb_key_skipped_count: 1 Rocksdb_block_cache_hit_count: 1 Rocksdb_block_read_count: 1 Rocksdb_block_read_byte: 100" + "Get_commit_ts_time: 1 Get_latest_ts_time: 1 Commit_backoff_time: 1 Backoff_types: [backoff1 backoff2] " + + "Slowest_commit_rpc_detail: {total:1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, " + + "scan_detail: {total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, " + + "block: {cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}} " + + "Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1 " + + "Process_keys: 10 Total_keys: 100 Rocksdb_delete_skipped_count: 1 Rocksdb_key_skipped_count: 1 Rocksdb_block_cache_hit_count: 1 Rocksdb_block_read_count: 1 Rocksdb_block_read_byte: 100 Rocksdb_block_read_time: 0.001" require.Equal(t, expected, detail.String()) detail = &ExecDetails{} require.Equal(t, "", detail.String()) @@ -133,7 +177,7 @@ func TestCopRuntimeStats(t *testing.T) { cop.scanDetail.RocksdbBlockReadCount = 0 // Print all fields even though the value of some fields is 0. str := "tikv_task:{proc max:1s, min:2ns, avg: 500ms, p80:1s, p95:1s, iters:4, tasks:2}, " + - "scan_detail: {total_process_keys: 0, total_process_keys_size: 0, total_keys: 15, rocksdb: {delete_skipped_count: 5, key_skipped_count: 0, block: {cache_hit_count: 10, read_count: 0, read_byte: 100 Bytes}}}" + "scan_detail: {total_keys: 15, rocksdb: {delete_skipped_count: 5, block: {cache_hit_count: 10, read_byte: 100 Bytes}}}" require.Equal(t, str, cop.String()) zeroScanDetail := util.ScanDetail{} @@ -185,11 +229,48 @@ func TestRuntimeStatsWithCommit(t *testing.T) { CommitTime: time.Second, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ - CommitBackoffTime: int64(time.Second), - BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }, }, WriteKeys: 3, WriteSize: 66, @@ -202,7 +283,13 @@ func TestRuntimeStatsWithCommit(t *testing.T) { stats := &RuntimeStatsWithCommit{ Commit: commitDetail, } - expect := "commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, backoff: {time: 1s, type: [backoff1 backoff2]}, resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" + expect := "commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, backoff: {time: 1s, type: [backoff1 backoff2]}, " + + "slowest_commit_rpc: {total: 1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, scan_detail: " + + "{total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, block: " + + "{cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}}, " + + "resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" require.Equal(t, expect, stats.String()) lockDetail := &util.LockKeysDetails{ @@ -212,12 +299,50 @@ func TestRuntimeStatsWithCommit(t *testing.T) { BackoffTime: int64(time.Second * 3), Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{ - "backoff4", - "backoff5", - "backoff5", - }}, + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails + }{ + BackoffTypes: []string{ + "backoff4", + "backoff5", + "backoff5", + }, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }}, LockRPCTime: int64(time.Second * 5), LockRPCCount: 50, RetryCount: 2, @@ -228,7 +353,13 @@ func TestRuntimeStatsWithCommit(t *testing.T) { stats = &RuntimeStatsWithCommit{ LockKeys: lockDetail, } - expect = "lock_keys: {time:1s, region:2, keys:10, resolve_lock:2s, backoff: {time: 3s, type: [backoff4 backoff5]}, lock_rpc:5s, rpc_count:50, retry_count:2}" + expect = "lock_keys: {time:1s, region:2, keys:10, resolve_lock:2s, backoff: {time: 3s, type: [backoff4 backoff5]}, " + + "slowest_rpc: {total: 1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, scan_detail: " + + "{total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, block: " + + "{cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}}, " + + "lock_rpc:5s, rpc_count:50, retry_count:2}" require.Equal(t, expect, stats.String()) } diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 041de2989241f..264ea838bde75 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -185,8 +185,12 @@ func TestAddStatement(t *testing.T) { LocalLatchTime: 50, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 1000, BackoffTypes: []string{boTxnLockName}, @@ -317,8 +321,12 @@ func TestAddStatement(t *testing.T) { LocalLatchTime: 5, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 100, BackoffTypes: []string{boTxnLockName}, @@ -603,8 +611,12 @@ func generateAnyExecInfo() *StmtExecInfo { LocalLatchTime: 10, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 200, BackoffTypes: []string{boTxnLockName},