diff --git a/executor/inspection_profile.go b/executor/inspection_profile.go new file mode 100644 index 0000000000000..e06e218b88fdf --- /dev/null +++ b/executor/inspection_profile.go @@ -0,0 +1,479 @@ +// 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 executor + +import ( + "bytes" + "context" + "fmt" + "math" + "strings" + "time" + + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/sqlexec" +) + +type profileBuilder struct { + sctx sessionctx.Context + idMap map[string]uint64 + idAllocator uint64 + totalValue float64 + uniqueMap map[string]struct{} + buf *bytes.Buffer + start time.Time + end time.Time +} + +type metricNode struct { + table string + name string + label []string + condition string + labelValue map[string]float64 + value float64 + unit int64 + children []*metricNode + // isPartOfParent indicates the parent of this node not fully contain this node. + isPartOfParent bool + initialized bool +} + +func (n *metricNode) getName(label string) string { + name := n.table + if n.name != "" { + name = n.name + } + if len(label) != 0 { + name = name + "." + label + } + return name +} + +func (n *metricNode) getValue(pb *profileBuilder) (float64, error) { + if n.initialized { + return n.value, nil + } + n.initialized = true + v, err := pb.getMetricValue(n) + if err != nil { + return 0, err + } + if math.IsNaN(v) { + n.value = 0 + } else { + n.value = v + } + return n.value, nil +} + +func (pb *profileBuilder) getMetricValue(n *metricNode) (float64, error) { + if n.labelValue != nil { + return n.value, nil + } + n.labelValue = make(map[string]float64) + var query string + format := "2006-01-02 15:04:05" + queryCondition := fmt.Sprintf("where time >= '%v' and time <= '%v'", pb.start.Format(format), pb.end.Format(format)) + if n.condition != "" { + queryCondition += (" and " + n.condition) + } + if len(n.label) == 0 { + query = fmt.Sprintf("select sum(value), '' from `metrics_schema`.`%v_total_time` %v", n.table, queryCondition) + } else { + query = fmt.Sprintf("select sum(value), `%[3]s` from `metrics_schema`.`%[1]s_total_time` %[2]s group by `%[3]s` having sum(value) > 0", + n.table, queryCondition, strings.Join(n.label, "`,`")) + } + rows, _, err := pb.sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(context.Background(), query) + if err != nil { + return 0, err + } + if len(rows) == 0 || rows[0].Len() == 0 { + return 0, nil + } + + for _, row := range rows { + v := row.GetFloat64(0) + if n.unit != 0 { + v = v / float64(n.unit) + } + label := "" + for i := 1; i < row.Len(); i++ { + if i > 1 { + label += "," + } + label += row.GetString(i) + } + if label == "" && len(n.label) > 0 { + continue + } + n.labelValue[label] = v + n.value += v + } + return n.value, nil +} + +// NewProfileBuilder returns a new profileBuilder. +func NewProfileBuilder(sctx sessionctx.Context, start, end time.Time) *profileBuilder { + return &profileBuilder{ + sctx: sctx, + idMap: make(map[string]uint64), + idAllocator: uint64(1), + buf: bytes.NewBuffer(make([]byte, 0, 1024)), + uniqueMap: make(map[string]struct{}), + start: start, + end: end, + } +} + +// Collect uses to collect the related metric information. +func (pb *profileBuilder) Collect() error { + pb.buf.WriteString(fmt.Sprintf(`digraph "%s" {`, "tidb_profile")) + pb.buf.WriteByte('\n') + pb.buf.WriteString(`node [style=filled fillcolor="#f8f8f8"]`) + pb.buf.WriteByte('\n') + err := pb.addMetricTree(pb.genTiDBQueryTree(), "tidb_query_total_time") + if err != nil { + return err + } + return nil +} + +// Build returns the metric profile dot. +func (pb *profileBuilder) Build() []byte { + pb.buf.WriteByte('}') + return pb.buf.Bytes() +} + +func (pb *profileBuilder) getNameID(name string) uint64 { + if id, ok := pb.idMap[name]; ok { + return id + } + id := pb.idAllocator + pb.idAllocator++ + pb.idMap[name] = id + return id +} + +func (pb *profileBuilder) addMetricTree(root *metricNode, name string) error { + if root == nil { + return nil + } + pb.buf.WriteString(fmt.Sprintf(`subgraph %[1]s { "%[1]s" [shape=box fontsize=16 label="Type: %[1]s\lTime: %s\lDuration: %s\l"] }`, name, pb.start.String(), pb.end.Sub(pb.start).String())) + pb.buf.WriteByte('\n') + v, err := root.getValue(pb) + if err != nil { + return err + } + if v != 0 { + pb.totalValue = v + } else { + pb.totalValue = 1 + } + return pb.traversal(root) +} + +func (pb *profileBuilder) traversal(n *metricNode) error { + if n == nil { + return nil + } + nodeName := n.getName("") + if _, ok := pb.uniqueMap[nodeName]; ok { + return nil + } + pb.uniqueMap[nodeName] = struct{}{} + selfValue, err := n.getValue(pb) + if err != nil { + return err + } + + if pb.ignoreFraction(selfValue, pb.totalValue) { + return nil + } + totalChildrenValue := float64(0) + for _, child := range n.children { + childValue, err := child.getValue(pb) + if err != nil { + return err + } + pb.addNodeEdge(n, child, childValue) + if !child.isPartOfParent { + totalChildrenValue += childValue + } + } + selfCost := selfValue - totalChildrenValue + pb.addNode(n, selfCost, selfValue) + for _, child := range n.children { + err := pb.traversal(child) + if err != nil { + return err + } + } + return nil +} + +func (pb *profileBuilder) addNodeEdge(parent, child *metricNode, childValue float64) { + if pb.ignoreFraction(childValue, pb.totalValue) { + return + } + style := "" + if child.isPartOfParent { + style = "dotted" + } + if len(parent.label) == 0 { + label := "" + if !child.isPartOfParent { + label = fmt.Sprintf(" %.2fs", childValue) + } + pb.addEdge(parent.getName(""), child.getName(""), label, style, childValue) + } else { + for label, v := range parent.labelValue { + if pb.ignoreFraction(v, pb.totalValue) { + continue + } + pb.addEdge(parent.getName(label), child.getName(""), "", style, childValue) + } + } +} + +func (pb *profileBuilder) addNode(n *metricNode, selfCost, nodeTotal float64) { + name := n.getName("") + weight := selfCost + if len(n.label) > 0 { + for label, v := range n.labelValue { + if pb.ignoreFraction(v, pb.totalValue) { + continue + } + labelValue := fmt.Sprintf(" %.2fs", v) + pb.addEdge(n.getName(""), n.getName(label), labelValue, "", v) + labelValue = fmt.Sprintf("%s\n %.2fs (%.2f%%)", n.getName(label), v, v*100/pb.totalValue) + pb.addNodeDef(n.getName(label), labelValue, v, v) + } + weight = selfCost / 2 + // Since this node has labels, all cost was consume on the children, so the selfCost is 0. + selfCost = 0 + } + + label := fmt.Sprintf("%s\n %.2fs (%.2f%%)\nof %.2fs (%.2f%%)", + name, selfCost, selfCost*100/pb.totalValue, nodeTotal, nodeTotal*100/pb.totalValue) + pb.addNodeDef(n.getName(""), label, weight, selfCost) +} + +func (pb *profileBuilder) addNodeDef(name, labelValue string, fontWeight, colorWeight float64) { + baseFontSize, maxFontGrowth := 5, 18.0 + fontSize := baseFontSize + fontSize += int(math.Ceil(maxFontGrowth * math.Sqrt(math.Abs(fontWeight)/pb.totalValue))) + + pb.buf.WriteString(fmt.Sprintf(`N%d [label="%s" fontsize=%d shape=box color="%s" fillcolor="%s"]`, + pb.getNameID(name), labelValue, fontSize, + pb.dotColor(colorWeight/pb.totalValue, false), + pb.dotColor(colorWeight/pb.totalValue, true))) + pb.buf.WriteByte('\n') +} + +func (pb *profileBuilder) addEdge(from, to, label, style string, value float64) { + weight := 1 + int(math.Min(value*100/pb.totalValue, 100)) + color := pb.dotColor(value/pb.totalValue, false) + pb.buf.WriteString(fmt.Sprintf(`N%d -> N%d [`, pb.getNameID(from), pb.getNameID(to))) + if label != "" { + pb.buf.WriteString(fmt.Sprintf(` label="%s" `, label)) + } + if style != "" { + pb.buf.WriteString(fmt.Sprintf(` style="%s" `, style)) + } + pb.buf.WriteString(fmt.Sprintf(` weight=%d color="%s"]`, weight, color)) + pb.buf.WriteByte('\n') +} + +func (pb *profileBuilder) ignoreFraction(value, total float64) bool { + return value*100/total < 0.01 +} + +// dotColor function is copy from https://github.com/google/pprof. +func (pb *profileBuilder) dotColor(score float64, isBackground bool) string { + // A float between 0.0 and 1.0, indicating the extent to which + // colors should be shifted away from grey (to make positive and + // negative values easier to distinguish, and to make more use of + // the color range.) + const shift = 0.7 + // Saturation and value (in hsv colorspace) for background colors. + const bgSaturation = 0.1 + const bgValue = 0.93 + // Saturation and value (in hsv colorspace) for foreground colors. + const fgSaturation = 1.0 + const fgValue = 0.7 + // Choose saturation and value based on isBackground. + var saturation float64 + var value float64 + if isBackground { + saturation = bgSaturation + value = bgValue + } else { + saturation = fgSaturation + value = fgValue + } + + // Limit the score values to the range [-1.0, 1.0]. + score = math.Max(-1.0, math.Min(1.0, score)) + + // Reduce saturation near score=0 (so it is colored grey, rather than yellow). + if math.Abs(score) < 0.2 { + saturation *= math.Abs(score) / 0.2 + } + + // Apply 'shift' to move scores away from 0.0 (grey). + if score > 0.0 { + score = math.Pow(score, (1.0 - shift)) + } + if score < 0.0 { + score = -math.Pow(-score, (1.0 - shift)) + } + + var r, g, b float64 // red, green, blue + if score < 0.0 { + g = value + r = value * (1 + saturation*score) + } else { + r = value + g = value * (1 - saturation*score) + } + b = value * (1 - saturation) + return fmt.Sprintf("#%02x%02x%02x", uint8(r*255.0), uint8(g*255.0), uint8(b*255.0)) +} + +func (pb *profileBuilder) genTiDBQueryTree() *metricNode { + tidbKVRequest := &metricNode{ + table: "tidb_kv_request", + isPartOfParent: true, + label: []string{"type"}, + children: []*metricNode{ + { + table: "tidb_batch_client_wait", + }, + { + table: "tidb_batch_client_wait_conn", + }, + { + table: "tidb_batch_client_unavailable", + }, + { + table: "pd_client_cmd", + condition: "type not in ('tso','wait','tso_async_wait')", + }, + { + table: "tikv_grpc_message", + children: []*metricNode{ + { + table: "tikv_cop_request", + children: []*metricNode{ + { + table: "tikv_cop_wait", + label: []string{"type"}, + }, + {table: "tikv_cop_handle"}, + }, + }, + { + table: "tikv_scheduler_command", + children: []*metricNode{ + {table: "tikv_scheduler_latch_wait"}, + {table: "tikv_scheduler_processing_read"}, + { + table: "tikv_storage_async_request", + children: []*metricNode{ + { + table: "tikv_storage_async_request", + name: "tikv_storage_async_request.snapshot", + condition: "type='snapshot'", + }, + { + table: "tikv_storage_async_request", + name: "tikv_storage_async_request.write", + condition: "type='write'", + children: []*metricNode{ + {table: "tikv_raftstore_propose_wait"}, + { + table: "tikv_raftstore_process", + children: []*metricNode{ + {table: "tikv_raftstore_append_log"}, + }, + }, + {table: "tikv_raftstore_commit_log"}, + {table: "tikv_raftstore_apply_wait"}, + {table: "tikv_raftstore_apply_log"}, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + tidbExecute := &metricNode{ + table: "tidb_execute", + children: []*metricNode{ + { + table: "pd_start_tso_wait", + }, + { + table: "tidb_auto_id_request", + }, + { + table: "tidb_cop", + isPartOfParent: true, + children: []*metricNode{ + { + table: "tidb_kv_backoff", + label: []string{"type"}, + isPartOfParent: true, + }, + tidbKVRequest, + }, + }, + { + table: "tidb_txn_cmd", + label: []string{"type"}, + children: []*metricNode{ + { + table: "tidb_kv_backoff", + label: []string{"type"}, + isPartOfParent: true, + }, + tidbKVRequest, + }, + }, + }, + } + queryTime := &metricNode{ + table: "tidb_query", + label: []string{"sql_type"}, + children: []*metricNode{ + { + table: "tidb_get_token", + unit: int64(10e5), + }, + { + table: "tidb_parse", + }, + { + table: "tidb_compile", + }, + tidbExecute, + }, + } + + return queryTime +} diff --git a/executor/inspection_summary.go b/executor/inspection_summary.go index 78742b73f12e3..8ff8b7a4c64d2 100644 --- a/executor/inspection_summary.go +++ b/executor/inspection_summary.go @@ -71,22 +71,22 @@ var inspectionSummaryRules = map[string][]string{ "tidb_kv_backoff_duration", "tidb_kv_request_duration", "pd_client_cmd_duration", - "tikv_grpc_messge_duration", + "tikv_grpc_message_duration", "tikv_average_grpc_messge_duration", "tikv_channel_full", "tikv_scheduler_is_busy", "tikv_coprocessor_is_busy", "tikv_engine_write_stall", - "tikv_apply_log_avg_duration", - "tikv_apply_log_duration", - "tikv_append_log_avg_duration", - "tikv_append_log_duration", - "tikv_commit_log_avg_duration", - "tikv_commit_log_duration", - "tikv_process_duration", - "tikv_propose_wait_duration", + "tikv_raftstore_apply_log_avg_duration", + "tikv_raftstore_apply_log_duration", + "tikv_raftstore_append_log_avg_duration", + "tikv_raftstore_append_log_duration", + "tikv_raftstore_commit_log_avg_duration", + "tikv_raftstore_commit_log_duration", + "tikv_raftstore_process_duration", + "tikv_raftstore_propose_wait_duration", "tikv_propose_avg_wait_duration", - "tikv_apply_wait_duration", + "tikv_raftstore_apply_wait_duration", "tikv_apply_avg_wait_duration", "tikv_check_split_duration", "tikv_storage_async_request_duration", @@ -155,7 +155,7 @@ var inspectionSummaryRules = map[string][]string{ "tikv_grpc_avg_req_batch_size", "tikv_grpc_avg_resp_batch_size", "tikv_grpc_errors", - "tikv_grpc_messge_duration", + "tikv_grpc_message_duration", "tikv_grpc_qps", "tikv_grpc_req_batch_size", "tikv_grpc_resp_batch_size", @@ -219,7 +219,7 @@ var inspectionSummaryRules = map[string][]string{ "tikv_grpc_avg_req_batch_size", "tikv_grpc_avg_resp_batch_size", "tikv_grpc_errors", - "tikv_grpc_messge_duration", + "tikv_grpc_message_duration", "tikv_grpc_qps", "tikv_grpc_req_batch_size", "tikv_grpc_resp_batch_size", @@ -238,15 +238,15 @@ var inspectionSummaryRules = map[string][]string{ "tikv_scheduler_stage", "tikv_scheduler_writing_bytes", "tikv_propose_avg_wait_duration", - "tikv_propose_wait_duration", - "tikv_append_log_avg_duration", - "tikv_append_log_duration", - "tikv_commit_log_avg_duration", - "tikv_commit_log_duration", + "tikv_raftstore_propose_wait_duration", + "tikv_raftstore_append_log_avg_duration", + "tikv_raftstore_append_log_duration", + "tikv_raftstore_commit_log_avg_duration", + "tikv_raftstore_commit_log_duration", "tikv_apply_avg_wait_duration", - "tikv_apply_log_avg_duration", - "tikv_apply_log_duration", - "tikv_apply_wait_duration", + "tikv_raftstore_apply_log_avg_duration", + "tikv_raftstore_apply_log_duration", + "tikv_raftstore_apply_wait_duration", "tikv_engine_wal_sync_operations", "tikv_engine_write_duration", "tikv_engine_write_operations", @@ -388,18 +388,18 @@ var inspectionSummaryRules = map[string][]string{ "tikv_approximate_avg_region_size", "tikv_approximate_region_size_histogram", "tikv_approximate_region_size", - "tikv_append_log_avg_duration", - "tikv_append_log_duration", - "tikv_commit_log_avg_duration", - "tikv_commit_log_duration", + "tikv_raftstore_append_log_avg_duration", + "tikv_raftstore_append_log_duration", + "tikv_raftstore_commit_log_avg_duration", + "tikv_raftstore_commit_log_duration", "tikv_apply_avg_wait_duration", - "tikv_apply_log_avg_duration", - "tikv_apply_log_duration", - "tikv_apply_wait_duration", - "tikv_process_duration", - "tikv_process_handled", + "tikv_raftstore_apply_log_avg_duration", + "tikv_raftstore_apply_log_duration", + "tikv_raftstore_apply_wait_duration", + "tikv_raftstore_process_duration", + "tikv_raftstore_process_handled", "tikv_propose_avg_wait_duration", - "tikv_propose_wait_duration", + "tikv_raftstore_propose_wait_duration", "tikv_raft_dropped_messages", "tikv_raft_log_speed", "tikv_raft_message_avg_batch_size", diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index f9ba0e21495dc..2f4be0f0811d4 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -655,6 +655,22 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance"}, Quantile: 0.95, }, + "tidb_batch_client_wait_conn_duration": { + PromQL: "histogram_quantile($QUANTILE, sum(rate(tidb_tikvclient_batch_client_wait_connection_establish_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le, instance))", + Labels: []string{"instance"}, + Quantile: 0.95, + Comment: "The quantile of batch client wait new connection establish durations", + }, + "tidb_batch_client_wait_conn_total_count": { + PromQL: "sum(increase(tidb_tikvclient_batch_client_wait_connection_establish_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + Labels: []string{"instance"}, + Comment: "The total count of batch client wait new connection establish", + }, + "tidb_batch_client_wait_conn_total_time": { + PromQL: "sum(increase(tidb_tikvclient_batch_client_wait_connection_establish_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + Labels: []string{"instance"}, + Comment: "The total time of batch client wait new connection establish", + }, "tidb_batch_client_unavailable_duration": { Comment: "The quantile of kv storage batch processing unvailable durations", PromQL: "histogram_quantile($QUANTILE, sum(rate(tidb_tikvclient_batch_client_unavailable_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le, instance))", @@ -1069,7 +1085,7 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The quantile size of requests into request batch per TiKV instance", }, - "tikv_grpc_messge_duration": { + "tikv_grpc_message_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_grpc_msg_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,type,instance))`, Labels: []string{"instance", "type"}, Quantile: 0.99, @@ -1138,33 +1154,33 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type"}, Comment: "The total number of peers validated by the PD worker", }, - "tikv_apply_log_avg_duration": { + "tikv_raftstore_apply_log_avg_duration": { PromQL: `sum(rate(tikv_raftstore_apply_log_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) / sum(rate(tikv_raftstore_apply_log_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) `, Labels: []string{"instance"}, Comment: "The average time consumed when Raft applies log", }, - "tikv_apply_log_duration": { + "tikv_raftstore_apply_log_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_apply_log_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))`, Labels: []string{"instance"}, Quantile: 0.99, Comment: "The quantile time consumed when Raft applies log", }, - "tikv_append_log_avg_duration": { + "tikv_raftstore_append_log_avg_duration": { PromQL: `sum(rate(tikv_raftstore_append_log_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) / sum(rate(tikv_raftstore_append_log_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION]))`, Labels: []string{"instance"}, Comment: "The avg time consumed when Raft appends log", }, - "tikv_append_log_duration": { + "tikv_raftstore_append_log_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_append_log_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))`, Labels: []string{"instance"}, Quantile: 0.99, Comment: "The quantile time consumed when Raft appends log", }, - "tikv_commit_log_avg_duration": { + "tikv_raftstore_commit_log_avg_duration": { PromQL: `sum(rate(tikv_raftstore_commit_log_duration_seconds_sum[$RANGE_DURATION])) / sum(rate(tikv_raftstore_commit_log_duration_seconds_count[$RANGE_DURATION]))`, Comment: "The time consumed when Raft commits log", }, - "tikv_commit_log_duration": { + "tikv_raftstore_commit_log_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_commit_log_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))`, Labels: []string{"instance"}, Quantile: 0.99, @@ -1175,12 +1191,12 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance"}, Comment: "The count of ready handled of Raft", }, - "tikv_process_handled": { + "tikv_raftstore_process_handled": { PromQL: `sum(rate(tikv_raftstore_raft_process_duration_secs_count{$LABEL_CONDITIONS}[$RANGE_DURATION]))`, Labels: []string{"instance", "type"}, Comment: "The count of different process type of Raft", }, - "tikv_process_duration": { + "tikv_raftstore_process_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_raft_process_duration_secs_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance,type))`, Labels: []string{"instance", "type"}, Quantile: 0.99, @@ -1249,7 +1265,7 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type"}, Comment: "The total number of proposals per type in raft", }, - "tikv_propose_wait_duration": { + "tikv_raftstore_propose_wait_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_request_wait_time_duration_secs_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))`, Labels: []string{"instance"}, Quantile: 0.99, @@ -1260,7 +1276,7 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance"}, Comment: "The average wait time of each proposal", }, - "tikv_apply_wait_duration": { + "tikv_raftstore_apply_wait_duration": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_raftstore_apply_wait_time_duration_secs_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))`, Labels: []string{"instance"}, Quantile: 0.99, @@ -1354,6 +1370,22 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type"}, Comment: "The average time which is caused by latch wait in command", }, + "tikv_scheduler_processing_read_duration": { + PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_scheduler_processing_read_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance,type))`, + Labels: []string{"instance", "type"}, + Quantile: 0.99, + Comment: "The quantile time of scheduler processing read in command", + }, + "tikv_scheduler_processing_read_total_count": { + PromQL: "sum(increase(tikv_scheduler_processing_read_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", + Labels: []string{"instance", "type"}, + Comment: "The total count of scheduler processing read in command", + }, + "tikv_scheduler_processing_read_total_time": { + PromQL: "sum(increase(tikv_scheduler_processing_read_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", + Labels: []string{"instance", "type"}, + Comment: "The total time of scheduler processing read in command", + }, "tikv_scheduler_keys_read": { PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_scheduler_kv_command_key_read_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance,type))`, @@ -1573,8 +1605,8 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The quantile of time consumed when handling coprocessor requests", }, "tikv_cop_wait_duration": { - PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_coprocessor_request_wait_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,req,instance))`, - Labels: []string{"instance", "req"}, + PromQL: `histogram_quantile($QUANTILE, sum(rate(tikv_coprocessor_request_wait_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,req,type,instance))`, + Labels: []string{"instance", "req", "type"}, Quantile: 1, Comment: "The quantile of time consumed when coprocessor requests are wait for being handled", }, @@ -2583,6 +2615,22 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "sql_type"}, Comment: "The total time of TiDB query durations(second)", }, + "tidb_txn_cmd_duration": { + PromQL: `histogram_quantile($QUANTILE, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,type,instance))`, + Labels: []string{"instance", "type"}, + Quantile: 0.90, + Comment: "The quantile of TiDB transaction command durations(second)", + }, + "tidb_txn_cmd_total_count": { + PromQL: "sum(increase(tidb_tikvclient_txn_cmd_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", + Labels: []string{"instance", "type"}, + Comment: "The total count of TiDB transaction command", + }, + "tidb_txn_cmd_total_time": { + PromQL: "sum(increase(tidb_tikvclient_txn_cmd_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", + Labels: []string{"instance", "type"}, + Comment: "The total time of TiDB transaction command", + }, "tidb_slow_query_cop_process_total_count": { PromQL: "sum(increase(tidb_server_slow_query_cop_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, @@ -2643,31 +2691,31 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type", "sql_type"}, Comment: "The total time of transaction execution durations, including retry(second)", }, - "tikv_append_log_total_count": { + "tikv_raftstore_append_log_total_count": { PromQL: "sum(increase(tikv_raftstore_append_log_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of Raft appends log", }, - "tikv_append_log_total_time": { + "tikv_raftstore_append_log_total_time": { PromQL: "sum(increase(tikv_raftstore_append_log_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of Raft appends log", }, - "tikv_apply_log_total_count": { + "tikv_raftstore_apply_log_total_count": { PromQL: "sum(increase(tikv_raftstore_apply_log_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of Raft applies log", }, - "tikv_apply_log_total_time": { + "tikv_raftstore_apply_log_total_time": { PromQL: "sum(increase(tikv_raftstore_apply_log_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of Raft applies log", }, - "tikv_apply_wait_total_count": { + "tikv_raftstore_apply_wait_total_count": { PromQL: "sum(increase(tikv_raftstore_apply_wait_time_duration_secs_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, }, - "tikv_apply_wait_total_time": { + "tikv_raftstore_apply_wait_total_time": { PromQL: "sum(increase(tikv_raftstore_apply_wait_time_duration_secs_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, }, @@ -2697,12 +2745,12 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance"}, Comment: "The total time of time consumed when running split check in .9999", }, - "tikv_commit_log_total_count": { + "tikv_raftstore_commit_log_total_count": { PromQL: "sum(increase(tikv_raftstore_commit_log_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of Raft commits log", }, - "tikv_commit_log_total_time": { + "tikv_raftstore_commit_log_total_time": { PromQL: "sum(increase(tikv_raftstore_commit_log_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of Raft commits log", @@ -2728,13 +2776,13 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The total time of time consumed to handle coprocessor read requests", }, "tikv_cop_wait_total_count": { - PromQL: "sum(increase(tikv_coprocessor_request_wait_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,req)", - Labels: []string{"instance", "req"}, + PromQL: "sum(increase(tikv_coprocessor_request_wait_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,req,type)", + Labels: []string{"instance", "req", "type"}, Comment: "The total count of coprocessor requests that wait for being handled", }, "tikv_cop_wait_total_time": { - PromQL: "sum(increase(tikv_coprocessor_request_wait_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,req)", - Labels: []string{"instance", "req"}, + PromQL: "sum(increase(tikv_coprocessor_request_wait_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,req,type)", + Labels: []string{"instance", "req", "type"}, Comment: "The total time of time consumed when coprocessor requests are wait for being handled", }, "tikv_raft_store_events_total_count": { @@ -2757,12 +2805,12 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "task"}, Comment: "The total time of time consumed when executing GC tasks", }, - "tikv_grpc_messge_total_count": { + "tikv_grpc_message_total_count": { PromQL: "sum(increase(tikv_grpc_msg_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, Comment: "The total count of tikv execution gRPC message", }, - "tikv_grpc_messge_total_time": { + "tikv_grpc_message_total_time": { PromQL: "sum(increase(tikv_grpc_msg_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, Comment: "The total time of execution time of gRPC message", @@ -2803,22 +2851,22 @@ var MetricTableMap = map[string]MetricTableDef{ PromQL: "sum(increase(tikv_lock_manager_waiter_lifetime_duration_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, }, - "tikv_process_total_count": { + "tikv_raftstore_process_total_count": { PromQL: "sum(increase(tikv_raftstore_raft_process_duration_secs_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, Comment: "The total count of peer processes in Raft", }, - "tikv_process_total_time": { + "tikv_raftstore_process_total_time": { PromQL: "sum(increase(tikv_raftstore_raft_process_duration_secs_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, Comment: "The total time of peer processes in Raft", }, - "tikv_propose_wait_total_count": { + "tikv_raftstore_propose_wait_total_count": { PromQL: "sum(increase(tikv_raftstore_request_wait_time_duration_secs_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of each proposal", }, - "tikv_propose_wait_total_time": { + "tikv_raftstore_propose_wait_total_time": { PromQL: "sum(increase(tikv_raftstore_request_wait_time_duration_secs_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of wait time of each proposal", diff --git a/metrics/session.go b/metrics/session.go index 172236e03f8f1..4e775b01f3754 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -137,4 +137,6 @@ const ( LblInternal = "internal" LblStore = "store" LblAddress = "address" + LblBatchGet = "batch_get" + LblGet = "get" ) diff --git a/server/http_handler.go b/server/http_handler.go index 258fd67bc200e..867c0239a2b0b 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -402,6 +402,10 @@ type allServerInfoHandler struct { *tikvHandlerTool } +type profileHandler struct { + *tikvHandlerTool +} + // valueHandler is the handler for get value. type valueHandler struct { } @@ -1738,6 +1742,42 @@ func (h dbTableHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeData(w, dbTblInfo) } +// ServeHTTP handles request of TiDB metric profile. +func (h profileHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + sctx, err := session.CreateSession(h.Store) + if err != nil { + writeError(w, err) + return + } + var start, end time.Time + if req.FormValue("end") != "" { + end, err = time.ParseInLocation(time.RFC3339, req.FormValue("end"), sctx.GetSessionVars().Location()) + if err != nil { + writeError(w, err) + return + } + } else { + end = time.Now() + } + if req.FormValue("start") != "" { + start, err = time.ParseInLocation(time.RFC3339, req.FormValue("start"), sctx.GetSessionVars().Location()) + if err != nil { + writeError(w, err) + return + } + } else { + start = end.Add(-time.Minute * 10) + } + pb := executor.NewProfileBuilder(sctx, start, end) + err = pb.Collect() + if err != nil { + writeError(w, err) + return + } + _, err = w.Write(pb.Build()) + terror.Log(errors.Trace(err)) +} + // testHandler is the handler for tests. It's convenient to provide some APIs for integration tests. type testHandler struct { *tikvHandlerTool diff --git a/server/http_status.go b/server/http_status.go index 890f7e3ef65af..9049d494eb9b7 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -150,6 +150,8 @@ func (s *Server) startHTTPServer() { router.Handle("/mvcc/hex/{hexKey}", mvccTxnHandler{tikvHandlerTool, opMvccGetByHex}) router.Handle("/mvcc/index/{db}/{table}/{index}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) + // HTTP path for generate metric profile. + router.Handle("/metrics/profile", profileHandler{tikvHandlerTool}) // HTTP path for web UI. if host, port, err := net.SplitHostPort(s.statusAddr); err == nil { if host == "" { diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index bb21923eb2699..35895afa8f908 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -20,6 +20,11 @@ import ( "fmt" "strings" "sync" +<<<<<<< HEAD +======= + "sync/atomic" + "time" +>>>>>>> 9d536c4... *: add HTTP API to generate TiDB metric profile (#18272) "unsafe" "github.com/opentracing/opentracing-go" @@ -184,6 +189,9 @@ func appendBatchKeysBySize(b []batchKeys, region RegionVerID, keys [][]byte, siz } func (s *tikvSnapshot) batchGetKeysByRegions(bo *Backoffer, keys [][]byte, collectF func(k, v []byte)) error { + defer func(start time.Time) { + tikvTxnCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) + }(time.Now()) groups, _, err := s.store.regionCache.GroupKeysByRegion(bo, keys, nil) if err != nil { return errors.Trace(err) @@ -305,6 +313,10 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { ctx = opentracing.ContextWithSpan(ctx, span1) } + defer func(start time.Time) { + tikvTxnCmdHistogramWithGet.Observe(time.Since(start).Seconds()) + }(time.Now()) + ctx = context.WithValue(ctx, txnStartKey, s.version.Ver) val, err := s.get(NewBackofferWithVars(ctx, getMaxBackoff, s.vars), k) if err != nil { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 98ec24a76899c..47f27e2b074df 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -42,6 +42,8 @@ var ( var ( tikvTxnCmdHistogramWithCommit = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblCommit) tikvTxnCmdHistogramWithRollback = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblRollback) + tikvTxnCmdHistogramWithBatchGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblBatchGet) + tikvTxnCmdHistogramWithGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblGet) ) // tikvTxn implements kv.Transaction.