Files
tidb/executor/partition_table_test.go

281 lines
14 KiB
Go

// 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 executor_test
import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/testkit"
)
func (s *partitionTableSuite) TestFourReader(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists pt")
tk.MustExec(`create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) (
partition p0 values less than (4),
partition p1 values less than (7),
partition p2 values less than (10))`)
tk.MustExec("insert into pt values (0, 0), (2, 2), (4, 4), (6, 6), (7, 7), (9, 9), (null, null)")
// Table reader
tk.MustQuery("select * from pt").Sort().Check(testkit.Rows("0 0", "2 2", "4 4", "6 6", "7 7", "9 9", "<nil> <nil>"))
// Table reader: table dual
tk.MustQuery("select * from pt where c > 10").Check(testkit.Rows())
// Table reader: one partition
tk.MustQuery("select * from pt where c > 8").Check(testkit.Rows("9 9"))
// Table reader: more than one partition
tk.MustQuery("select * from pt where c < 2 or c >= 9").Check(testkit.Rows("0 0", "9 9"))
// Index reader
tk.MustQuery("select c from pt").Sort().Check(testkit.Rows("0", "2", "4", "6", "7", "9", "<nil>"))
tk.MustQuery("select c from pt where c > 10").Check(testkit.Rows())
tk.MustQuery("select c from pt where c > 8").Check(testkit.Rows("9"))
tk.MustQuery("select c from pt where c < 2 or c >= 9").Check(testkit.Rows("0", "9"))
// Index lookup
tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt").Sort().Check(testkit.Rows("0 0", "2 2", "4 4", "6 6", "7 7", "9 9", "<nil> <nil>"))
tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10").Check(testkit.Rows())
tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8").Check(testkit.Rows("9 9"))
tk.MustQuery("select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9").Check(testkit.Rows("0 0", "9 9"))
// Index Merge
tk.MustExec("set @@tidb_enable_index_merge = 1")
tk.MustQuery("select /*+ use_index(i_c, i_id) */ * from pt where id = 4 or c < 7").Check(testkit.Rows("0 0", "2 2", "4 4", "6 6"))
}
func (s *partitionTableSuite) TestPartitionIndexJoin(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("set @@session.tidb_enable_table_partition = 1")
tk.MustExec("set @@session.tidb_enable_list_partition = 1")
for i := 0; i < 3; i++ {
tk.MustExec("drop table if exists p, t")
if i == 0 {
// Test for range partition
tk.MustExec(`create table p (id int, c int, key i_id(id), key i_c(c)) partition by range (c) (
partition p0 values less than (4),
partition p1 values less than (7),
partition p2 values less than (10))`)
} else if i == 1 {
// Test for list partition
tk.MustExec(`create table p (id int, c int, key i_id(id), key i_c(c)) partition by list (c) (
partition p0 values in (1,2,3,4),
partition p1 values in (5,6,7),
partition p2 values in (8, 9,10))`)
} else {
// Test for hash partition
tk.MustExec(`create table p (id int, c int, key i_id(id), key i_c(c)) partition by hash(c) partitions 5;`)
}
tk.MustExec("create table t (id int)")
tk.MustExec("insert into p values (3,3), (4,4), (6,6), (9,9)")
tk.MustExec("insert into t values (4), (9)")
// Build indexLookUp in index join
tk.MustQuery("select /*+ INL_JOIN(p) */ * from p, t where p.id = t.id").Sort().Check(testkit.Rows("4 4 4", "9 9 9"))
// Build index reader in index join
tk.MustQuery("select /*+ INL_JOIN(p) */ p.id from p, t where p.id = t.id").Check(testkit.Rows("4", "9"))
}
}
func (s *partitionTableSuite) TestPartitionUnionScanIndexJoin(c *C) {
// For issue https://github.com/pingcap/tidb/issues/19152
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int)) partition by range (c_int) ( partition p0 values less than (10), partition p1 values less than maxvalue)")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int, c_str)) partition by hash (c_int) partitions 4")
tk.MustExec("insert into t1 values (10, 'interesting neumann')")
tk.MustExec("insert into t2 select * from t1")
tk.MustExec("begin")
tk.MustExec("insert into t2 values (11, 'hopeful hoover');")
tk.MustQuery("select /*+ INL_JOIN(t1,t2) */ * from t1 join t2 on t1.c_int = t2.c_int and t1.c_str = t2.c_str where t1.c_int in (10, 11)").Check(testkit.Rows("10 interesting neumann 10 interesting neumann"))
tk.MustQuery("select /*+ INL_HASH_JOIN(t1,t2) */ * from t1 join t2 on t1.c_int = t2.c_int and t1.c_str = t2.c_str where t1.c_int in (10, 11)").Check(testkit.Rows("10 interesting neumann 10 interesting neumann"))
tk.MustExec("commit")
}
func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("use test")
// For issue 19458.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c_int int)")
tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9)")
tk.MustExec("DROP TABLE IF EXISTS `t1`")
tk.MustExec(`CREATE TABLE t1 (
c_int int NOT NULL,
c_str varchar(40) NOT NULL,
c_datetime datetime NOT NULL,
c_timestamp timestamp NULL DEFAULT NULL,
c_double double DEFAULT NULL,
c_decimal decimal(12,6) DEFAULT NULL,
PRIMARY KEY (c_int,c_str,c_datetime)
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci
PARTITION BY RANGE (c_int)
(PARTITION p0 VALUES LESS THAN (2) ENGINE = InnoDB,
PARTITION p1 VALUES LESS THAN (4) ENGINE = InnoDB,
PARTITION p2 VALUES LESS THAN (6) ENGINE = InnoDB,
PARTITION p3 VALUES LESS THAN (8) ENGINE = InnoDB,
PARTITION p4 VALUES LESS THAN (10) ENGINE = InnoDB,
PARTITION p5 VALUES LESS THAN (20) ENGINE = InnoDB,
PARTITION p6 VALUES LESS THAN (50) ENGINE = InnoDB,
PARTITION p7 VALUES LESS THAN (1000000000) ENGINE = InnoDB)`)
tk.MustExec("INSERT INTO `t1` VALUES (19,'nifty feistel','2020-02-28 04:01:28','2020-02-04 06:11:57',32.430079,1.284000),(20,'objective snyder','2020-04-15 17:55:04','2020-05-30 22:04:13',37.690874,9.372000)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values (22, 'wizardly saha', '2020-05-03 16:35:22', '2020-05-03 02:18:42', 96.534810, 0.088)")
tk.MustQuery("select c_int from t where (select min(t1.c_int) from t1 where t1.c_int > t.c_int) > (select count(*) from t1 where t1.c_int > t.c_int) order by c_int").Check(testkit.Rows(
"1", "2", "3", "4", "5", "6", "7", "8", "9"))
tk.MustExec("rollback")
// For issue 19450.
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int))")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int)) partition by hash (c_int) partitions 4")
tk.MustExec("insert into t1 values (1, 'romantic robinson', 4.436), (2, 'stoic chaplygin', 9.826), (3, 'vibrant shamir', 6.300), (4, 'hungry wilson', 4.900), (5, 'naughty swartz', 9.524)")
tk.MustExec("insert into t2 select * from t1")
tk.MustQuery("select * from t1 where c_decimal in (select c_decimal from t2 where t1.c_int = t2.c_int or t1.c_int = t2.c_int and t1.c_str > t2.c_str)").Check(testkit.Rows(
"1 romantic robinson 4.436000",
"2 stoic chaplygin 9.826000",
"3 vibrant shamir 6.300000",
"4 hungry wilson 4.900000",
"5 naughty swartz 9.524000"))
// For issue 19450 release-4.0
tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`)
tk.MustQuery("select * from t1 where c_decimal in (select c_decimal from t2 where t1.c_int = t2.c_int or t1.c_int = t2.c_int and t1.c_str > t2.c_str)").Check(testkit.Rows(
"1 romantic robinson 4.436000",
"2 stoic chaplygin 9.826000",
"3 vibrant shamir 6.300000",
"4 hungry wilson 4.900000",
"5 naughty swartz 9.524000"))
}
func (s *partitionTableSuite) TestImproveCoverage(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("use test")
tk.MustExec(`create table coverage_rr (
pk1 varchar(35) NOT NULL,
pk2 int NOT NULL,
c int,
PRIMARY KEY (pk1,pk2)) partition by hash(pk2) partitions 4;`)
tk.MustExec("create table coverage_dt (pk1 varchar(35), pk2 int)")
tk.MustExec("insert into coverage_rr values ('ios', 3, 2),('android', 4, 7),('linux',5,1)")
tk.MustExec("insert into coverage_dt values ('apple',3),('ios',3),('linux',5)")
tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'")
tk.MustQuery("select /*+ INL_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1"))
tk.MustQuery("select /*+ INL_MERGE_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1"))
}
func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t_info_null")
tk.MustExec(`CREATE TABLE t_info_null (
id bigint(20) unsigned NOT NULL AUTO_INCREMENT,
date date NOT NULL,
media varchar(32) NOT NULL DEFAULT '0',
app varchar(32) NOT NULL DEFAULT '',
xxx bigint(20) NOT NULL DEFAULT '0',
PRIMARY KEY (id, date),
UNIQUE KEY idx_media_id (media, date, app)
) PARTITION BY RANGE COLUMNS(date) (
PARTITION p201912 VALUES LESS THAN ("2020-01-01"),
PARTITION p202001 VALUES LESS THAN ("2020-02-01"),
PARTITION p202002 VALUES LESS THAN ("2020-03-01"),
PARTITION p202003 VALUES LESS THAN ("2020-04-01"),
PARTITION p202004 VALUES LESS THAN ("2020-05-01"),
PARTITION p202005 VALUES LESS THAN ("2020-06-01"),
PARTITION p202006 VALUES LESS THAN ("2020-07-01"),
PARTITION p202007 VALUES LESS THAN ("2020-08-01"),
PARTITION p202008 VALUES LESS THAN ("2020-09-01"),
PARTITION p202009 VALUES LESS THAN ("2020-10-01"),
PARTITION p202010 VALUES LESS THAN ("2020-11-01"),
PARTITION p202011 VALUES LESS THAN ("2020-12-01")
)`)
is := infoschema.GetInfoSchema(tk.Se)
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null"))
c.Assert(err, IsNil)
tbInfo := tbl.Meta()
// Mock for a case that the tableInfo.Partition is not nil, but tableInfo.Partition.Enable is false.
// That may happen when upgrading from a old version TiDB.
tbInfo.Partition.Enable = false
tbInfo.Partition.Num = 0
tk.MustExec("set @@tidb_partition_prune_mode = 'static'")
tk.MustQuery("explain select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows("Batch_Point_Get_5 2.00 root table:t_info_null, index:idx_media_id(media, date, app) keep order:false, desc:false"))
tk.MustQuery("explain select * from t_info_null").Check(testkit.Rows("TableReader_5 10000.00 root data:TableFullScan_4",
"└─TableFullScan_4 10000.00 cop[tikv] table:t_info_null keep order:false, stats:pseudo"))
// No panic.
tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows())
}
func (s *globalIndexSuite) TestGlobalIndexScan(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists p")
tk.MustExec(`create table p (id int, c int) partition by range (c) (
partition p0 values less than (4),
partition p1 values less than (7),
partition p2 values less than (10))`)
tk.MustExec("alter table p add unique idx(id)")
tk.MustExec("insert into p values (1,3), (3,4), (5,6), (7,9)")
tk.MustQuery("select id from p use index (idx)").Check(testkit.Rows("1", "3", "5", "7"))
}
func (s *globalIndexSuite) TestGlobalIndexDoubleRead(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists p")
tk.MustExec(`create table p (id int, c int) partition by range (c) (
partition p0 values less than (4),
partition p1 values less than (7),
partition p2 values less than (10))`)
tk.MustExec("alter table p add unique idx(id)")
tk.MustExec("insert into p values (1,3), (3,4), (5,6), (7,9)")
tk.MustQuery("select * from p use index (idx)").Check(testkit.Rows("1 3", "3 4", "5 6", "7 9"))
}
func (s *globalIndexSuite) TestIssue21731(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists p, t")
tk.MustExec("create table t (a int, b int, unique index idx(a)) partition by list columns(b) (partition p0 values in (1), partition p1 values in (2));")
}
func (s *globalIndexSuite) TestBatchPointGetTablePartition(c *C) {
testKit := testkit.NewTestKitWithInit(c, s.store)
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int, b int, primary key(a,b)) partition by hash(b) partitions 2")
testKit.MustExec("insert into t values(1,1),(1,2),(2,1),(2,2)")
testKit.MustExec("set @@tidb_partition_prune_mode = 'static'")
testKit.MustQuery("select * from t where a in (1,2) and b = 1").Sort().Check(testkit.Rows(
"1 1",
"2 1",
))
testKit.MustQuery("select * from t where a = 1 and b in (1,2)").Sort().Check(testkit.Rows(
"1 1",
"1 2",
))
testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic'")
testKit.MustQuery("select * from t where a in (1,2) and b = 1").Sort().Check(testkit.Rows(
"1 1",
"2 1",
))
testKit.MustQuery("select * from t where a = 1 and b in (1,2)").Sort().Check(testkit.Rows(
"1 1",
"1 2",
))
}