Skip to content

Commit

Permalink
*: support the binary plan through EXPLAIN format='binary' and tidb_d…
Browse files Browse the repository at this point in the history
…ecode_binary_plan (pingcap#36221)

ref pingcap#35889
  • Loading branch information
time-and-fate authored Jul 18, 2022
1 parent 5dcf2ef commit 4cade24
Show file tree
Hide file tree
Showing 19 changed files with 1,240 additions and 18 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -2531,8 +2531,8 @@ def go_deps():
name = "com_github_pingcap_tipb",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/tipb",
sum = "h1:XaTE4ZhQbQtQZtAVzlZh/Pf6SjFfMSTe1ia2nGcl36Y=",
version = "v0.0.0-20220706024432-7be3cc83a7d5",
sum = "h1:hE1dQdnvxWCHhD0snX67paV9y6inq8TxVFbsKqjaTQk=",
version = "v0.0.0-20220714100504-7d3474676bc9",
)
go_repository(
name = "com_github_pkg_browser",
Expand Down
1 change: 0 additions & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4361,7 +4361,6 @@ func TestGetResultRowsCount(t *testing.T) {
require.True(t, ok)
cnt := executor.GetResultRowsCount(tk.Session().GetSessionVars().StmtCtx, p)
require.Equal(t, ca.row, cnt, fmt.Sprintf("sql: %v", ca.sql))
require.Equal(t, cnt, ca.row, fmt.Sprintf("sql: %v", ca.sql))
}
}

Expand Down
2 changes: 1 addition & 1 deletion executor/showtest/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1445,7 +1445,7 @@ func TestShowBuiltin(t *testing.T) {
res := tk.MustQuery("show builtins;")
require.NotNil(t, res)
rows := res.Rows()
const builtinFuncNum = 275
const builtinFuncNum = 276
require.Equal(t, len(rows), builtinFuncNum)
require.Equal(t, rows[0][0].(string), "abs")
require.Equal(t, rows[builtinFuncNum-1][0].(string), "yearweek")
Expand Down
1 change: 1 addition & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -834,6 +834,7 @@ var funcs = map[string]functionClass{
ast.TiDBVersion: &tidbVersionFunctionClass{baseFunctionClass{ast.TiDBVersion, 0, 0}},
ast.TiDBIsDDLOwner: &tidbIsDDLOwnerFunctionClass{baseFunctionClass{ast.TiDBIsDDLOwner, 0, 0}},
ast.TiDBDecodePlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodePlan, 1, 1}},
ast.TiDBDecodeBinaryPlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodeBinaryPlan, 1, 1}},
ast.TiDBDecodeSQLDigests: &tidbDecodeSQLDigestsFunctionClass{baseFunctionClass{ast.TiDBDecodeSQLDigests, 1, 2}},

// TiDB Sequence function.
Expand Down
32 changes: 30 additions & 2 deletions expression/builtin_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/privilege"
Expand Down Expand Up @@ -938,8 +939,12 @@ func (c *tidbDecodePlanFunctionClass) getFunction(ctx sessionctx.Context, args [
if err != nil {
return nil, err
}
sig := &builtinTiDBDecodePlanSig{bf}
return sig, nil
if c.funcName == ast.TiDBDecodePlan {
return &builtinTiDBDecodePlanSig{bf}, nil
} else if c.funcName == ast.TiDBDecodeBinaryPlan {
return &builtinTiDBDecodeBinaryPlanSig{bf}, nil
}
return nil, errors.New("unknown decode plan function")
}

type builtinTiDBDecodePlanSig struct {
Expand All @@ -964,6 +969,29 @@ func (b *builtinTiDBDecodePlanSig) evalString(row chunk.Row) (string, bool, erro
return planTree, false, nil
}

type builtinTiDBDecodeBinaryPlanSig struct {
baseBuiltinFunc
}

func (b *builtinTiDBDecodeBinaryPlanSig) Clone() builtinFunc {
newSig := &builtinTiDBDecodeBinaryPlanSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
return newSig
}

func (b *builtinTiDBDecodeBinaryPlanSig) evalString(row chunk.Row) (string, bool, error) {
planString, isNull, err := b.args[0].EvalString(b.ctx, row)
if isNull || err != nil {
return "", isNull, err
}
planTree, err := plancodec.DecodeBinaryPlan(planString)
if err != nil {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
return "", false, nil
}
return planTree, false, nil
}

type nextValFunctionClass struct {
baseFunctionClass
}
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ require (
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
github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5
github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9
github.com/prometheus/client_golang v1.12.2
github.com/prometheus/client_model v0.2.0
github.com/prometheus/common v0.32.1
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -684,8 +684,8 @@ github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8=
github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops=
github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5 h1:XaTE4ZhQbQtQZtAVzlZh/Pf6SjFfMSTe1ia2nGcl36Y=
github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9 h1:hE1dQdnvxWCHhD0snX67paV9y6inq8TxVFbsKqjaTQk=
github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
Expand Down
1 change: 1 addition & 0 deletions parser/ast/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,7 @@ const (
TiDBVersion = "tidb_version"
TiDBIsDDLOwner = "tidb_is_ddl_owner"
TiDBDecodePlan = "tidb_decode_plan"
TiDBDecodeBinaryPlan = "tidb_decode_binary_plan"
TiDBDecodeSQLDigests = "tidb_decode_sql_digests"
FormatBytes = "format_bytes"
FormatNanoTime = "format_nano_time"
Expand Down
2 changes: 2 additions & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4892,6 +4892,8 @@ func TestExplain(t *testing.T) {
{"EXPLAIN FORMAT = JSON FOR CONNECTION 1", true, "EXPLAIN FORMAT = 'JSON' FOR CONNECTION 1"},
{"EXPLAIN FORMAT = JSON SELECT 1", true, "EXPLAIN FORMAT = 'JSON' SELECT 1"},
{"EXPLAIN FORMAT = 'hint' SELECT 1", true, "EXPLAIN FORMAT = 'hint' SELECT 1"},
{"EXPLAIN ANALYZE FORMAT = 'verbose' SELECT 1", true, "EXPLAIN ANALYZE FORMAT = 'verbose' SELECT 1"},
{"EXPLAIN ANALYZE FORMAT = 'binary' SELECT 1", true, "EXPLAIN ANALYZE FORMAT = 'binary' SELECT 1"},
{"EXPLAIN ALTER TABLE t1 ADD INDEX (a)", true, "EXPLAIN FORMAT = 'row' ALTER TABLE `t1` ADD INDEX(`a`)"},
{"EXPLAIN ALTER TABLE t1 ADD a varchar(255)", true, "EXPLAIN FORMAT = 'row' ALTER TABLE `t1` ADD COLUMN `a` VARCHAR(255)"},
}
Expand Down
20 changes: 16 additions & 4 deletions planner/core/access_object.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ func (d DynamicPartitionAccessObjects) SetIntoPB(pb *tipb.ExplainOperator) {
if len(obj.err) > 0 {
continue
}
pbObj := pbObjSlice[i]
pbObj := &pbObjSlice[i]
pbObj.Database = obj.Database
pbObj.Table = obj.Table
pbObj.AllPartitions = obj.AllPartitions
Expand All @@ -119,7 +119,11 @@ func (d DynamicPartitionAccessObjects) SetIntoPB(pb *tipb.ExplainOperator) {
for i := range pbObjSlice {
pbObjs.Objects = append(pbObjs.Objects, &pbObjSlice[i])
}
pb.AccessObject = &tipb.ExplainOperator_DynamicPartitionObjects{DynamicPartitionObjects: &pbObjs}
pb.AccessObjects = []*tipb.AccessObject{
{
AccessObject: &tipb.AccessObject_DynamicPartitionObjects{DynamicPartitionObjects: &pbObjs},
},
}
}

// IndexAccess represents the index accessed by an operator.
Expand Down Expand Up @@ -202,7 +206,11 @@ func (s *ScanAccessObject) SetIntoPB(pb *tipb.ExplainOperator) {
for i := range s.Indexes {
pbObj.Indexes = append(pbObj.Indexes, s.Indexes[i].ToPB())
}
pb.AccessObject = &tipb.ExplainOperator_ScanObject{ScanObject: &pbObj}
pb.AccessObjects = []*tipb.AccessObject{
{
AccessObject: &tipb.AccessObject_ScanObject{ScanObject: &pbObj},
},
}
}

// OtherAccessObject represents other kinds of access.
Expand All @@ -222,7 +230,11 @@ func (o OtherAccessObject) SetIntoPB(pb *tipb.ExplainOperator) {
if pb == nil {
return
}
pb.AccessObject = &tipb.ExplainOperator_OtherObject{OtherObject: string(o)}
pb.AccessObjects = []*tipb.AccessObject{
{
AccessObject: &tipb.AccessObject_OtherObject{OtherObject: string(o)},
},
}
}

// AccessObject implements dataAccesser interface.
Expand Down
133 changes: 133 additions & 0 deletions planner/core/binary_plan_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
// Copyright 2022 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package core_test

import (
"encoding/base64"
"fmt"
"regexp"
"testing"

"github.com/golang/snappy"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
)

func simplifyAndCheckBinaryPlan(t *testing.T, pb *tipb.ExplainData) {
if pb.Main != nil {
simplifyAndCheckBinaryOperator(t, pb.Main, pb.WithRuntimeStats)
}
for _, cte := range pb.Ctes {
if cte != nil {
simplifyAndCheckBinaryOperator(t, cte, pb.WithRuntimeStats)
}
}
}

func simplifyAndCheckBinaryOperator(t *testing.T, pb *tipb.ExplainOperator, withRuntimeStats bool) {
if withRuntimeStats {
if pb.TaskType == tipb.TaskType_root {
require.NotEmpty(t, pb.RootBasicExecInfo)
} else if pb.TaskType != tipb.TaskType_unknown {
require.NotEmpty(t, pb.CopExecInfo)
}
}
pb.RootBasicExecInfo = ""
pb.RootGroupExecInfo = nil
pb.CopExecInfo = ""
match, err := regexp.MatchString("((Table|Index).*Scan)|CTEFullScan|Point_Get", pb.Name)
if err == nil && match {
require.NotNil(t, pb.AccessObjects)
}
// AccessObject field is an interface and json.Unmarshall can't handle it, so we don't check it against the json output.
pb.AccessObjects = nil
// MemoryBytes and DiskBytes are not stable sometimes.
pb.MemoryBytes = 0
pb.DiskBytes = 0
if len(pb.Children) > 0 {
for _, op := range pb.Children {
if op != nil {
simplifyAndCheckBinaryOperator(t, op, withRuntimeStats)
}
}
}
}

func TestBinaryPlanInExplainAndSlowLog(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// If we don't set this, it will be false sometimes and the cost in the result will be different.
tk.MustExec("set @@tidb_enable_chunk_rpc=true")

var input []string
var output []struct {
SQL string
BinaryPlan *tipb.ExplainData
}
planSuiteData := core.GetBinaryPlanSuiteData()
planSuiteData.GetTestCases(t, &input, &output)

for i, test := range input {
comment := fmt.Sprintf("case:%v sql:%s", i, test)
if len(test) < 7 || test[:7] != "explain" {
tk.MustExec(test)
testdata.OnRecord(func() {
output[i].SQL = test
output[i].BinaryPlan = nil
})
continue
}
result := testdata.ConvertRowsToStrings(tk.MustQuery(test).Rows())
require.Equal(t, len(result), 1, comment)
s := result[0]

b, err := base64.StdEncoding.DecodeString(s)
require.NoError(t, err)
b, err = snappy.Decode(nil, b)
require.NoError(t, err)
binary := &tipb.ExplainData{}
err = binary.Unmarshal(b)
require.NoError(t, err)
testdata.OnRecord(func() {
output[i].SQL = test
output[i].BinaryPlan = binary
})
simplifyAndCheckBinaryPlan(t, binary)
require.Equal(t, output[i].BinaryPlan, binary)
}
}

func TestInvalidDecodeBinaryPlan(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

str1 := "some random bytes"
str2 := base64.StdEncoding.EncodeToString([]byte(str1))
str3 := base64.StdEncoding.EncodeToString(snappy.Encode(nil, []byte(str1)))

tk.MustQuery(`select tidb_decode_binary_plan('` + str1 + `')`).Check(testkit.Rows(""))
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 illegal base64 data at input byte 4"))
tk.MustQuery(`select tidb_decode_binary_plan('` + str2 + `')`).Check(testkit.Rows(""))
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 snappy: corrupt input"))
tk.MustQuery(`select tidb_decode_binary_plan('` + str3 + `')`).Check(testkit.Rows(""))
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 proto: illegal wireType 7"))
}
Loading

0 comments on commit 4cade24

Please sign in to comment.