From 293da99d6389b37161623df0a49043038fec8f44 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 2 Sep 2020 10:15:22 +0800 Subject: [PATCH] planner: add store type to statement summary (#17825) (#19159) --- infoschema/tables_test.go | 32 +++++++++++----------- planner/core/encode.go | 15 ++++++++++- util/plancodec/codec.go | 42 +++++++++++++++++++++-------- util/plancodec/codec_test.go | 52 ++++++++++++++++++++++++++++++++++++ 4 files changed, 113 insertions(+), 28 deletions(-) create mode 100644 util/plancodec/codec_test.go diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 4e1f937ac849d..b0181439e83f3 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -953,10 +953,10 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { max_prewrite_regions, avg_affected_rows, query_sample_text, plan from information_schema.statements_summary where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("Select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask\testRows\toperator info\n" + - "\tIndexLookUp_10\troot\t100 \t\n" + - "\t├─IndexScan_8 \tcop \t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t100 \ttable:t, keep order:false, stats:pseudo")) + ).Check(testkit.Rows("Select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10\troot \t100 \t\n" + + "\t├─IndexScan_8 \tcop[tikv]\t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop[tikv]\t100 \ttable:t, keep order:false, stats:pseudo")) // select ... order by tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, @@ -974,10 +974,10 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { max_prewrite_regions, avg_affected_rows, query_sample_text, plan from information_schema.statements_summary where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("Select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask\testRows\toperator info\n" + - "\tIndexLookUp_10\troot\t100 \t\n" + - "\t├─IndexScan_8 \tcop \t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t100 \ttable:t, keep order:false, stats:pseudo")) + ).Check(testkit.Rows("Select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10\troot \t100 \t\n" + + "\t├─IndexScan_8 \tcop[tikv]\t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop[tikv]\t100 \ttable:t, keep order:false, stats:pseudo")) // Disable it again. tk.MustExec("set global tidb_enable_stmt_summary = false") @@ -1024,10 +1024,10 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { max_prewrite_regions, avg_affected_rows, query_sample_text, plan from information_schema.statements_summary where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("Select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask\testRows\toperator info\n" + - "\tIndexLookUp_10\troot\t1000 \t\n" + - "\t├─IndexScan_8 \tcop \t1000 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t1000 \ttable:t, keep order:false, stats:pseudo")) + ).Check(testkit.Rows("Select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10\troot \t1000 \t\n" + + "\t├─IndexScan_8 \tcop[tikv]\t1000 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop[tikv]\t1000 \ttable:t, keep order:false, stats:pseudo")) // Disable it in global scope. tk.MustExec("set global tidb_enable_stmt_summary = false") @@ -1043,10 +1043,10 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { max_prewrite_regions, avg_affected_rows, query_sample_text, plan from information_schema.statements_summary where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("Select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask\testRows\toperator info\n" + - "\tIndexLookUp_10\troot\t1000 \t\n" + - "\t├─IndexScan_8 \tcop \t1000 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t1000 \ttable:t, keep order:false, stats:pseudo")) + ).Check(testkit.Rows("Select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10\troot \t1000 \t\n" + + "\t├─IndexScan_8 \tcop[tikv]\t1000 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop[tikv]\t1000 \ttable:t, keep order:false, stats:pseudo")) // Unset session variable. tk.MustExec("set session tidb_enable_stmt_summary = ''") diff --git a/planner/core/encode.go b/planner/core/encode.go index e9f70f54b42e5..b98a02868cc66 100644 --- a/planner/core/encode.go +++ b/planner/core/encode.go @@ -21,6 +21,7 @@ import ( "sync" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/plancodec" ) @@ -57,8 +58,20 @@ func (pn *planEncoder) encodePlanTree(p PhysicalPlan) string { } func (pn *planEncoder) encodePlan(p PhysicalPlan, isRoot bool, depth int) { + var storeType kv.StoreType = kv.UnSpecified + if !isRoot { + switch copPlan := p.(type) { + case *PhysicalTableReader: + storeType = copPlan.StoreType + case *PhysicalTableScan: + storeType = copPlan.StoreType + default: + storeType = kv.TiKV + } + } + taskTypeInfo := plancodec.EncodeTaskType(isRoot, storeType) actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfo(p.SCtx(), p) - plancodec.EncodePlanNode(depth, p.ID(), p.TP(), isRoot, p.statsInfo().RowCount, p.ExplainInfo(), actRows, analyzeInfo, memoryInfo, diskInfo, &pn.buf) + plancodec.EncodePlanNode(depth, p.ID(), p.TP(), p.statsInfo().RowCount, taskTypeInfo, p.ExplainInfo(), actRows, analyzeInfo, memoryInfo, diskInfo, &pn.buf) pn.encodedPlans[p.ID()] = true depth++ diff --git a/util/plancodec/codec.go b/util/plancodec/codec.go index 0b4d683c7fae0..60966aedea4d5 100644 --- a/util/plancodec/codec.go +++ b/util/plancodec/codec.go @@ -22,6 +22,7 @@ import ( "github.com/golang/snappy" "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/texttree" ) @@ -294,11 +295,11 @@ func decodePlanInfo(str string) (*planInfo, error) { p.fields = append(p.fields, PhysicalIDToTypeString(planID)+idSeparator+ids[1]) // task type case 2: - if v == rootTaskType { - p.fields = append(p.fields, "root") - } else { - p.fields = append(p.fields, "cop") + task, err := decodeTaskType(v) + if err != nil { + return nil, errors.Errorf("decode plan: %v, task type: %v, error: %v", str, v, err) } + p.fields = append(p.fields, task) default: p.fields = append(p.fields, v) } @@ -307,17 +308,13 @@ func decodePlanInfo(str string) (*planInfo, error) { } // EncodePlanNode is used to encode the plan to a string. -func EncodePlanNode(depth, pid int, planType string, isRoot bool, rowCount float64, - explainInfo, actRows, analyzeInfo, memoryInfo, diskInfo string, buf *bytes.Buffer) { +func EncodePlanNode(depth, pid int, planType string, rowCount float64, + taskTypeInfo, explainInfo, actRows, analyzeInfo, memoryInfo, diskInfo string, buf *bytes.Buffer) { buf.WriteString(strconv.Itoa(depth)) buf.WriteByte(separator) buf.WriteString(encodeID(planType, pid)) buf.WriteByte(separator) - if isRoot { - buf.WriteString(rootTaskType) - } else { - buf.WriteString(copTaskType) - } + buf.WriteString(taskTypeInfo) buf.WriteByte(separator) buf.WriteString(strconv.FormatFloat(rowCount, 'f', -1, 64)) buf.WriteByte(separator) @@ -357,6 +354,29 @@ func encodeID(planType string, id int) string { return strconv.Itoa(planID) + idSeparator + strconv.Itoa(id) } +// EncodeTaskType is used to encode task type to a string. +func EncodeTaskType(isRoot bool, storeType kv.StoreType) string { + if isRoot { + return rootTaskType + } + return copTaskType + idSeparator + strconv.Itoa((int)(storeType)) +} + +func decodeTaskType(str string) (string, error) { + segs := strings.Split(str, idSeparator) + if segs[0] == rootTaskType { + return "root", nil + } + if len(segs) == 1 { // be compatible to `NormalizePlanNode`, which doesn't encode storeType in task field. + return "cop", nil + } + storeType, err := strconv.Atoi(segs[1]) + if err != nil { + return "", err + } + return "cop[" + ((kv.StoreType)(storeType)).Name() + "]", nil +} + // Compress is used to compress the input with zlib. func Compress(input []byte) string { compressBytes := snappy.Encode(nil, input) diff --git a/util/plancodec/codec_test.go b/util/plancodec/codec_test.go new file mode 100644 index 0000000000000..1f98adda4cf99 --- /dev/null +++ b/util/plancodec/codec_test.go @@ -0,0 +1,52 @@ +// 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 plancodec + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/kv" +) + +var _ = Suite(&testPlanCodecSuite{}) + +type testPlanCodecSuite struct{} + +type encodeTaskTypeCase struct { + IsRoot bool + StoreType kv.StoreType + EncodedStr string + DecodedStr string +} + +func (s *testPlanCodecSuite) TestEncodeTaskType(c *C) { + cases := []encodeTaskTypeCase{ + {true, kv.UnSpecified, "0", "root"}, + {false, kv.TiKV, "1_0", "cop[tikv]"}, + {false, kv.TiFlash, "1_1", "cop[tiflash]"}, + {false, kv.TiDB, "1_2", "cop[tidb]"}, + } + for _, cas := range cases { + c.Assert(EncodeTaskType(cas.IsRoot, cas.StoreType), Equals, cas.EncodedStr) + str, err := decodeTaskType(cas.EncodedStr) + c.Assert(err, IsNil) + c.Assert(str, Equals, cas.DecodedStr) + } + + str, err := decodeTaskType("1") + c.Assert(err, IsNil) + c.Assert(str, Equals, "cop") + + _, err = decodeTaskType("1_x") + c.Assert(err, NotNil) +}