776 lines
39 KiB
Go
776 lines
39 KiB
Go
// Copyright 2019 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 executor_test
|
|
|
|
import (
|
|
"bytes"
|
|
"crypto/tls"
|
|
"fmt"
|
|
"math"
|
|
"strconv"
|
|
"strings"
|
|
"sync"
|
|
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/parser/auth"
|
|
"github.com/pingcap/tidb/planner/core"
|
|
"github.com/pingcap/tidb/session"
|
|
txninfo "github.com/pingcap/tidb/session/txninfo"
|
|
"github.com/pingcap/tidb/sessionctx/variable"
|
|
"github.com/pingcap/tidb/util"
|
|
"github.com/pingcap/tidb/util/israce"
|
|
"github.com/pingcap/tidb/util/kvcache"
|
|
"github.com/pingcap/tidb/util/testkit"
|
|
)
|
|
|
|
// mockSessionManager is a mocked session manager which is used for test.
|
|
type mockSessionManager1 struct {
|
|
PS []*util.ProcessInfo
|
|
}
|
|
|
|
func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo {
|
|
return nil
|
|
}
|
|
|
|
// ShowProcessList implements the SessionManager.ShowProcessList interface.
|
|
func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo {
|
|
ret := make(map[uint64]*util.ProcessInfo)
|
|
for _, item := range msm.PS {
|
|
ret[item.ID] = item
|
|
}
|
|
return ret
|
|
}
|
|
|
|
func (msm *mockSessionManager1) GetProcessInfo(id uint64) (*util.ProcessInfo, bool) {
|
|
for _, item := range msm.PS {
|
|
if item.ID == id {
|
|
return item, true
|
|
}
|
|
}
|
|
return &util.ProcessInfo{}, false
|
|
}
|
|
|
|
// Kill implements the SessionManager.Kill interface.
|
|
func (msm *mockSessionManager1) Kill(cid uint64, query bool) {
|
|
}
|
|
|
|
func (msm *mockSessionManager1) KillAllConnections() {
|
|
}
|
|
|
|
func (msm *mockSessionManager1) UpdateTLSConfig(cfg *tls.Config) {
|
|
}
|
|
|
|
func (msm *mockSessionManager1) ServerID() uint64 {
|
|
return 1
|
|
}
|
|
|
|
func (s *testSerialSuite) TestExplainFor(c *C) {
|
|
tkRoot := testkit.NewTestKitWithInit(c, s.store)
|
|
tkUser := testkit.NewTestKitWithInit(c, s.store)
|
|
tkRoot.MustExec("drop table if exists t1, t2;")
|
|
tkRoot.MustExec("create table t1(c1 int, c2 int)")
|
|
tkRoot.MustExec("create table t2(c1 int, c2 int)")
|
|
tkRoot.MustExec("create user tu@'%'")
|
|
tkRoot.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890"))
|
|
tkUser.Se.Auth(&auth.UserIdentity{Username: "tu", Hostname: "localhost", CurrentUser: true, AuthUsername: "tu", AuthHostname: "%"}, nil, []byte("012345678901234567890"))
|
|
|
|
tkRoot.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tkRoot.MustQuery("select * from t1;")
|
|
tkRootProcess := tkRoot.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkRootProcess}
|
|
tkRoot.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tkUser.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tkRoot.MustQuery(fmt.Sprintf("explain for connection %d", tkRootProcess.ID)).Check(testkit.Rows(
|
|
"TableReader_5 10000.00 root data:TableFullScan_4",
|
|
"└─TableFullScan_4 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo",
|
|
))
|
|
tkRoot.MustExec("set @@tidb_enable_collect_execution_info=1;")
|
|
check := func() {
|
|
tkRootProcess = tkRoot.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkRootProcess}
|
|
tkRoot.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tkUser.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
rows := tkRoot.MustQuery(fmt.Sprintf("explain for connection %d", tkRootProcess.ID)).Rows()
|
|
c.Assert(len(rows), Equals, 2)
|
|
c.Assert(len(rows[0]), Equals, 9)
|
|
buf := bytes.NewBuffer(nil)
|
|
for i, row := range rows {
|
|
if i > 0 {
|
|
buf.WriteString("\n")
|
|
}
|
|
for j, v := range row {
|
|
if j > 0 {
|
|
buf.WriteString(" ")
|
|
}
|
|
buf.WriteString(fmt.Sprintf("%v", v))
|
|
}
|
|
}
|
|
c.Assert(buf.String(), Matches, ""+
|
|
"TableReader_5 10000.00 0 root time:.*, loops:1, cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+
|
|
"└─TableFullScan_4 10000.00 0 cop.* table:t1 tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A")
|
|
}
|
|
tkRoot.MustQuery("select * from t1;")
|
|
check()
|
|
tkRoot.MustQuery("explain analyze select * from t1;")
|
|
check()
|
|
err := tkUser.ExecToErr(fmt.Sprintf("explain for connection %d", tkRootProcess.ID))
|
|
c.Check(core.ErrAccessDenied.Equal(err), IsTrue)
|
|
err = tkUser.ExecToErr("explain for connection 42")
|
|
c.Check(core.ErrNoSuchThread.Equal(err), IsTrue)
|
|
|
|
tkRootProcess.Plan = nil
|
|
ps = []*util.ProcessInfo{tkRootProcess}
|
|
tkRoot.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tkRoot.MustExec(fmt.Sprintf("explain for connection %d", tkRootProcess.ID))
|
|
}
|
|
|
|
func (s *testSerialSuite) TestExplainForVerbose(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk2 := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustExec("use test;")
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk.MustExec("drop table if exists t1")
|
|
tk.MustExec("create table t1(id int);")
|
|
tk.MustQuery("select * from t1;")
|
|
tkRootProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkRootProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tk2.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
|
|
rs := tk.MustQuery("explain format = 'verbose' select * from t1").Rows()
|
|
rs2 := tk2.MustQuery(fmt.Sprintf("explain format = 'verbose' for connection %d", tkRootProcess.ID)).Rows()
|
|
c.Assert(len(rs), Equals, len(rs2))
|
|
for i := range rs {
|
|
c.Assert(rs[i], DeepEquals, rs2[i])
|
|
}
|
|
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=1;")
|
|
tk.MustExec("drop table if exists t2")
|
|
tk.MustExec("create table t2(id int);")
|
|
tk.MustQuery("select * from t2;")
|
|
tkRootProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkRootProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tk2.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
rs = tk.MustQuery("explain format = 'verbose' select * from t2").Rows()
|
|
rs2 = tk2.MustQuery(fmt.Sprintf("explain format = 'verbose' for connection %d", tkRootProcess.ID)).Rows()
|
|
c.Assert(len(rs), Equals, len(rs2))
|
|
for i := range rs {
|
|
// "id", "estRows", "estCost", "task", "access object", "operator info"
|
|
c.Assert(len(rs[i]), Equals, 6)
|
|
// "id", "estRows", "estCost", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"
|
|
c.Assert(len(rs2[i]), Equals, 10)
|
|
for j := 0; j < 3; j++ {
|
|
c.Assert(rs[i][j], Equals, rs2[i][j])
|
|
}
|
|
}
|
|
}
|
|
|
|
func (s *testSerialSuite) TestIssue11124(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk2 := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk.MustExec("drop table if exists kankan1")
|
|
tk.MustExec("drop table if exists kankan2")
|
|
tk.MustExec("create table kankan1(id int, name text);")
|
|
tk.MustExec("create table kankan2(id int, h1 text);")
|
|
tk.MustExec("insert into kankan1 values(1, 'a'), (2, 'a');")
|
|
tk.MustExec("insert into kankan2 values(2, 'z');")
|
|
tk.MustQuery("select t1.id from kankan1 t1 left join kankan2 t2 on t1.id = t2.id where (case when t1.name='b' then 'case2' when t1.name='a' then 'case1' else NULL end) = 'case1'")
|
|
tkRootProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkRootProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
tk2.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
|
|
rs := tk.MustQuery("explain select t1.id from kankan1 t1 left join kankan2 t2 on t1.id = t2.id where (case when t1.name='b' then 'case2' when t1.name='a' then 'case1' else NULL end) = 'case1'").Rows()
|
|
rs2 := tk2.MustQuery(fmt.Sprintf("explain for connection %d", tkRootProcess.ID)).Rows()
|
|
for i := range rs {
|
|
c.Assert(rs[i], DeepEquals, rs2[i])
|
|
}
|
|
}
|
|
|
|
func (s *testSuite) TestExplainMemTablePredicate(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustQuery("desc select * from METRICS_SCHEMA.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13' ").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:tidb_query_duration PromQL:histogram_quantile(0.9, sum(rate(tidb_server_handle_query_duration_seconds_bucket{}[60s])) by (le,sql_type,instance)), start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s"))
|
|
tk.MustQuery("desc select * from METRICS_SCHEMA.up where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13' ").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:up PromQL:up{}, start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s"))
|
|
tk.MustQuery("desc select * from information_schema.cluster_log where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:CLUSTER_LOG start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13"))
|
|
tk.MustQuery("desc select * from information_schema.cluster_log where level in ('warn','error') and time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:CLUSTER_LOG start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, log_levels:["error","warn"]`))
|
|
tk.MustQuery("desc select * from information_schema.cluster_log where type in ('high_cpu_1','high_memory_1') and time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:CLUSTER_LOG start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, node_types:["high_cpu_1","high_memory_1"]`))
|
|
tk.MustQuery("desc select * from information_schema.slow_query").Check(testkit.Rows(
|
|
"MemTableScan_4 10000.00 root table:SLOW_QUERY only search in the current 'tidb-slow.log' file"))
|
|
tk.MustQuery("desc select * from information_schema.slow_query where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:SLOW_QUERY start_time:2019-12-23 16:10:13.000000, end_time:2019-12-23 16:30:13.000000"))
|
|
tk.MustExec("set @@time_zone = '+00:00';")
|
|
tk.MustQuery("desc select * from information_schema.slow_query where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:SLOW_QUERY start_time:2019-12-23 16:10:13.000000, end_time:2019-12-23 16:30:13.000000"))
|
|
}
|
|
|
|
func (s *testSuite) TestExplainClusterTable(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustQuery("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb","tikv"]`))
|
|
tk.MustQuery("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:CLUSTER_CONFIG instances:["192.168.1.7:2379"]`))
|
|
tk.MustQuery("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb"], instances:["192.168.1.7:2379"]`))
|
|
}
|
|
|
|
func (s *testSuite) TestInspectionResultTable(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustQuery("desc select * from information_schema.inspection_result where rule = 'ddl' and rule = 'config'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RESULT skip_inspection:true`))
|
|
tk.MustQuery("desc select * from information_schema.inspection_result where rule in ('ddl', 'config')").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RESULT rules:["config","ddl"], items:[]`))
|
|
tk.MustQuery("desc select * from information_schema.inspection_result where item in ('ddl.lease', 'raftstore.threadpool')").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RESULT rules:[], items:["ddl.lease","raftstore.threadpool"]`))
|
|
tk.MustQuery("desc select * from information_schema.inspection_result where item in ('ddl.lease', 'raftstore.threadpool') and rule in ('ddl', 'config')").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RESULT rules:["config","ddl"], items:["ddl.lease","raftstore.threadpool"]`))
|
|
}
|
|
|
|
func (s *testSuite) TestInspectionRuleTable(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection"]`))
|
|
tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection","summary"]`))
|
|
tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'").Check(testkit.Rows(
|
|
`MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`))
|
|
}
|
|
|
|
type testPrepareSerialSuite struct {
|
|
*baseTestSuite
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestExplainForConnPlanCache(c *C) {
|
|
if israce.RaceEnabled {
|
|
c.Skip("skip race test")
|
|
}
|
|
orgEnable := core.PreparedPlanCacheEnabled()
|
|
defer func() {
|
|
core.SetPreparedPlanCache(orgEnable)
|
|
}()
|
|
core.SetPreparedPlanCache(true)
|
|
|
|
var err error
|
|
tk1 := testkit.NewTestKit(c, s.store)
|
|
tk1.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
|
|
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
|
|
})
|
|
c.Assert(err, IsNil)
|
|
tk2 := testkit.NewTestKitWithInit(c, s.store)
|
|
|
|
tk1.MustExec("use test")
|
|
tk1.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk1.MustExec("drop table if exists t")
|
|
tk1.MustExec("create table t(a int)")
|
|
tk1.MustExec("prepare stmt from 'select * from t where a = ?'")
|
|
tk1.MustExec("set @p0='1'")
|
|
|
|
executeQuery := "execute stmt using @p0"
|
|
explainQuery := "explain for connection " + strconv.FormatUint(tk1.Se.ShowProcess().ID, 10)
|
|
explainResult := testkit.Rows(
|
|
"TableReader_7 8000.00 root data:Selection_6",
|
|
"└─Selection_6 8000.00 cop[tikv] eq(cast(test.t.a, double BINARY), 1)",
|
|
" └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo",
|
|
)
|
|
|
|
// Now the ProcessInfo held by mockSessionManager1 will not be updated in real time.
|
|
// So it needs to be reset every time before tk2 query.
|
|
// TODO: replace mockSessionManager1 with another mockSessionManager.
|
|
|
|
// single test
|
|
tk1.MustExec(executeQuery)
|
|
tk2.Se.SetSessionManager(&mockSessionManager1{
|
|
PS: []*util.ProcessInfo{tk1.Se.ShowProcess()},
|
|
})
|
|
tk2.MustQuery(explainQuery).Check(explainResult)
|
|
|
|
// multiple test, '1000' is both effective and efficient.
|
|
repeats := 1000
|
|
var wg sync.WaitGroup
|
|
wg.Add(2)
|
|
|
|
go func() {
|
|
for i := 0; i < repeats; i++ {
|
|
tk1.MustExec(executeQuery)
|
|
}
|
|
wg.Done()
|
|
}()
|
|
|
|
go func() {
|
|
for i := 0; i < repeats; i++ {
|
|
tk2.Se.SetSessionManager(&mockSessionManager1{
|
|
PS: []*util.ProcessInfo{tk1.Se.ShowProcess()},
|
|
})
|
|
tk2.MustQuery(explainQuery).Check(explainResult)
|
|
}
|
|
wg.Done()
|
|
}()
|
|
|
|
wg.Wait()
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestSavedPlanPanicPlanCache(c *C) {
|
|
orgEnable := core.PreparedPlanCacheEnabled()
|
|
defer func() {
|
|
core.SetPreparedPlanCache(orgEnable)
|
|
}()
|
|
core.SetPreparedPlanCache(true)
|
|
|
|
var err error
|
|
tk := testkit.NewTestKit(c, s.store)
|
|
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
|
|
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
|
|
})
|
|
c.Assert(err, IsNil)
|
|
|
|
tk.MustExec("use test")
|
|
tk.MustExec("drop table if exists t")
|
|
tk.MustExec("create table t(a int, b int, c int generated always as (a+b) stored)")
|
|
tk.MustExec("insert into t(a,b) values(1,1)")
|
|
tk.MustExec("begin")
|
|
tk.MustExec("update t set b = 2 where a = 1")
|
|
tk.MustExec("prepare stmt from 'select b from t where a > ?'")
|
|
tk.MustExec("set @p = 0")
|
|
tk.MustQuery("execute stmt using @p").Check(testkit.Rows(
|
|
"2",
|
|
))
|
|
tk.MustExec("set @p = 1")
|
|
tk.MustQuery("execute stmt using @p").Check(testkit.Rows())
|
|
err = tk.ExecToErr("insert into t(a,b,c) values(3,3,3)")
|
|
c.Assert(err, NotNil)
|
|
c.Assert(err.Error(), Equals, "[planner:3105]The value specified for generated column 'c' in table 't' is not allowed.")
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestExplainDotForExplainPlan(c *C) {
|
|
tk := testkit.NewTestKit(c, s.store)
|
|
|
|
rows := tk.MustQuery("select connection_id()").Rows()
|
|
c.Assert(len(rows), Equals, 1)
|
|
connID := rows[0][0].(string)
|
|
tk.MustQuery("explain format = 'brief' select 1").Check(testkit.Rows(
|
|
"Projection 1.00 root 1->Column#1",
|
|
"└─TableDual 1.00 root rows:1",
|
|
))
|
|
|
|
tkProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
|
|
tk.MustQuery(fmt.Sprintf("explain format=\"dot\" for connection %s", connID)).Check(nil)
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestExplainDotForQuery(c *C) {
|
|
tk := testkit.NewTestKit(c, s.store)
|
|
tk2 := testkit.NewTestKit(c, s.store)
|
|
|
|
rows := tk.MustQuery("select connection_id()").Rows()
|
|
c.Assert(len(rows), Equals, 1)
|
|
connID := rows[0][0].(string)
|
|
tk.MustQuery("select 1")
|
|
tkProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
|
|
expected := tk2.MustQuery("explain format=\"dot\" select 1").Rows()
|
|
got := tk.MustQuery(fmt.Sprintf("explain format=\"dot\" for connection %s", connID)).Rows()
|
|
for i := range got {
|
|
c.Assert(got[i], DeepEquals, expected[i])
|
|
}
|
|
}
|
|
|
|
func (s *testSuite) TestExplainTableStorage(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))
|
|
tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))
|
|
tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'").Check(testkit.Rows(
|
|
"MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))
|
|
}
|
|
|
|
func (s *testSuite) TestInspectionSummaryTable(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule='ddl'").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root eq(Column#1, "ddl")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where 'ddl'=rule or rule='config'").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root or(eq("ddl", Column#1), eq(Column#1, "config"))`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["config","ddl"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where 'ddl'=rule or rule='config' or rule='slow_query'").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root or(eq("ddl", Column#1), or(eq(Column#1, "config"), eq(Column#1, "slow_query")))`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["config","ddl","slow_query"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where (rule='config' or rule='slow_query') and (metrics_name='metric_name3' or metrics_name='metric_name1')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root or(eq(Column#1, "config"), eq(Column#1, "slow_query")), or(eq(Column#3, "metric_name3"), eq(Column#3, "metric_name1"))`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["config","slow_query"], metric_names:["metric_name1","metric_name3"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule in ('ddl', 'slow_query')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root in(Column#1, "ddl", "slow_query")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl","slow_query"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule in ('ddl', 'slow_query') and metrics_name='metric_name1'").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root eq(Column#3, "metric_name1"), in(Column#1, "ddl", "slow_query")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl","slow_query"], metric_names:["metric_name1"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule in ('ddl', 'slow_query') and metrics_name in ('metric_name1', 'metric_name2')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root in(Column#1, "ddl", "slow_query"), in(Column#3, "metric_name1", "metric_name2")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl","slow_query"], metric_names:["metric_name1","metric_name2"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule='ddl' and metrics_name in ('metric_name1', 'metric_name2')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root eq(Column#1, "ddl"), in(Column#3, "metric_name1", "metric_name2")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl"], metric_names:["metric_name1","metric_name2"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule='ddl' and metrics_name='metric_NAME3'").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root eq(Column#1, "ddl"), eq(Column#3, "metric_NAME3")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["ddl"], metric_names:["metric_name3"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where rule in ('ddl', 'config') and rule in ('slow_query', 'config')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root in(Column#1, "ddl", "config"), in(Column#1, "slow_query", "config")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["config"]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where metrics_name in ('metric_name1', 'metric_name4') and metrics_name in ('metric_name5', 'metric_name4') and rule in ('ddl', 'config') and rule in ('slow_query', 'config') and quantile in (0.80, 0.90)").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root in(Column#1, "ddl", "config"), in(Column#1, "slow_query", "config"), in(Column#3, "metric_name1", "metric_name4"), in(Column#3, "metric_name5", "metric_name4")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY rules:["config"], metric_names:["metric_name4"], quantiles:[0.800000,0.900000]`,
|
|
))
|
|
tk.MustQuery("desc select * from information_schema.inspection_summary where metrics_name in ('metric_name1', 'metric_name4') and metrics_name in ('metric_name5', 'metric_name4') and metrics_name in ('metric_name5', 'metric_name1') and metrics_name in ('metric_name1', 'metric_name3')").Check(testkit.Rows(
|
|
`Selection_5 8000.00 root in(Column#3, "metric_name1", "metric_name3"), in(Column#3, "metric_name1", "metric_name4"), in(Column#3, "metric_name5", "metric_name1"), in(Column#3, "metric_name5", "metric_name4")`,
|
|
`└─MemTableScan_6 10000.00 root table:INSPECTION_SUMMARY skip_inspection: true`,
|
|
))
|
|
}
|
|
|
|
func (s *testSuite) TestExplainTiFlashSystemTables(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tiflashInstance := "192.168.1.7:3930"
|
|
database := "test"
|
|
table := "t"
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_TABLES where TIFLASH_INSTANCE = '%s'", tiflashInstance)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_TABLES tiflash_instances:[\"%s\"]", tiflashInstance)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_SEGMENTS where TIFLASH_INSTANCE = '%s'", tiflashInstance)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_SEGMENTS tiflash_instances:[\"%s\"]", tiflashInstance)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_TABLES where TIDB_DATABASE = '%s'", database)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_TABLES tidb_databases:[\"%s\"]", database)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_SEGMENTS where TIDB_DATABASE = '%s'", database)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_SEGMENTS tidb_databases:[\"%s\"]", database)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_TABLES where TIDB_TABLE = '%s'", table)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_TABLES tidb_tables:[\"%s\"]", table)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_SEGMENTS where TIDB_TABLE = '%s'", table)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_SEGMENTS tidb_tables:[\"%s\"]", table)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_TABLES where TIFLASH_INSTANCE = '%s' and TIDB_DATABASE = '%s' and TIDB_TABLE = '%s'", tiflashInstance, database, table)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_TABLES tiflash_instances:[\"%s\"], tidb_databases:[\"%s\"], tidb_tables:[\"%s\"]", tiflashInstance, database, table)))
|
|
tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TIFLASH_SEGMENTS where TIFLASH_INSTANCE = '%s' and TIDB_DATABASE = '%s' and TIDB_TABLE = '%s'", tiflashInstance, database, table)).Check(testkit.Rows(
|
|
fmt.Sprintf("MemTableScan_5 10000.00 root table:TIFLASH_SEGMENTS tiflash_instances:[\"%s\"], tidb_databases:[\"%s\"], tidb_tables:[\"%s\"]", tiflashInstance, database, table)))
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestPointGetUserVarPlanCache(c *C) {
|
|
orgEnable := core.PreparedPlanCacheEnabled()
|
|
defer func() {
|
|
core.SetPreparedPlanCache(orgEnable)
|
|
}()
|
|
core.SetPreparedPlanCache(true)
|
|
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890"))
|
|
|
|
tk.MustExec("use test")
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
|
|
tk.MustExec("drop table if exists t1")
|
|
tk.MustExec("CREATE TABLE t1 (a BIGINT, b VARCHAR(40), PRIMARY KEY (a, b))")
|
|
tk.MustExec("INSERT INTO t1 VALUES (1,'3'),(2,'4')")
|
|
tk.MustExec("drop table if exists t2")
|
|
tk.MustExec("CREATE TABLE t2 (a BIGINT, b VARCHAR(40), UNIQUE KEY idx_a (a))")
|
|
tk.MustExec("INSERT INTO t2 VALUES (1,'1'),(2,'2')")
|
|
tk.MustExec("prepare stmt from 'select * from t1, t2 where t1.a = t2.a and t2.a = ?'")
|
|
tk.MustExec("set @a=1")
|
|
tk.MustQuery("execute stmt using @a").Check(testkit.Rows(
|
|
"1 3 1 1",
|
|
))
|
|
tkProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
// t2 should use PointGet.
|
|
rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows()
|
|
c.Assert(strings.Contains(fmt.Sprintf("%v", rows[3][0]), "Point_Get"), IsTrue)
|
|
c.Assert(strings.Contains(fmt.Sprintf("%v", rows[3][3]), "table:t2"), IsTrue)
|
|
|
|
tk.MustExec("set @a=2")
|
|
tk.MustQuery("execute stmt using @a").Check(testkit.Rows(
|
|
"2 4 2 2",
|
|
))
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
// t2 should use PointGet, range is changed to [2,2].
|
|
rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows()
|
|
c.Assert(strings.Contains(fmt.Sprintf("%v", rows[3][0]), "Point_Get"), IsTrue)
|
|
c.Assert(strings.Contains(fmt.Sprintf("%v", rows[3][3]), "table:t2"), IsTrue)
|
|
tk.MustQuery("execute stmt using @a").Check(testkit.Rows(
|
|
"2 4 2 2",
|
|
))
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestExpressionIndexPreparePlanCache(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
|
|
orgEnable := core.PreparedPlanCacheEnabled()
|
|
defer func() {
|
|
core.SetPreparedPlanCache(orgEnable)
|
|
}()
|
|
core.SetPreparedPlanCache(true)
|
|
|
|
var err error
|
|
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
|
|
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
|
|
})
|
|
c.Assert(err, IsNil)
|
|
|
|
tk.MustExec("use test")
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk.MustExec("drop table if exists t")
|
|
tk.MustExec("create table t(a int, b int, key ((a+b)));")
|
|
tk.MustExec("prepare stmt from 'select * from t where a+b = ?'")
|
|
tk.MustExec("set @a = 123")
|
|
tk.MustExec("execute stmt using @a")
|
|
|
|
tkProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 5)
|
|
c.Assert(res.Rows()[3][3], Matches, ".*expression_index.*")
|
|
c.Assert(res.Rows()[3][4], Matches, ".*[123,123].*")
|
|
|
|
tk.MustExec("set @a = 1234")
|
|
tk.MustExec("execute stmt using @a")
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 5)
|
|
c.Assert(res.Rows()[3][3], Matches, ".*expression_index.*")
|
|
c.Assert(res.Rows()[3][4], Matches, ".*[1234,1234].*")
|
|
}
|
|
|
|
func (s *testPrepareSerialSuite) TestIssue28259(c *C) {
|
|
tk := testkit.NewTestKitWithInit(c, s.store)
|
|
|
|
orgEnable := core.PreparedPlanCacheEnabled()
|
|
defer func() {
|
|
core.SetPreparedPlanCache(orgEnable)
|
|
}()
|
|
core.SetPreparedPlanCache(true)
|
|
|
|
var err error
|
|
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
|
|
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
|
|
})
|
|
c.Assert(err, IsNil)
|
|
|
|
// test for indexRange
|
|
tk.MustExec("use test")
|
|
tk.MustExec("set @@tidb_enable_collect_execution_info=0;")
|
|
tk.MustExec("drop table if exists UK_GCOL_VIRTUAL_18588;")
|
|
tk.MustExec("CREATE TABLE `UK_GCOL_VIRTUAL_18588` (`COL1` bigint(20), UNIQUE KEY `UK_COL1` (`COL1`)" +
|
|
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;")
|
|
tk.MustExec("insert into UK_GCOL_VIRTUAL_18588 values('8502658334322817163');")
|
|
tk.MustExec(`prepare stmt from 'select col1 from UK_GCOL_VIRTUAL_18588 where col1 between ? and ? or col1 < ?';`)
|
|
tk.MustExec("set @a=5516958330762833919, @b=8551969118506051323, @c=2887622822023883594;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows("8502658334322817163"))
|
|
|
|
tkProcess := tk.Se.ShowProcess()
|
|
ps := []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 3)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
tk.MustExec("set @a=-1696020282760139948, @b=-2619168038882941276, @c=-4004648990067362699;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 3)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*IndexFullScan.*")
|
|
|
|
res = tk.MustQuery("explain format = 'brief' select col1 from UK_GCOL_VIRTUAL_18588 use index(UK_COL1) " +
|
|
"where col1 between -1696020282760139948 and -2619168038882941276 or col1 < -4004648990067362699;")
|
|
c.Assert(len(res.Rows()), Equals, 3)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*IndexFullScan.*")
|
|
res = tk.MustQuery("explain format = 'brief' select col1 from UK_GCOL_VIRTUAL_18588 use index(UK_COL1) " +
|
|
"where col1 between 5516958330762833919 and 8551969118506051323 or col1 < 2887622822023883594;")
|
|
c.Assert(len(res.Rows()), Equals, 2)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
tk.MustExec("drop table if exists t;")
|
|
tk.MustExec("CREATE TABLE t (a int, b int, index idx(a, b));")
|
|
tk.MustExec("insert into t values(1, 0);")
|
|
tk.MustExec(`prepare stmt from 'select a from t where (a between ? and ? or a < ?) and b < 1;'`)
|
|
tk.MustExec("set @a=0, @b=2, @c=2;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows("1"))
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1), or(and(ge(test.t.a, 0), le(test.t.a, 2)), lt(test.t.a, 2))")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
tk.MustExec("set @a=2, @b=1, @c=1;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1), or(and(ge(test.t.a, 2), le(test.t.a, 1)), lt(test.t.a, 1))")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexFullScan.*")
|
|
|
|
res = tk.MustQuery("explain format = 'brief' select a from t use index(idx) " +
|
|
"where (a between 0 and 2 or a < 2) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1)")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexRangeScan.*")
|
|
res = tk.MustQuery("explain format = 'brief' select a from t use index(idx) " +
|
|
"where (a between 2 and 1 or a < 1) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1)")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
// test for indexLookUp
|
|
tk.MustExec("drop table if exists t;")
|
|
tk.MustExec("CREATE TABLE t (a int, b int, index idx(a));")
|
|
tk.MustExec("insert into t values(1, 0);")
|
|
tk.MustExec(`prepare stmt from 'select a from t where (a between ? and ? or a < ?) and b < 1;'`)
|
|
tk.MustExec("set @a=0, @b=2, @c=2;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows("1"))
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 6)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexLookUp.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
tk.MustExec("set @a=2, @b=1, @c=1;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 6)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexLookUp.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*IndexFullScan.*")
|
|
|
|
res = tk.MustQuery("explain format = 'brief' select a from t use index(idx) " +
|
|
"where (a between 0 and 2 or a < 2) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 5)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexLookUp.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*IndexRangeScan.*")
|
|
res = tk.MustQuery("explain format = 'brief' select a from t use index(idx) " +
|
|
"where (a between 2 and 1 or a < 1) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 5)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*IndexLookUp.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*IndexRangeScan.*")
|
|
|
|
// test for tableReader
|
|
tk.MustExec("drop table if exists t;")
|
|
tk.MustExec("CREATE TABLE t (a int PRIMARY KEY CLUSTERED, b int);")
|
|
tk.MustExec("insert into t values(1, 0);")
|
|
tk.MustExec(`prepare stmt from 'select a from t where (a between ? and ? or a < ?) and b < 1;'`)
|
|
tk.MustExec("set @a=0, @b=2, @c=2;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows("1"))
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*TableReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1), or(and(ge(test.t.a, 0), le(test.t.a, 2)), lt(test.t.a, 2))")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*TableRangeScan.*")
|
|
|
|
tk.MustExec("set @a=2, @b=1, @c=1;")
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
|
|
tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows())
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*TableReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1), or(and(ge(test.t.a, 2), le(test.t.a, 1)), lt(test.t.a, 1))")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*TableRangeScan.*")
|
|
|
|
res = tk.MustQuery("explain format = 'brief' select a from t " +
|
|
"where (a between 0 and 2 or a < 2) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*TableReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1)")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*TableRangeScan.*")
|
|
res = tk.MustQuery("explain format = 'brief' select a from t " +
|
|
"where (a between 2 and 1 or a < 1) and b < 1;")
|
|
c.Assert(len(res.Rows()), Equals, 4)
|
|
c.Assert(res.Rows()[1][0], Matches, ".*TableReader.*")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*Selection.*")
|
|
c.Assert(res.Rows()[2][4], Equals, "lt(test.t.b, 1)")
|
|
c.Assert(res.Rows()[3][0], Matches, ".*TableRangeScan.*")
|
|
|
|
tk.MustExec("drop table if exists t;")
|
|
tk.MustExec("CREATE TABLE t (a int primary key, b int, c int, d int);")
|
|
tk.MustExec(`prepare stmt from 'select * from t where ((a > ? and a < 5 and b > 2) or (a > ? and a < 10 and c > 3)) and d = 5;';`)
|
|
tk.MustExec("set @a=1, @b=8;")
|
|
tk.MustQuery("execute stmt using @a,@b;").Check(testkit.Rows())
|
|
tkProcess = tk.Se.ShowProcess()
|
|
ps = []*util.ProcessInfo{tkProcess}
|
|
tk.Se.SetSessionManager(&mockSessionManager1{PS: ps})
|
|
res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10))
|
|
c.Assert(len(res.Rows()), Equals, 3)
|
|
c.Assert(res.Rows()[0][0], Matches, ".*TableReader.*")
|
|
c.Assert(res.Rows()[1][0], Matches, ".*Selection.*")
|
|
// The duplicate expressions can not be eliminated because the conditions are not the same.
|
|
// So this may be a bad case in some situations.
|
|
c.Assert(res.Rows()[1][4], Equals, "eq(test.t.d, 5), or(and(gt(test.t.a, 1), and(lt(test.t.a, 5), gt(test.t.b, 2))), and(gt(test.t.a, 8), and(lt(test.t.a, 10), gt(test.t.c, 3)))), or(and(gt(test.t.a, 1), lt(test.t.a, 5)), and(gt(test.t.a, 8), lt(test.t.a, 10)))")
|
|
c.Assert(res.Rows()[2][0], Matches, ".*TableRangeScan.*")
|
|
}
|