Skip to content

Commit

Permalink
planner: fix a panic during column pruning (#47883) (#48812)
Browse files Browse the repository at this point in the history
close #47331
  • Loading branch information
ti-chi-bot authored Nov 26, 2023
1 parent 7c76d11 commit 5396730
Show file tree
Hide file tree
Showing 5 changed files with 215 additions and 92 deletions.
142 changes: 51 additions & 91 deletions executor/explain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"testing"
"time"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/auth"
plannercore "github.com/pingcap/tidb/planner/core"
Expand Down Expand Up @@ -188,6 +187,57 @@ func checkMemoryInfo(t *testing.T, tk *testkit.TestKit, sql string) {
}
}

func TestIssue47331(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk.MustExec(`create table t1(
id1 varchar(2) DEFAULT '00',
id2 varchar(30) NOT NULL,
id3 datetime DEFAULT NULL,
id4 varchar(100) NOT NULL DEFAULT 'ecifdata',
id5 datetime NOT NULL DEFAULT CURRENT_TIMESTAMP,
id6 int(11) DEFAULT NULL,
id7 int(11) DEFAULT NULL,
UNIQUE KEY UI_id2 (id2),
KEY ix_id1 (id1)
)`)
tk.MustExec("drop table if exists t2")
tk.MustExec(`create table t2(
id10 varchar(40) NOT NULL,
id2 varchar(30) NOT NULL,
KEY IX_id2 (id2),
PRIMARY KEY (id10)
)`)
tk.MustExec("drop table if exists t3")
tk.MustExec(`create table t3(
id20 varchar(40) DEFAULT NULL,
UNIQUE KEY IX_id20 (id20)
)`)
tk.MustExec(`
explain
UPDATE t1 a
SET a.id1 = '04',
a.id3 = CURRENT_TIMESTAMP,
a.id4 = SUBSTRING_INDEX(USER(), '@', 1),
a.id5 = CURRENT_TIMESTAMP
WHERE a.id1 = '03'
AND a.id6 - IFNULL(a.id7, 0) =
(
SELECT COUNT(1)
FROM t2 b, t3 c
WHERE b.id10 = c.id20
AND b.id2 = a.id2
AND b.id2 in (
SELECT rn.id2
FROM t1 rn
WHERE rn.id1 = '03'
)
);
`)
}

func TestMemoryAndDiskUsageAfterClose(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -299,96 +349,6 @@ func checkActRowsNotEmpty(t *testing.T, tk *testkit.TestKit, sql string) {
}
}

func checkActRows(t *testing.T, tk *testkit.TestKit, sql string, expected []string) {
actRowsCol := 2
rows := tk.MustQuery("explain analyze " + sql).Rows()
require.Equal(t, len(expected), len(rows))
for id, row := range rows {
strs := make([]string, len(row))
for i, c := range row {
strs[i] = c.(string)
}

require.Equal(t, expected[id], strs[actRowsCol], fmt.Sprintf("error comparing %s", sql))
}
}

func TestCheckActRowsWithUnistore(t *testing.T) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableCollectExecutionInfo = true
})
store := testkit.CreateMockStore(t)
// testSuite1 use default mockstore which is unistore
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set tidb_cost_model_version=2")
tk.MustExec("drop table if exists t_unistore_act_rows")
tk.MustExec("create table t_unistore_act_rows(a int, b int, index(a, b))")
tk.MustExec("insert into t_unistore_act_rows values (1, 0), (1, 0), (2, 0), (2, 1)")
tk.MustExec("analyze table t_unistore_act_rows")
tk.MustExec("set @@tidb_merge_join_concurrency= 5;")

type testStruct struct {
sql string
expected []string
}

tests := []testStruct{
{
sql: "select * from t_unistore_act_rows",
expected: []string{"4", "4"},
},
{
sql: "select * from t_unistore_act_rows where a > 1",
expected: []string{"2", "2"},
},
{
sql: "select * from t_unistore_act_rows where a > 1 and b > 0",
expected: []string{"1", "1", "2"},
},
{
sql: "select b from t_unistore_act_rows",
expected: []string{"4", "4"},
},
{
sql: "select * from t_unistore_act_rows where b > 0",
expected: []string{"1", "1", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows",
expected: []string{"1", "1", "1", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows group by a",
expected: []string{"2", "2", "2", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows group by b",
expected: []string{"2", "4", "4"},
},
{
sql: "with cte(a) as (select a from t_unistore_act_rows) select (select 1 from cte limit 1) from cte;",
expected: []string{"4", "1", "1", "1", "4", "4", "4", "4", "4"},
},
{
sql: "select a, row_number() over (partition by b) from t_unistore_act_rows;",
expected: []string{"4", "4", "4", "4", "4", "4", "4"},
},
{
sql: "select /*+ merge_join(t1, t2) */ * from t_unistore_act_rows t1 join t_unistore_act_rows t2 on t1.b = t2.b;",
expected: []string{"10", "10", "4", "4", "4", "4", "4", "4", "4", "4", "4", "4"},
},
}

// Default RPC encoding may cause statistics explain result differ and then the test unstable.
tk.MustExec("set @@tidb_enable_chunk_rpc = on")

for _, test := range tests {
checkActRows(t, tk, test.sql, test.expected)
}
}

func TestExplainAnalyzeCTEMemoryAndDiskInfo(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down
15 changes: 15 additions & 0 deletions executor/test/explain/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "explain_test",
srcs = [
"explain_test.go",
"main_test.go",
],
deps = [
"//config",
"//testkit",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
],
)
117 changes: 117 additions & 0 deletions executor/test/explain/explain_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
// Copyright 2023 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 explain

import (
"fmt"
"testing"
"time"

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)

func TestCheckActRowsWithUnistore(t *testing.T) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableCollectExecutionInfo = true
})

store := testkit.CreateMockStore(t)
// testSuite1 use default mockstore which is unistore
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set tidb_cost_model_version=2")
tk.MustExec("drop table if exists t_unistore_act_rows")
tk.MustExec("create table t_unistore_act_rows(a int, b int, index(a, b))")
tk.MustExec("insert into t_unistore_act_rows values (1, 0), (1, 0), (2, 0), (2, 1)")
tk.MustExec("analyze table t_unistore_act_rows")
tk.MustExec("set @@tidb_merge_join_concurrency= 5;")

time.Sleep(time.Second * 5)
type testStruct struct {
sql string
expected []string
}

tests := []testStruct{
{
sql: "select * from t_unistore_act_rows",
expected: []string{"4", "4"},
},
{
sql: "select * from t_unistore_act_rows where a > 1",
expected: []string{"2", "2"},
},
{
sql: "select * from t_unistore_act_rows where a > 1 and b > 0",
expected: []string{"1", "1", "2"},
},
{
sql: "select b from t_unistore_act_rows",
expected: []string{"4", "4"},
},
{
sql: "select * from t_unistore_act_rows where b > 0",
expected: []string{"1", "1", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows",
expected: []string{"1", "1", "1", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows group by a",
expected: []string{"2", "2", "2", "4"},
},
{
sql: "select count(*) from t_unistore_act_rows group by b",
expected: []string{"2", "4", "4"},
},
{
sql: "with cte(a) as (select a from t_unistore_act_rows) select (select 1 from cte limit 1) from cte;",
expected: []string{"4", "1", "1", "1", "4", "4", "4", "4", "4"},
},
{
sql: "select a, row_number() over (partition by b) from t_unistore_act_rows;",
expected: []string{"4", "4", "4", "4", "4", "4", "4"},
},
{
sql: "select /*+ merge_join(t1, t2) */ * from t_unistore_act_rows t1 join t_unistore_act_rows t2 on t1.b = t2.b;",
expected: []string{"10", "10", "4", "4", "4", "4", "4", "4", "4", "4", "4", "4"},
},
}

// Default RPC encoding may cause statistics explain result differ and then the test unstable.
tk.MustExec("set @@tidb_enable_chunk_rpc = on")

for _, test := range tests {
checkActRows(t, tk, test.sql, test.expected)
}
}

func checkActRows(t *testing.T, tk *testkit.TestKit, sql string, expected []string) {
actRowsCol := 2
rows := tk.MustQuery("explain analyze " + sql).Rows()
require.Equal(t, len(expected), len(rows))
for id, row := range rows {
strs := make([]string, len(row))
for i, c := range row {
strs[i] = c.(string)
}

require.Equal(t, expected[id], strs[actRowsCol], fmt.Sprintf("error comparing %s", sql))
}
}
30 changes: 30 additions & 0 deletions executor/test/explain/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
// 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 explain

import (
"testing"

"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"),
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),
goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"),
}
goleak.VerifyTestMain(m, opts...)
}
3 changes: 2 additions & 1 deletion planner/core/rule_join_reorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,8 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP
proj := LogicalProjection{
Exprs: expression.Column2Exprs(originalSchema.Columns),
}.Init(p.SCtx(), p.SelectBlockOffset())
proj.SetSchema(originalSchema)
// Clone the schema here, because the schema may be changed by column pruning rules.
proj.SetSchema(originalSchema.Clone())
proj.SetChildren(p)
p = proj
}
Expand Down

0 comments on commit 5396730

Please sign in to comment.