Skip to content

Commit

Permalink
planner: add store type to statement summary (#17825) (#19159)
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-srebot authored Sep 2, 2020
1 parent 54840da commit 293da99
Show file tree
Hide file tree
Showing 4 changed files with 113 additions and 28 deletions.
32 changes: 16 additions & 16 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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")
Expand Down Expand Up @@ -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")
Expand All @@ -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 = ''")
Expand Down
15 changes: 14 additions & 1 deletion planner/core/encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"sync"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/util/plancodec"
)

Expand Down Expand Up @@ -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++
Expand Down
42 changes: 31 additions & 11 deletions util/plancodec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (

"github.com/golang/snappy"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/util/texttree"
)

Expand Down Expand Up @@ -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)
}
Expand All @@ -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)
Expand Down Expand Up @@ -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)
Expand Down
52 changes: 52 additions & 0 deletions util/plancodec/codec_test.go
Original file line number Diff line number Diff line change
@@ -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)
}

0 comments on commit 293da99

Please sign in to comment.