From 2d1a805d81de08c954f245ba31651ba57172478d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 16 Jun 2021 23:00:37 +0800 Subject: [PATCH] cherry pick #21148 to release-4.0 Signed-off-by: ti-srebot --- executor/builder.go | 137 ++ executor/distsql.go | 16 + executor/executor.go | 44 +- executor/executor_test.go | 157 ++ executor/partition_table_test.go | 2688 ++++++++++++++++++++++ executor/table_reader.go | 36 + planner/core/exhaust_physical_plans.go | 1 + planner/core/logical_plan_builder.go | 60 + planner/core/logical_plans.go | 13 + planner/core/physical_plans.go | 3 + planner/core/planbuilder.go | 39 +- planner/core/rule_column_pruning.go | 5 +- planner/core/rule_partition_processor.go | 468 +++- 13 files changed, 3639 insertions(+), 28 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 101f9245eb856..7c5af63606b29 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -564,6 +564,16 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor tblID2Handle: v.TblID2Handle, partitionedTable: v.PartitionedTable, } + if len(e.partitionedTable) > 0 { + schema := v.Schema() + e.tblID2PIDColumnIndex = make(map[int64]int) + for i := 0; i < len(v.ExtraPIDInfo.Columns); i++ { + col := v.ExtraPIDInfo.Columns[i] + tblID := v.ExtraPIDInfo.TblIDs[i] + offset := schema.ColumnIndex(col) + e.tblID2PIDColumnIndex[tblID] = offset + } + } return e } @@ -2342,6 +2352,9 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea storeType: v.StoreType, batchCop: v.BatchCop, } + if tbl.Meta().Partition != nil { + e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) + } e.buildVirtualColumnInfo() if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) @@ -2368,6 +2381,33 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea return e, nil } +<<<<<<< HEAD +======= +func extraPIDColumnIndex(schema *expression.Schema) offsetOptional { + for idx, col := range schema.Columns { + if col.ID == model.ExtraPidColID { + return newOffset(idx) + } + } + return 0 +} + +func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Executor { + startTs, err := b.getSnapshotTS() + if err != nil { + b.err = err + return nil + } + gather := &MPPGather{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + is: b.is, + originalPlan: v.GetTablePlan(), + startTS: startTs, + } + return gather +} + +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) // buildTableReader builds a table reader executor. It first build a no range table reader, // and then update it ranges from table scan plan. func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) *TableReaderExecutor { @@ -2381,6 +2421,42 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) +<<<<<<< HEAD +======= + + if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + return ret + } + // When isPartition is set, it means the union rewriting is done, so a partition reader is prefered. + if ok, _ := ts.IsPartition(); ok { + return ret + } + + pi := ts.Table.GetPartitionInfo() + if pi == nil { + return ret + } + + tmp, _ := b.is.TableByID(ts.Table.ID) + tbl := tmp.(table.PartitionedTable) + partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames) + if err != nil { + b.err = err + return nil + } + if v.StoreType == kv.TiFlash { + sctx.IsTiFlash.Store(true) + } + + if len(partitions) == 0 { + return &TableDualExec{baseExecutor: *ret.base()} + } + ret.kvRangeBuilder = kvRangeBuilderFromRangeAndPartition{ + sctx: b.ctx, + partitions: partitions, + } + +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) return ret } @@ -2449,6 +2525,35 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) * ret.ranges = is.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexNames = append(sctx.IndexNames, is.Table.Name.O+":"+is.Index.Name.O) +<<<<<<< HEAD +======= + + if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + return ret + } + // When isPartition is set, it means the union rewriting is done, so a partition reader is prefered. + if ok, _ := is.IsPartition(); ok { + return ret + } + + pi := is.Table.GetPartitionInfo() + if pi == nil { + return ret + } + + if is.Index.Global { + return ret + } + + tmp, _ := b.is.TableByID(is.Table.ID) + tbl := tmp.(table.PartitionedTable) + partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames) + if err != nil { + b.err = err + return nil + } + ret.partitions = partitions +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) return ret } @@ -2516,6 +2621,9 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn tblPlans: v.TablePlans, PushedLimit: v.PushedLimit, } + if ok, _ := ts.IsPartition(); ok { + e.extraPIDColumnIndex = extraPIDColumnIndex(v.Schema()) + } if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) @@ -2552,6 +2660,35 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexNames = append(sctx.IndexNames, is.Table.Name.O+":"+is.Index.Name.O) sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) +<<<<<<< HEAD +======= + + if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + return ret + } + + if pi := is.Table.GetPartitionInfo(); pi == nil { + return ret + } + + if is.Index.Global { + return ret + } + if ok, _ := is.IsPartition(); ok { + // Already pruned when translated to logical union. + return ret + } + + tmp, _ := b.is.TableByID(is.Table.ID) + tbl := tmp.(table.PartitionedTable) + partitions, err := partitionPruning(b.ctx, tbl, v.PartitionInfo.PruningConds, v.PartitionInfo.PartitionNames, v.PartitionInfo.Columns, v.PartitionInfo.ColumnNames) + if err != nil { + b.err = err + return nil + } + ret.partitionTableMode = true + ret.prunedPartitions = partitions +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) return ret } diff --git a/executor/distsql.go b/executor/distsql.go index af8ede2792127..5b2471edd3831 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -364,6 +364,9 @@ type IndexLookUpExecutor struct { PushedLimit *plannercore.PushedDownLimit stats *IndexLookUpRunTimeStats + + // extraPIDColumnIndex is used for partition reader to add an extra partition ID column, default -1 + extraPIDColumnIndex offsetOptional } type checkIndexValue struct { @@ -533,6 +536,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { tableReaderExec := &TableReaderExecutor{ +<<<<<<< HEAD baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()), table: e.table, dagPB: e.tableRequest, @@ -542,6 +546,18 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []in feedback: statistics.NewQueryFeedback(0, nil, 0, false), corColInFilter: e.corColInTblSide, plans: e.tblPlans, +======= + baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()), + table: table, + dagPB: e.tableRequest, + startTS: e.startTS, + columns: e.columns, + streaming: e.tableStreaming, + feedback: statistics.NewQueryFeedback(0, nil, 0, false), + corColInFilter: e.corColInTblSide, + plans: e.tblPlans, + extraPIDColumnIndex: e.extraPIDColumnIndex, +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) } tableReaderExec.buildVirtualColumnInfo() tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles, true) diff --git a/executor/executor.go b/executor/executor.go index 818c7099634ba..0f431e43a8780 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -866,32 +866,31 @@ type SelectLockExec struct { Lock ast.SelectLockType keys []kv.Key +<<<<<<< HEAD tblID2Handle map[int64][]*expression.Column partitionedTable []table.PartitionedTable // tblID2Table is cached to reduce cost. tblID2Table map[int64]table.PartitionedTable inited bool +======= + tblID2Handle map[int64][]plannercore.HandleCols + + // All the partition tables in the children of this executor. + partitionedTable []table.PartitionedTable + + // When SelectLock work on the partition table, we need the partition ID + // instead of table ID to calculate the lock KV. In that case, partition ID is store as an + // extra column in the chunk row. + // tblID2PIDColumnIndex stores the column index in the chunk row. The children may be join + // of multiple tables, so the map struct is used. + tblID2PIDColumnIndex map[int64]int +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) } // Open implements the Executor Open interface. func (e *SelectLockExec) Open(ctx context.Context) error { - if err := e.baseExecutor.Open(ctx); err != nil { - return err - } - - if len(e.tblID2Handle) > 0 && len(e.partitionedTable) > 0 { - e.tblID2Table = make(map[int64]table.PartitionedTable, len(e.partitionedTable)) - for id := range e.tblID2Handle { - for _, p := range e.partitionedTable { - if id == p.Meta().ID { - e.tblID2Table[id] = p - } - } - } - } - - return nil + return e.baseExecutor.Open(ctx) } // Next implements the Executor Next interface. @@ -909,15 +908,14 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { if req.NumRows() > 0 { iter := chunk.NewIterator4Chunk(req) for row := iter.Begin(); row != iter.End(); row = iter.Next() { + for id, cols := range e.tblID2Handle { physicalID := id - if pt, ok := e.tblID2Table[id]; ok { - // On a partitioned table, we have to use physical ID to encode the lock key! - p, err := pt.GetPartitionByRow(e.ctx, row.GetDatumRow(e.base().retFieldTypes)) - if err != nil { - return err - } - physicalID = p.GetPhysicalID() + if len(e.partitionedTable) > 0 { + // Replace the table ID with partition ID. + // The partition ID is returned as an extra column from the table reader. + offset := e.tblID2PIDColumnIndex[id] + physicalID = row.GetInt64(offset) } for _, col := range cols { diff --git a/executor/executor_test.go b/executor/executor_test.go index f27cfaee5fd75..90393da8c8bc0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -6499,6 +6499,163 @@ func (s *testSuite) TestIssue19667(c *C) { tk.MustQuery(`SELECT DATE_ADD(a, INTERVAL 1 SECOND) FROM t`).Check(testkit.Rows("1988-04-17 02:00:00")) } +<<<<<<< HEAD +======= +func issue20975Prepare(c *C, store kv.Storage) (*testkit.TestKit, *testkit.TestKit) { + tk1 := testkit.NewTestKit(c, store) + tk2 := testkit.NewTestKit(c, store) + tk1.MustExec("use test") + tk1.MustExec("drop table if exists t1, t2") + tk2.MustExec("use test") + tk1.MustExec("create table t1(id int primary key, c int)") + tk1.MustExec("insert into t1 values(1, 10), (2, 20)") + return tk1, tk2 +} + +func (s *testSuite) TestIssue20975UpdateNoChange(c *C) { + tk1, tk2 := issue20975Prepare(c, s.store) + tk1.MustExec("begin pessimistic") + tk1.MustExec("update t1 set c=c") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdate(c *C) { + tk1, tk2 := issue20975Prepare(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdatePointGet(c *C) { + tk1, tk2 := issue20975Prepare(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id=1 for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id=1 for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdateBatchPointGet(c *C) { + tk1, tk2 := issue20975Prepare(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id in (1, 2) for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id in (1, 2) for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +func issue20975PreparePartitionTable(c *C, store kv.Storage) (*testkit.TestKit, *testkit.TestKit) { + tk1 := testkit.NewTestKit(c, store) + tk2 := testkit.NewTestKit(c, store) + tk1.MustExec("use test") + tk1.MustExec("drop table if exists t1, t2") + tk2.MustExec("use test") + tk1.MustExec(`create table t1(id int primary key, c int) partition by range (id) ( + partition p1 values less than (10), + partition p2 values less than (20) + )`) + tk1.MustExec("insert into t1 values(1, 10), (2, 20), (11, 30), (12, 40)") + return tk1, tk2 +} + +func (s *testSuite) TestIssue20975UpdateNoChangeWithPartitionTable(c *C) { + tk1, tk2 := issue20975PreparePartitionTable(c, s.store) + + // Set projection concurrency to avoid data race here. + // TODO: remove this line after fixing https://github.com/pingcap/tidb/issues/25496 + tk1.Se.GetSessionVars().Concurrency.SetProjectionConcurrency(0) + + tk1.MustExec("begin pessimistic") + tk1.MustExec("update t1 set c=c") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdateWithPartitionTable(c *C) { + tk1, tk2 := issue20975PreparePartitionTable(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdatePointGetWithPartitionTable(c *C) { + tk1, tk2 := issue20975PreparePartitionTable(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id=1 for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id=12 for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id=1 for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id=12 for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +func (s *testSuite) TestIssue20975SelectForUpdateBatchPointGetWithPartitionTable(c *C) { + tk1, tk2 := issue20975PreparePartitionTable(c, s.store) + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id in (1, 2) for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id in (11, 12) for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") + + tk1.MustExec("begin") + tk1.MustExec("select * from t1 where id in (1, 11) for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id in (1, 2) for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id in (11, 12) for update") + tk2.MustExec("create table t2(a int)") + tk1.MustExec("commit") + + tk1.MustExec("begin pessimistic") + tk1.MustExec("select * from t1 where id in (1, 11) for update") + tk2.MustExec("drop table t2") + tk1.MustExec("commit") +} + +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) func (s *testSuite) TestIssue20305(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index e834776216a49..c3fe6ae5f7ee3 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -14,6 +14,14 @@ package executor_test import ( +<<<<<<< HEAD +======= + "fmt" + "math/rand" + "strings" + "time" + +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" @@ -107,3 +115,2683 @@ func (s *testSuite9) TestPartitionInfoDisable(c *C) { // 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()) } +<<<<<<< HEAD +======= + +func (s *partitionTableSuite) TestOrderByandLimit(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_orderby_limit") + tk.MustExec("use test_orderby_limit") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec("create table thash(a int, b int, index idx_a(a), index idx_b(b)) partition by hash(a) partitions 4;") + + // regular table + tk.MustExec("create table tregular(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test indexLookUp + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange use index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexLookUp"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test tableReader + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "TableReader"), IsTrue) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexReader + for i := 0; i < 100; i++ { + // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select a from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexReader"), IsTrue) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexMerge + for i := 0; i < 100; i++ { + // explain select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // check if IndexMerge is used + // select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // can return the correct value + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > 2 or b < 5 order by a, b limit %v;", y) + queryRegular := fmt.Sprintf("select * from tregular where a > 2 or b < 5 order by a, b limit %v;", y) + c.Assert(tk.HasPlan(queryPartition, "IndexMerge"), IsTrue) // check if indexMerge is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_batchget_pointget") + tk.MustExec("use test_batchget_pointget") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash partition table + tk.MustExec("create table thash(a int, unique key(a)) partition by hash(a) partitions 4;") + + // regular partition table + tk.MustExec("create table tregular(a int, unique key(a));") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test PointGet + for i := 0; i < 100; i++ { + // explain select a from t where a = {x}; // x >= 1 and x <= 100 Check if PointGet is used + // select a from t where a={x}; // the result is {x} + x := rand.Intn(100) + 1 + queryHash := fmt.Sprintf("select a from thash where a=%v", x) + queryRegular := fmt.Sprintf("select a from thash where a=%v", x) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(tk.MustQuery(queryRegular).Rows()) + } + + // test empty PointGet + queryHash := "select a from thash where a=200" + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(testkit.Rows()) + + // test BatchGet + for i := 0; i < 100; i++ { + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + + queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) + queryRegular := fmt.Sprintf("select a from tregular where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestView(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_view") + tk.MustExec("use test_view") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a varchar(10), b varchar(10), key(a)) partition by range columns(a) ( + partition p0 values less than ('300'), + partition p1 values less than ('600'), + partition p2 values less than ('900'), + partition p3 values less than ('9999'))`) + tk.MustExec(`create table t1 (a int, b int, key(a))`) + tk.MustExec(`create table t2 (a varchar(10), b varchar(10), key(a))`) + + // insert the same data into thash and t1 + vals := make([]string, 0, 3000) + for i := 0; i < 3000; i++ { + vals = append(vals, fmt.Sprintf(`(%v, %v)`, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(fmt.Sprintf(`insert into thash values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t1 values %v`, strings.Join(vals, ", "))) + + // insert the same data into trange and t2 + vals = vals[:0] + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf(`("%v", "%v")`, rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec(fmt.Sprintf(`insert into trange values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t2 values %v`, strings.Join(vals, ", "))) + + // test views on a single table + tk.MustExec(`create definer='root'@'localhost' view vhash as select a*2 as a, a+b as b from thash`) + tk.MustExec(`create definer='root'@'localhost' view v1 as select a*2 as a, a+b as b from t1`) + tk.MustExec(`create definer='root'@'localhost' view vrange as select concat(a, b) as a, a+b as b from trange`) + tk.MustExec(`create definer='root'@'localhost' view v2 as select concat(a, b) as a, a+b as b from t2`) + for i := 0; i < 100; i++ { + xhash := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where b>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where b>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v and b>=%v`, xhash, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v and b>=%v`, xhash, xhash)).Sort().Rows()) + + xrange := fmt.Sprintf(`"%v"`, rand.Intn(1000)) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where b>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where b>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v and b<=%v`, xrange, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v and b<=%v`, xrange, xrange)).Sort().Rows()) + } + + // test views on both tables + tk.MustExec(`create definer='root'@'localhost' view vboth as select thash.a+trange.a as a, thash.b+trange.b as b from thash, trange where thash.a=trange.a`) + tk.MustExec(`create definer='root'@'localhost' view vt as select t1.a+t2.a as a, t1.b+t2.b as b from t1, t2 where t1.a=t2.a`) + for i := 0; i < 100; i++ { + x := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where b>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where b>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v and b>=%v`, x, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v and b>=%v`, x, x)).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dr_join") + tk.MustExec("use test_dr_join") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range partition + tk.MustExec("create table thash (a int, b int, c int, primary key(a), index idx_b(b)) partition by hash(a) partitions 4;") + tk.MustExec(`create table trange (a int, b int, c int, primary key(a), index idx_b(b)) partition by range(a) ( +   partition p0 values less than(1000), +   partition p1 values less than(2000), +   partition p2 values less than(3000), +   partition p3 values less than(4000));`) + + // regualr table + tk.MustExec(`create table tnormal (a int, b int, c int, primary key(a), index idx_b(b));`) + tk.MustExec(`create table touter (a int, b int, c int);`) + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v, %v)", rand.Intn(4000), rand.Intn(4000), rand.Intn(4000))) + } + tk.MustExec("insert ignore into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tnormal values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into touter values " + strings.Join(vals, ",")) + + // test indexLookUp + hash + queryPartition := "select /*+ INL_JOIN(touter, thash) */ * from touter join thash use index(idx_b) on touter.b = thash.b" + queryRegular := "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root partition:all ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test_dr_join.thash.b))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(test_dr_join.thash.b, test_dr_join.touter.b)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:thash keep order:false, stats:pseudo")) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test tableReader + hash + queryPartition = "select /*+ INL_JOIN(touter, thash) */ * from touter join thash on touter.a = thash.a" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.thash.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.thash.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 1.00 cop[tikv] table:thash range: decided by [test_dr_join.touter.a], keep order:false, stats:pseudo")) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexReader + hash + queryPartition = "select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexReader(Probe) 1.25 root partition:all index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test_dr_join.thash.b))", + " └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(test_dr_join.thash.b, test_dr_join.touter.b)], keep order:false, stats:pseudo")) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexLookUp + range + // explain select /*+ INL_JOIN(touter, tinner) */ * from touter join tinner use index(a) on touter.a = tinner.a; + queryPartition = "select /*+ INL_JOIN(touter, trange) */ * from touter join trange use index(idx_b) on touter.b = trange.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root partition:all ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test_dr_join.trange.b))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:trange, index:idx_b(b) range: decided by [eq(test_dr_join.trange.b, test_dr_join.touter.b)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:trange keep order:false, stats:pseudo")) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test tableReader + range + queryPartition = "select /*+ INL_JOIN(touter, trange) */ * from touter join trange on touter.a = trange.a;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a;" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.trange.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.trange.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 1.00 cop[tikv] table:trange range: decided by [test_dr_join.touter.a], keep order:false, stats:pseudo")) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexReader + range + // explain select /*+ INL_JOIN(touter, tinner) */ tinner.a from touter join tinner on touter.a = tinner.a; + queryPartition = "select /*+ INL_JOIN(touter, trange) */ trange.b from touter join trange use index(idx_b) on touter.b = trange.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexReader(Probe) 1.25 root partition:all index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test_dr_join.trange.b))", + " └─IndexRangeScan 1.25 cop[tikv] table:trange, index:idx_b(b) range: decided by [eq(test_dr_join.trange.b, test_dr_join.touter.b)], keep order:false, stats:pseudo")) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + +func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("create database pruing_under_index_join") + tk.MustExec("use pruing_under_index_join") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int, c int, primary key(a), index idx_b(b))`) + tk.MustExec(`create table thash (a int, b int, c int, primary key(a), index idx_b(b)) partition by hash(a) partitions 4`) + tk.MustExec(`create table touter (a int, b int, c int)`) + + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v, %v)", i, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into touter values ` + strings.Join(vals, ", ")) + + // case 1: IndexReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexReader(Probe) 1.25 root partition:all index:Selection`, + ` └─Selection 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) + + // case 2: TableReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:TableReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.a, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.a)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan`, + ` └─TableRangeScan 1.00 cop[tikv] table:thash range: decided by [pruing_under_index_join.touter.b], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(primary) on touter.b = tnormal.a`).Sort().Rows()) + + // case 3: IndexLookUp in the inner side + read all inner columns + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexLookUp(Probe) 1.25 root partition:all `, + ` ├─Selection(Build) 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` │ └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`, + ` └─TableRowIDScan(Probe) 1.25 cop[tikv] table:thash keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) +} + +func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_pointget") + tk.MustExec("use test_pointget") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + + // hash partition table + tk.MustExec("create table thash(a int unsigned, unique key(a)) partition by hash(a) partitions 4;") + + // insert data into list partition table + tk.MustExec("insert into tlist values(1,1), (2,2), (3, 3), (4, 4), (5,5), (6, 6), (7,7), (8, 8), (9, 9), (10, 10), (11, 11), (12, 12);") + // regular partition table + tk.MustExec("create table tregular1(a int, unique key(a));") + tk.MustExec("create table tregular2(a int, unique key(a));") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") + + // test BatchGet + for i := 0; i < 100; i++ { + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular1 := fmt.Sprintf("select a from tregular1 where a in (%v)", strings.Join(points, ",")) + + queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryHash, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryRange := fmt.Sprintf("select a from trange where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + points = make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(12) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular2 := fmt.Sprintf("select a from tregular2 where a in (%v)", strings.Join(points, ",")) + queryList := fmt.Sprintf("select a from tlist where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryList, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryList).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test different data type + // unsigned flag + // partition table and reguar table pair + tk.MustExec(`create table trange3(a int unsigned, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + tk.MustExec("create table tregular3(a int unsigned, unique key(a));") + vals = make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into trange3 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular3 values " + strings.Join(vals, ",")) + // test BatchGet + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular := fmt.Sprintf("select a from tregular3 where a in (%v)", strings.Join(points, ",")) + queryRange := fmt.Sprintf("select a from trange3 where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + +func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_global_stats") + tk.MustExec("use test_global_stats") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range and list partition + tk.MustExec("create table thash(a int, b int, key(a)) partition by hash(a) partitions 4") + tk.MustExec(`create table trange(a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec(`create table tlist(a int, b int, key(a)) partition by list (a) ( + partition p0 values in (0, 1, 2, 3, 4, 5, 6, 7, 8, 9), + partition p0 values in (10, 11, 12, 13, 14, 15, 16, 17, 18, 19), + partition p0 values in (20, 21, 22, 23, 24, 25, 26, 27, 28, 29), + partition p0 values in (30, 31, 32, 33, 34, 35, 36, 37, 38, 39), + partition p0 values in (40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))`) + + // construct some special data distribution + vals := make([]string, 0, 1000) + listVals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + if i < 10 { + // for hash and range partition, 1% of records are in [0, 100) + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(100), rand.Intn(100))) + // for list partition, 1% of records are equal to 0 + listVals = append(listVals, "(0, 0)") + } else { + vals = append(vals, fmt.Sprintf("(%v, %v)", 100+rand.Intn(900), 100+rand.Intn(900))) + listVals = append(listVals, fmt.Sprintf("(%v, %v)", 1+rand.Intn(50), 1+rand.Intn(50))) + } + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into tlist values " + strings.Join(listVals, ",")) + + // before analyzing, the planner will choose TableScan to access the 1% of records + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + tk.MustExec("analyze table thash") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table tlist") + + // after analyzing, the planner will use the Index(a) + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") + + // create SQL bindings + tk.MustExec("create session binding for select * from thash where a<100 using select * from thash ignore index(a) where a<100") + tk.MustExec("create session binding for select * from trange where a<100 using select * from trange ignore index(a) where a<100") + tk.MustExec("create session binding for select * from tlist where a<100 using select * from tlist ignore index(a) where a<100") + + // use TableScan again since the Index(a) is ignored + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + // drop SQL bindings + tk.MustExec("drop session binding for select * from thash where a<100") + tk.MustExec("drop session binding for select * from trange where a<100") + tk.MustExec("drop session binding for select * from tlist where a<100") + + // use Index(a) again + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") +} + +func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_partition_joins") + tk.MustExec("use test_partition_joins") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range partition + tk.MustExec("create table thash(a int, b int, key(a)) partition by hash(a) partitions 4") + tk.MustExec("create table tregular1(a int, b int, key(a))") + + tk.MustExec(`create table trange(a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec("create table tregular2(a int, b int, key(a))") + + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) + + // random params + x1 := rand.Intn(1000) + x2 := rand.Intn(1000) + x3 := rand.Intn(1000) + x4 := rand.Intn(1000) + + // group 1 + // hash_join range partition and hash partition + queryHash := fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) + queryRegular := fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 2 + // hash_join range partition and regular table + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 3 + // merge_join range partition and hash partition + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 4 + // merge_join range partition and regular table + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // new table instances + tk.MustExec("create table thash2(a int, b int, index idx(a)) partition by hash(a) partitions 4") + tk.MustExec("create table tregular3(a int, b int, index idx(a))") + + tk.MustExec(`create table trange2(a int, b int, index idx(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec("create table tregular4(a int, b int, index idx(a))") + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into thash2 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular3 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into trange2 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular4 values " + strings.Join(vals, ",")) + + // group 5 + // index_merge_join range partition and range partition + // Currently don't support index merge join on two partition tables. Only test warning. + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v;", x1) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.a > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange.b > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.b > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + // group 6 + // index_merge_join range partition and regualr table + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and trange.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 7 + // index_hash_join hash partition and hash partition + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v);", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v) and thash2.a in (%v, %v);", x1, x2, x3, x4) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a > %v and thash2.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 8 + // index_hash_join hash partition and hash partition + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v);", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a > %v and tregular3.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + +func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { + tk.MustExec("create table trange(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") + tk.MustExec("create table thash(a int, b int) partition by hash(a) partitions 4;") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into trange values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into thash values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into t values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("set session tidb_partition_prune_mode='dynamic'") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table thash") + tk.MustExec("analyze table t") +} + +type testData4Expression struct { + sql string + partitions []string +} + +func (s *partitionTableSuite) TestDateColWithUnequalExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_datetime_unequal_expression") + tk.MustExec("create database db_datetime_unequal_expression") + tk.MustExec("use db_datetime_unequal_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec(`create table tp(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00"))`) + tk.MustExec(`create table t(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00"))`) + tk.MustExec(`insert into tp values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3)`) + tk.MustExec(`insert into t values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3)`) + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a != '2024-01-01 01:01:01'", + partitions: []string{"all"}, + }, + { + sql: "select * from %s where a != '2024-01-01 01:01:01' and a > '2015-09-09 00:00:00'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestToDaysColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_to_days_expression") + tk.MustExec("create database db_to_days_expression") + tk.MustExec("use db_to_days_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a date, b int) partition by range(to_days(a)) (partition p0 values less than (737822), partition p1 values less than (738019), partition p2 values less than (738154))") + tk.MustExec("create table t(a date, b int)") + tk.MustExec("insert into tp values('2020-01-01', 1), ('2020-03-02', 2), ('2020-05-05', 3), ('2020-11-11', 4)") + tk.MustExec("insert into t values('2020-01-01', 1), ('2020-03-02', 2), ('2020-05-05', 3), ('2020-11-11', 4)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '2020-08-16'", + partitions: []string{"p0,p1"}, + }, + { + sql: "select * from %s where a between '2020-05-01' and '2020-10-01'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestWeekdayWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_weekday_expression") + tk.MustExec("create database db_weekday_expression") + tk.MustExec("use db_weekday_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(3), partition p1 values less than(5), partition p2 values less than(8))") + tk.MustExec("create table t(a datetime, b int)") + tk.MustExec(`insert into tp values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0)`) + tk.MustExec(`insert into t values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0)`) + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a = '2020-08-17 00:00:00'", + partitions: []string{"p0"}, + }, + { + sql: "select * from %s where a= '2020-08-20 00:00:00' and a < '2020-08-22 00:00:00'", + partitions: []string{"p1"}, + }, + { + sql: " select * from %s where a < '2020-08-19 00:00:00'", + partitions: []string{"all"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestFloorUnixTimestampAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_floor_unix_timestamp_int_expression") + tk.MustExec("create database db_floor_unix_timestamp_int_expression") + tk.MustExec("use db_floor_unix_timestamp_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a timestamp, b int) partition by range(floor(unix_timestamp(a))) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400))") + tk.MustExec("create table t(a timestamp, b int)") + tk.MustExec("insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a > '2020-09-11 00:00:00'", + partitions: []string{"p2"}, + }, + { + sql: "select * from %s where a < '2020-07-07 01:00:00'", + partitions: []string{"p0,p1"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestUnixTimestampAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_unix_timestamp_int_expression") + tk.MustExec("create database db_unix_timestamp_int_expression") + tk.MustExec("use db_unix_timestamp_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a timestamp, b int) partition by range(unix_timestamp(a)) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400))") + tk.MustExec("create table t(a timestamp, b int)") + tk.MustExec("insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a > '2020-09-11 00:00:00'", + partitions: []string{"p2"}, + }, + { + sql: "select * from %s where a < '2020-07-07 01:00:00'", + partitions: []string{"p0,p1"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestDatetimeColAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_datetime_int_expression") + tk.MustExec("create database db_datetime_int_expression") + tk.MustExec("use db_datetime_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a datetime, b int) partition by range columns(a) (partition p0 values less than('2020-02-02 00:00:00'), partition p1 values less than('2020-09-01 00:00:00'), partition p2 values less than('2020-12-20 00:00:00'))") + tk.MustExec("create table t(a datetime, b int)") + tk.MustExec("insert into tp values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4)") + tk.MustExec("insert into t values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '2020-09-01 00:00:00'", + partitions: []string{"p0,p1"}, + }, + { + sql: "select * from %s where a > '2020-07-07 01:00:00'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestVarcharColAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_varchar_int_expression") + tk.MustExec("create database db_varchar_int_expression") + tk.MustExec("use db_varchar_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a varchar(255), b int) partition by range columns(a) (partition p0 values less than('ddd'), partition p1 values less than('ggggg'), partition p2 values less than('mmmmmm'))") + tk.MustExec("create table t(a varchar(255), b int)") + tk.MustExec("insert into tp values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6)") + tk.MustExec("insert into t values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '10'", + partitions: []string{"p0"}, + }, + { + sql: "select * from %s where a > 0", + partitions: []string{"all"}, + }, + { + sql: "select * from %s where a < 0", + partitions: []string{"all"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_equal_expression") + tk.MustExec("create database db_equal_expression") + tk.MustExec("use db_equal_expression") + createTable4DynamicPruneModeTestWithExpression(tk) + + tables := []string{"trange", "thash"} + tests := []testData4Expression{ + { + sql: "select * from %s where a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 4 or a = 1", + partitions: []string{ + "p0,p1", + "p0,p1", + }, + }, + { + sql: "select * from %s where a = -1", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a is NULL", + partitions: []string{ + "p0", + "p0", + }, + }, + { + sql: "select * from %s where b is NULL", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a > -1", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a >= 4 and a <= 5", + partitions: []string{ + "p1,p2", + "p0,p1", + }, + }, + { + sql: "select * from %s where a > 10", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a >=2 and a <= 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a between 2 and 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a < 2", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a <= 3", + partitions: []string{ + "p0,p1", + "all", + }, + }, + { + sql: "select * from %s where a in (2, 3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a in (1, 5)", + partitions: []string{ + "p0,p2", + "p1", + }, + }, + { + sql: "select * from %s where a not in (1, 5)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 and a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 2 and a = 3", + partitions: []string{ + // This means that we have no partition-read plan + "", + "", + }, + }, + { + sql: "select * from %s where a < 2 and a > 0", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a < 2 and a < 3", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a > 1 and a > 2", + partitions: []string{ + "p1,p2", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a = 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a in (3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a > 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a <= 1", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a between 2 and 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a != 2", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a > 4", + partitions: []string{ + "p2", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a != 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a = 3", + partitions: []string{ + "p1", + "p3", + }, + }, + { + sql: "select * from %s where not (a = 2)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where not (a > 2)", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where not (a < 2)", + partitions: []string{ + "all", + "all", + }, + }, + // cases that partition pruning can not work + { + sql: "select * from %s where a + 1 > 4", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a - 1 > 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a * 2 < 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a << 1 < 0", + partitions: []string{ + "all", + "all", + }, + }, + // comparison between int column and string column + { + sql: "select * from %s where a > '10'", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a > '10ab'", + partitions: []string{ + "dual", + "all", + }, + }, + } + + for _, t := range tests { + for i := range t.partitions { + sql := fmt.Sprintf(t.sql, tables[i]) + tk.MustPartition(sql, t.partitions[i]).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + } + } +} + +func (s *partitionTableSuite) TestAddDropPartitions(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_add_drop_partition") + tk.MustExec("use test_add_drop_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int) partition by range(a) ( + partition p0 values less than (5), + partition p1 values less than (10), + partition p2 values less than (15))`) + tk.MustExec(`insert into t values (2), (7), (12)`) + tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) + tk.MustPartition(`select * from t where a < 8`, "p0,p1").Sort().Check(testkit.Rows("2", "7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "2", "7")) + + // remove p0 + tk.MustExec(`alter table t drop partition p0`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "7")) + + // add 2 more partitions + tk.MustExec(`alter table t add partition (partition p3 values less than (20))`) + tk.MustExec(`alter table t add partition (partition p4 values less than (40))`) + tk.MustExec(`insert into t values (15), (25)`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) +} + +func (s *partitionTableSuite) PartitionPruningInTransaction(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_pruning_transaction") + defer tk.MustExec(`drop database test_pruning_transaction`) + tk.MustExec("use test_pruning_transaction") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec(`create table t(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11))`) + tk.MustExec(`begin`) + tk.MustPartition(`select * from t`, "all") + tk.MustPartition(`select * from t where a > 4`, "p1,p2") // partition pruning can work in transactions + tk.MustPartition(`select * from t where a > 7`, "p2") + tk.MustExec(`rollback`) +} + +func (s *partitionTableSuite) TestIssue25253(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database issue25253") + defer tk.MustExec("drop database issue25253") + tk.MustExec("use issue25253") + + tk.MustExec(`CREATE TABLE IDT_HP23902 ( + COL1 smallint DEFAULT NULL, + COL2 varchar(20) DEFAULT NULL, + COL4 datetime DEFAULT NULL, + COL3 bigint DEFAULT NULL, + COL5 float DEFAULT NULL, + KEY UK_COL1 (COL1) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + PARTITION BY HASH( COL1+30 ) + PARTITIONS 6`) + tk.MustExec(`insert ignore into IDT_HP23902 partition(p0, p1)(col1, col3) values(-10355, 1930590137900568573), (13810, -1332233145730692137)`) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1748 Found a row not matching the given partition set", + "Warning 1748 Found a row not matching the given partition set")) + tk.MustQuery(`select * from IDT_HP23902`).Check(testkit.Rows()) + + tk.MustExec(`create table t ( + a int + ) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (20))`) + tk.MustExec(`insert ignore into t partition(p0)(a) values(12)`) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1748 Found a row not matching the given partition set")) + tk.MustQuery(`select * from t`).Check(testkit.Rows()) +} + +func (s *partitionTableSuite) TestDML(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_DML") + defer tk.MustExec(`drop database test_DML`) + tk.MustExec("use test_DML") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tinner (a int, b int)`) + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000), + partition p4 values less than MAXVALUE)`) + + vals := make([]string, 0, 50) + for i := 0; i < 50; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tinner values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + // delete, insert, replace, update + for i := 0; i < 200; i++ { + var pattern string + switch rand.Intn(4) { + case 0: // delete + col := []string{"a", "b"}[rand.Intn(2)] + l := rand.Intn(40000) + r := l + rand.Intn(5000) + pattern = fmt.Sprintf(`delete from %%v where %v>%v and %v<%v`, col, l, col, r) + case 1: // insert + a, b := rand.Intn(40000), rand.Intn(40000) + pattern = fmt.Sprintf(`insert into %%v values (%v, %v)`, a, b) + case 2: // replace + a, b := rand.Intn(40000), rand.Intn(40000) + pattern = fmt.Sprintf(`replace into %%v(a, b) values (%v, %v)`, a, b) + case 3: // update + col := []string{"a", "b"}[rand.Intn(2)] + l := rand.Intn(40000) + r := l + rand.Intn(5000) + x := rand.Intn(1000) - 500 + pattern = fmt.Sprintf(`update %%v set %v=%v+%v where %v>%v and %v<%v`, col, col, x, col, l, col, r) + } + for _, tbl := range []string{"tinner", "thash", "trange"} { + tk.MustExec(fmt.Sprintf(pattern, tbl)) + } + + // check + r := tk.MustQuery(`select * from tinner`).Sort().Rows() + tk.MustQuery(`select * from thash`).Sort().Check(r) + tk.MustQuery(`select * from trange`).Sort().Check(r) + } +} + +func (s *partitionTableSuite) TestUnion(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_union") + defer tk.MustExec(`drop database test_union`) + tk.MustExec("use test_union") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int, b int, key(a))`) + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into t values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + randRange := func() (int, int) { + l, r := rand.Intn(40000), rand.Intn(40000) + if l > r { + l, r = r, l + } + return l, r + } + + for i := 0; i < 100; i++ { + a1l, a1r := randRange() + a2l, a2r := randRange() + b1l, b1r := randRange() + b2l, b2r := randRange() + for _, utype := range []string{"union all", "union distinct"} { + pattern := fmt.Sprintf(`select * from %%v where a>=%v and a<=%v and b>=%v and b<=%v + %v select * from %%v where a>=%v and a<=%v and b>=%v and b<=%v`, a1l, a1r, b1l, b1r, utype, a2l, a2r, b2l, b2r) + r := tk.MustQuery(fmt.Sprintf(pattern, "t", "t")).Sort().Rows() + tk.MustQuery(fmt.Sprintf(pattern, "thash", "thash")).Sort().Check(r) // hash + hash + tk.MustQuery(fmt.Sprintf(pattern, "trange", "trange")).Sort().Check(r) // range + range + tk.MustQuery(fmt.Sprintf(pattern, "trange", "thash")).Sort().Check(r) // range + hash + } + } +} + +func (s *partitionTableSuite) TestSubqueries(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_subquery") + defer tk.MustExec(`drop database test_subquery`) + tk.MustExec("use test_subquery") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table touter (a int, b int, index(a))`) + tk.MustExec(`create table tinner (a int, b int, c int, index(a))`) + tk.MustExec(`create table thash (a int, b int, c int, index(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, c int, index(a)) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000))`) + + outerVals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + outerVals = append(outerVals, fmt.Sprintf(`(%v, %v)`, rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into touter values ` + strings.Join(outerVals, ", ")) + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf(`(%v, %v, %v)`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tinner values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + // in + for i := 0; i < 50; i++ { + for _, op := range []string{"in", "not in"} { + x := rand.Intn(40000) + var r [][]interface{} + for _, t := range []string{"tinner", "thash", "trange"} { + q := fmt.Sprintf(`select * from touter where touter.a %v (select %v.b from %v where %v.a > touter.b and %v.c > %v)`, op, t, t, t, t, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } + } + + // exist + for i := 0; i < 50; i++ { + for _, op := range []string{"exists", "not exists"} { + x := rand.Intn(40000) + var r [][]interface{} + for _, t := range []string{"tinner", "thash", "trange"} { + q := fmt.Sprintf(`select * from touter where %v (select %v.b from %v where %v.a > touter.b and %v.c > %v)`, op, t, t, t, t, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } + } +} + +func (s *partitionTableSuite) TestSplitRegion(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_split_region") + tk.MustExec("use test_split_region") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int)`) + tk.MustExec(`create table thash (a int, b int, index(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, index(a)) partition by range(a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + tk.MustExec(`SPLIT TABLE thash INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + tk.MustExec(`SPLIT TABLE trange INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + + result := tk.MustQuery(`select * from tnormal where a>=1 and a<=15000`).Sort().Rows() + tk.MustPartition(`select * from trange where a>=1 and a<=15000`, "p0,p1").Sort().Check(result) + tk.MustPartition(`select * from thash where a>=1 and a<=15000`, "all").Sort().Check(result) + + result = tk.MustQuery(`select * from tnormal where a in (1, 10001, 20001)`).Sort().Rows() + tk.MustPartition(`select * from trange where a in (1, 10001, 20001)`, "p0,p1,p2").Sort().Check(result) + tk.MustPartition(`select * from thash where a in (1, 10001, 20001)`, "p1").Sort().Check(result) +} + +func (s *partitionTableSuite) TestParallelApply(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_parallel_apply") + tk.MustExec("use test_parallel_apply") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set tidb_enable_parallel_apply=true") + + tk.MustExec(`create table touter (a int, b int)`) + tk.MustExec(`create table tinner (a int, b int, key(a))`) + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, key(a)) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000))`) + + vouter := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vouter = append(vouter, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec("insert into touter values " + strings.Join(vouter, ", ")) + + vals := make([]string, 0, 2000) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec("insert into tinner values " + strings.Join(vals, ", ")) + tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) + tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) + + // parallel apply + hash partition + IndexReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(thash.a) from thash use index(a) where thash.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexReader 1.00 root partition:all index:StreamAgg`, // IndexReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.thash.a)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.thash.a, test_parallel_apply.touter.b)`, + ` └─IndexFullScan 10000.00 cop[tikv] table:thash, index:a(a) keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.a) from thash use index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.a) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + hash partition + TableReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(thash.b) from thash ignore index(a) where thash.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─TableReader 1.00 root partition:all data:StreamAgg`, // TableReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.thash.b)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.thash.a, test_parallel_apply.touter.b)`, + ` └─TableFullScan 10000.00 cop[tikv] table:thash keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.b) from thash ignore index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner ignore index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + hash partition + IndexLookUp as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexLookUp 1.00 root `, // IndexLookUp is a inner child of Apply + ` ├─Selection(Build) 8000.00 cop[tikv] gt(test_parallel_apply.tinner.a, test_parallel_apply.touter.b)`, + ` │ └─IndexFullScan 10000.00 cop[tikv] table:tinner, index:a(a) keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 cop[tikv] funcs:sum(test_parallel_apply.tinner.b)->Column#9`, + ` └─TableRowIDScan 8000.00 cop[tikv] table:tinner keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.b) from thash use index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + IndexReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(trange.a) from trange use index(a) where trange.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexReader 1.00 root partition:all index:StreamAgg`, // IndexReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.trange.a)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.trange.a, test_parallel_apply.touter.b)`, + ` └─IndexFullScan 10000.00 cop[tikv] table:trange, index:a(a) keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.a) from trange use index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.a) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + TableReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(trange.b) from trange ignore index(a) where trange.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─TableReader 1.00 root partition:all data:StreamAgg`, // TableReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.trange.b)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.trange.a, test_parallel_apply.touter.b)`, + ` └─TableFullScan 10000.00 cop[tikv] table:trange keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.b) from trange ignore index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner ignore index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + IndexLookUp as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexLookUp 1.00 root `, // IndexLookUp is a inner child of Apply + ` ├─Selection(Build) 8000.00 cop[tikv] gt(test_parallel_apply.tinner.a, test_parallel_apply.touter.b)`, + ` │ └─IndexFullScan 10000.00 cop[tikv] table:tinner, index:a(a) keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 cop[tikv] funcs:sum(test_parallel_apply.tinner.b)->Column#9`, + ` └─TableRowIDScan 8000.00 cop[tikv] table:tinner keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.b) from trange use index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // random queries + ops := []string{"!=", ">", "<", ">=", "<="} + aggFuncs := []string{"sum", "count", "max", "min"} + tbls := []string{"tinner", "thash", "trange"} + for i := 0; i < 50; i++ { + var r [][]interface{} + op := ops[rand.Intn(len(ops))] + agg := aggFuncs[rand.Intn(len(aggFuncs))] + x := rand.Intn(10000) + for _, tbl := range tbls { + q := fmt.Sprintf(`select * from touter where touter.a > (select %v(%v.b) from %v where %v.a%vtouter.b-%v)`, agg, tbl, tbl, tbl, op, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } +} + +func (s *partitionTableSuite) TestDirectReadingWithUnionScan(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_unionscan") + defer tk.MustExec(`drop database test_unionscan`) + tk.MustExec("use test_unionscan") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (30), + partition p2 values less than (50))`) + tk.MustExec(`create table thash(a int, b int, index idx_a(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table tnormal(a int, b int, index idx_a(a))`) + + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(50), rand.Intn(50))) + } + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + sql := fmt.Sprintf(`insert into %v values `+strings.Join(vals, ", "), tb) + tk.MustExec(sql) + } + + randCond := func(col string) string { + la, ra := rand.Intn(50), rand.Intn(50) + if la > ra { + la, ra = ra, la + } + return fmt.Sprintf(`%v>=%v and %v<=%v`, col, la, col, ra) + } + + tk.MustExec(`begin`) + for i := 0; i < 1000; i++ { + if i == 0 || rand.Intn(2) == 0 { // insert some inflight rows + val := fmt.Sprintf("(%v, %v)", rand.Intn(50), rand.Intn(50)) + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + sql := fmt.Sprintf(`insert into %v values `+val, tb) + tk.MustExec(sql) + } + } else { + var sql string + switch rand.Intn(3) { + case 0: // table scan + sql = `select * from %v ignore index(idx_a) where ` + randCond(`b`) + case 1: // index reader + sql = `select a from %v use index(idx_a) where ` + randCond(`a`) + case 2: // index lookup + sql = `select * from %v use index(idx_a) where ` + randCond(`a`) + ` and ` + randCond(`b`) + } + switch rand.Intn(2) { + case 0: // order by a + sql += ` order by a` + case 1: // order by b + sql += ` order by b` + } + + var result [][]interface{} + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + q := fmt.Sprintf(sql, tb) + tk.HasPlan(q, `UnionScan`) + if result == nil { + result = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(result) + } + } + } + } + tk.MustExec(`rollback`) +} + +func (s *partitionTableSuite) TestIssue25030(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_issue_25030") + tk.MustExec("use test_issue_25030") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`CREATE TABLE tbl_936 ( + col_5410 smallint NOT NULL, + col_5411 double, + col_5412 boolean NOT NULL DEFAULT 1, + col_5413 set('Alice', 'Bob', 'Charlie', 'David') NOT NULL DEFAULT 'Charlie', + col_5414 varbinary(147) COLLATE 'binary' DEFAULT 'bvpKgYWLfyuTiOYSkj', + col_5415 timestamp NOT NULL DEFAULT '2021-07-06', + col_5416 decimal(6, 6) DEFAULT 0.49, + col_5417 text COLLATE utf8_bin, + col_5418 float DEFAULT 2048.0762299371554, + col_5419 int UNSIGNED NOT NULL DEFAULT 3152326370, + PRIMARY KEY (col_5419) ) + PARTITION BY HASH (col_5419) PARTITIONS 3`) + tk.MustQuery(`SELECT last_value(col_5414) OVER w FROM tbl_936 + WINDOW w AS (ORDER BY col_5410, col_5411, col_5412, col_5413, col_5414, col_5415, col_5416, col_5417, col_5418, col_5419) + ORDER BY col_5410, col_5411, col_5412, col_5413, col_5414, col_5415, col_5416, col_5417, col_5418, col_5419, nth_value(col_5412, 5) OVER w`). + Check(testkit.Rows()) // can work properly without any error or panic +} + +func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_unsigned_partition") + tk.MustExec("use test_unsigned_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash_pk (a int unsigned, b int, primary key(a)) partition by hash (a) partitions 3`) + tk.MustExec(`create table trange_pk (a int unsigned, b int, primary key(a)) partition by range (a) ( + partition p1 values less than (100000), + partition p2 values less than (200000), + partition p3 values less than (300000), + partition p4 values less than (400000))`) + tk.MustExec(`create table tnormal_pk (a int unsigned, b int, primary key(a))`) + tk.MustExec(`create table thash_uniq (a int unsigned, b int, unique key(a)) partition by hash (a) partitions 3`) + tk.MustExec(`create table trange_uniq (a int unsigned, b int, unique key(a)) partition by range (a) ( + partition p1 values less than (100000), + partition p2 values less than (200000), + partition p3 values less than (300000), + partition p4 values less than (400000))`) + tk.MustExec(`create table tnormal_uniq (a int unsigned, b int, unique key(a))`) + + valColA := make(map[int]struct{}, 1000) + vals := make([]string, 0, 1000) + for len(vals) < 1000 { + a := rand.Intn(400000) + if _, ok := valColA[a]; ok { + continue + } + valColA[a] = struct{}{} + vals = append(vals, fmt.Sprintf("(%v, %v)", a, rand.Intn(400000))) + } + valStr := strings.Join(vals, ", ") + for _, tbl := range []string{"thash_pk", "trange_pk", "tnormal_pk", "thash_uniq", "trange_uniq", "tnormal_uniq"} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, valStr)) + } + + for i := 0; i < 100; i++ { + scanCond := fmt.Sprintf("a %v %v", []string{">", "<"}[rand.Intn(2)], rand.Intn(400000)) + pointCond := fmt.Sprintf("a = %v", rand.Intn(400000)) + batchCond := fmt.Sprintf("a in (%v, %v, %v)", rand.Intn(400000), rand.Intn(400000), rand.Intn(400000)) + + var rScan, rPoint, rBatch [][]interface{} + for tid, tbl := range []string{"tnormal_pk", "trange_pk", "thash_pk"} { + // unsigned + TableReader + scanSQL := fmt.Sprintf("select * from %v use index(primary) where %v", tbl, scanCond) + c.Assert(tk.HasPlan(scanSQL, "TableReader"), IsTrue) + r := tk.MustQuery(scanSQL).Sort() + if tid == 0 { + rScan = r.Rows() + } else { + r.Check(rScan) + } + + // unsigned + PointGet on PK + pointSQL := fmt.Sprintf("select * from %v use index(primary) where %v", tbl, pointCond) + tk.MustPointGet(pointSQL) + r = tk.MustQuery(pointSQL).Sort() + if tid == 0 { + rPoint = r.Rows() + } else { + r.Check(rPoint) + } + + // unsigned + BatchGet on PK + batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) + c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + r = tk.MustQuery(batchSQL).Sort() + if tid == 0 { + rBatch = r.Rows() + } else { + r.Check(rBatch) + } + } + + lookupCond := fmt.Sprintf("a %v %v", []string{">", "<"}[rand.Intn(2)], rand.Intn(400000)) + var rLookup [][]interface{} + for tid, tbl := range []string{"tnormal_uniq", "trange_uniq", "thash_uniq"} { + // unsigned + IndexReader + scanSQL := fmt.Sprintf("select a from %v use index(a) where %v", tbl, scanCond) + c.Assert(tk.HasPlan(scanSQL, "IndexReader"), IsTrue) + r := tk.MustQuery(scanSQL).Sort() + if tid == 0 { + rScan = r.Rows() + } else { + r.Check(rScan) + } + + // unsigned + IndexLookUp + lookupSQL := fmt.Sprintf("select * from %v use index(a) where %v", tbl, lookupCond) + tk.MustIndexLookup(lookupSQL) + r = tk.MustQuery(lookupSQL).Sort() + if tid == 0 { + rLookup = r.Rows() + } else { + r.Check(rLookup) + } + + // unsigned + PointGet on UniqueIndex + pointSQL := fmt.Sprintf("select * from %v use index(a) where %v", tbl, pointCond) + tk.MustPointGet(pointSQL) + r = tk.MustQuery(pointSQL).Sort() + if tid == 0 { + rPoint = r.Rows() + } else { + r.Check(rPoint) + } + + // unsigned + BatchGet on UniqueIndex + batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) + c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + r = tk.MustQuery(batchSQL).Sort() + if tid == 0 { + rBatch = r.Rows() + } else { + r.Check(rBatch) + } + } + } +} + +func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dr_agg") + tk.MustExec("use test_dr_agg") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec(`create table thash(a int, b int) partition by hash(a) partitions 4;`) + + // regular table + tk.MustExec("create table tregular1(a int, b int, index idx_a(a))") + tk.MustExec("create table tregular2(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(12)+1, rand.Intn(20))) + } + + tk.MustExec("insert into tlist values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) + + // test range partition + for i := 0; i < 200; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test hash partition + for i := 0; i < 200; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test list partition + for i := 0; i < 200; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(12) + 1 + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(12) + 1 + z := rand.Intn(12) + 1 + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestDynamicModeByDefault(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dynamic_by_default") + + tk.MustExec(`create table trange(a int, b int, primary key(a) clustered, index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than(500), + partition p2 values less than(1100));`) + tk.MustExec(`create table thash(a int, b int, primary key(a) clustered, index idx_b(b)) partition by hash(a) partitions 4;`) + + for _, q := range []string{ + "explain select * from trange where a>400", + "explain select * from thash where a>=100", + } { + for _, r := range tk.MustQuery(q).Rows() { + c.Assert(strings.Contains(strings.ToLower(r[0].(string)), "partitionunion"), IsFalse) + } + } +} + +func (s *partitionTableSuite) TestIssue24636(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_issue_24636") + tk.MustExec("use test_issue_24636") + + tk.MustExec(`CREATE TABLE t (a int, b date, c int, PRIMARY KEY (a,b)) + PARTITION BY RANGE ( TO_DAYS(b) ) ( + PARTITION p0 VALUES LESS THAN (737821), + PARTITION p1 VALUES LESS THAN (738289) + )`) + tk.MustExec(`INSERT INTO t (a, b, c) VALUES(0, '2021-05-05', 0)`) + tk.MustQuery(`select c from t use index(primary) where a=0 limit 1`).Check(testkit.Rows("0")) + + tk.MustExec(` + CREATE TABLE test_partition ( + a varchar(100) NOT NULL, + b date NOT NULL, + c varchar(100) NOT NULL, + d datetime DEFAULT NULL, + e datetime DEFAULT NULL, + f bigint(20) DEFAULT NULL, + g bigint(20) DEFAULT NULL, + h bigint(20) DEFAULT NULL, + i bigint(20) DEFAULT NULL, + j bigint(20) DEFAULT NULL, + k bigint(20) DEFAULT NULL, + l bigint(20) DEFAULT NULL, + PRIMARY KEY (a,b,c) /*T![clustered_index] NONCLUSTERED */ + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + PARTITION BY RANGE ( TO_DAYS(b) ) ( + PARTITION pmin VALUES LESS THAN (737821), + PARTITION p20200601 VALUES LESS THAN (738289))`) + tk.MustExec(`INSERT INTO test_partition (a, b, c, d, e, f, g, h, i, j, k, l) VALUES('aaa', '2021-05-05', '428ff6a1-bb37-42ac-9883-33d7a29961e6', '2021-05-06 08:13:38', '2021-05-06 13:28:08', 0, 8, 3, 0, 9, 1, 0)`) + tk.MustQuery(`select c,j,l from test_partition where c='428ff6a1-bb37-42ac-9883-33d7a29961e6' and a='aaa' limit 0, 200`).Check(testkit.Rows("428ff6a1-bb37-42ac-9883-33d7a29961e6 9 0")) +} + +func (s *partitionTableSuite) TestIdexMerge(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_idx_merge") + tk.MustExec("use test_idx_merge") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, primary key(a) clustered, index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, b int, primary key(a) clustered, index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec(`create table thash(a int, b int, primary key(a) clustered, index idx_b(b)) partition by hash(a) partitions 4;`) + + // regular table + tk.MustExec("create table tregular1(a int, b int, primary key(a) clustered)") + tk.MustExec("create table tregular2(a int, b int, primary key(a) clustered)") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + + tk.MustExec("insert ignore into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(12)+1, rand.Intn(20))) + } + + tk.MustExec("insert ignore into tlist values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tregular2 values " + strings.Join(vals, ",")) + + // test range partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(1099) + x2 := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test hash partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(1099) + x2 := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregualr1) */ * from tregular1 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test list partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(12) + 1 + x2 := rand.Intn(12) + 1 + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestIssue25309(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_issue_25309") + tk.MustExec("use test_issue_25309") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`CREATE TABLE tbl_500 ( + col_20 tinyint(4) NOT NULL, + col_21 varchar(399) CHARACTER SET utf8 COLLATE utf8_unicode_ci DEFAULT NULL, + col_22 json DEFAULT NULL, + col_23 blob DEFAULT NULL, + col_24 mediumint(9) NOT NULL, + col_25 float NOT NULL DEFAULT '7306.384497585912', + col_26 binary(196) NOT NULL, + col_27 timestamp DEFAULT '1976-12-08 00:00:00', + col_28 bigint(20) NOT NULL, + col_29 tinyint(1) NOT NULL DEFAULT '1', + PRIMARY KEY (col_29,col_20) /*T![clustered_index] NONCLUSTERED */, + KEY idx_7 (col_28,col_20,col_26,col_27,col_21,col_24), + KEY idx_8 (col_25,col_29,col_24) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`) + + tk.MustExec(`CREATE TABLE tbl_600 ( + col_60 int(11) NOT NULL DEFAULT '-776833487', + col_61 tinyint(1) NOT NULL DEFAULT '1', + col_62 tinyint(4) NOT NULL DEFAULT '-125', + PRIMARY KEY (col_62,col_60,col_61) /*T![clustered_index] NONCLUSTERED */, + KEY idx_19 (col_60) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_unicode_ci + PARTITION BY HASH( col_60 ) + PARTITIONS 1`) + + tk.MustExec(`insert into tbl_500 select -34, 'lrfGPPPUuZjtT', '{"obj1": {"sub_obj0": 100}}', 0x6C47636D, 1325624, 7306.3843, 'abc', '1976-12-08', 4757891479624162031, 0`) + tk.MustQuery(`select tbl_5.* from tbl_500 tbl_5 where col_24 in ( select col_62 from tbl_600 where tbl_5.col_26 < 'hSvHLdQeGBNIyOFXStV' )`).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 *partitionTableSuite) TestIssue20028(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("set @@tidb_partition_prune_mode='static-only'") + tk.MustExec(`create table t1 (c_datetime datetime, primary key (c_datetime)) +partition by range (to_days(c_datetime)) ( partition p0 values less than (to_days('2020-02-01')), +partition p1 values less than (to_days('2020-04-01')), +partition p2 values less than (to_days('2020-06-01')), +partition p3 values less than maxvalue)`) + tk.MustExec("create table t2 (c_datetime datetime, unique key(c_datetime))") + tk.MustExec("insert into t1 values ('2020-06-26 03:24:00'), ('2020-02-21 07:15:33'), ('2020-04-27 13:50:58')") + tk.MustExec("insert into t2 values ('2020-01-10 09:36:00'), ('2020-02-04 06:00:00'), ('2020-06-12 03:45:18')") + tk.MustExec("begin") + tk.MustQuery("select * from t1 join t2 on t1.c_datetime >= t2.c_datetime for update"). + Sort(). + Check(testkit.Rows( + "2020-02-21 07:15:33 2020-01-10 09:36:00", + "2020-02-21 07:15:33 2020-02-04 06:00:00", + "2020-04-27 13:50:58 2020-01-10 09:36:00", + "2020-04-27 13:50:58 2020-02-04 06:00:00", + "2020-06-26 03:24:00 2020-01-10 09:36:00", + "2020-06-26 03:24:00 2020-02-04 06:00:00", + "2020-06-26 03:24:00 2020-06-12 03:45:18")) + tk.MustExec("rollback") +} + +func (s *partitionTableSuite) TestSelectLockOnPartitionTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists pt") + tk.MustExec(`create table pt (id int primary key, k int, c int, index(k)) +partition by range (id) ( +partition p0 values less than (4), +partition p1 values less than (7), +partition p2 values less than (11))`) + + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use test") + + optimisticTableReader := func() { + tk.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk.MustExec("begin") + tk.MustQuery("select id, k from pt ignore index (k) where k = 5 for update").Check(testkit.Rows("5 5")) + tk2.MustExec("update pt set c = c + 1 where k = 5") + _, err := tk.Exec("commit") + c.Assert(err, NotNil) // Write conflict + } + + optimisticIndexReader := func() { + tk.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk.MustExec("begin") + // This is not index reader actually. + tk.MustQuery("select k from pt where k = 5 for update").Check(testkit.Rows("5")) + tk2.MustExec("update pt set c = c + 1 where k = 5") + _, err := tk.Exec("commit") + c.Assert(err, NotNil) + } + + optimisticIndexLookUp := func() { + tk.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'optimistic'") + tk.MustExec("begin") + tk.MustQuery("select c, k from pt use index (k) where k = 5 for update").Check(testkit.Rows("5 5")) + tk2.MustExec("update pt set c = c + 1 where k = 5") + _, err := tk.Exec("commit") + c.Assert(err, NotNil) + } + + pessimisticTableReader := func() { + tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk.MustExec("begin") + tk.MustQuery("select id, k from pt ignore index (k) where k = 5 for update").Check(testkit.Rows("5 5")) + ch := make(chan int, 2) + go func() { + tk2.MustExec("update pt set c = c + 1 where k = 5") + ch <- 1 + }() + time.Sleep(100 * time.Millisecond) + ch <- 2 + + // Check the operation in the goroutine is blocked, if not the first result in + // the channel should be 1. + c.Assert(<-ch, Equals, 2) + + tk.MustExec("commit") + <-ch + tk.MustQuery("select c from pt where k = 5").Check(testkit.Rows("6")) + } + + pessimisticIndexReader := func() { + tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk.MustExec("begin") + // This is not index reader actually. + tk.MustQuery("select k from pt where k = 5 for update").Check(testkit.Rows("5")) + ch := make(chan int, 2) + go func() { + tk2.MustExec("update pt set c = c + 1 where k = 5") + ch <- 1 + }() + time.Sleep(100 * time.Millisecond) + ch <- 2 + + // Check the operation in the goroutine is blocked, + c.Assert(<-ch, Equals, 2) + + tk.MustExec("commit") + <-ch + tk.MustQuery("select c from pt where k = 5").Check(testkit.Rows("6")) + } + + pessimisticIndexLookUp := func() { + tk.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk.MustExec("begin") + tk.MustQuery("select c, k from pt use index (k) where k = 5 for update").Check(testkit.Rows("5 5")) + ch := make(chan int, 2) + go func() { + tk2.MustExec("update pt set c = c + 1 where k = 5") + ch <- 1 + }() + time.Sleep(100 * time.Millisecond) + ch <- 2 + + // Check the operation in the goroutine is blocked, + c.Assert(<-ch, Equals, 2) + + tk.MustExec("commit") + <-ch + tk.MustQuery("select c from pt where k = 5").Check(testkit.Rows("6")) + } + + partitionModes := []string{ + "'dynamic-only'", + "'static-only'", + } + testCases := []func(){ + optimisticTableReader, + optimisticIndexLookUp, + optimisticIndexReader, + pessimisticTableReader, + pessimisticIndexReader, + pessimisticIndexLookUp, + } + + for _, mode := range partitionModes { + tk.MustExec("set @@tidb_partition_prune_mode=" + mode) + for _, c := range testCases { + tk.MustExec("replace into pt values (5, 5, 5)") + c() + } + } +} + +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));") +} + +type testOutput struct { + SQL string + Plan []string + Res []string +} + +func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { + for i, tt := range input { + var isSelect bool = false + if strings.HasPrefix(strings.ToLower(tt), "select ") { + isSelect = true + } + s.testData.OnRecord(func() { + output[i].SQL = tt + if isSelect { + output[i].Plan = s.testData.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + } else { + // Just verify SELECT (also avoid double INSERTs during record) + output[i].Res = nil + output[i].Plan = nil + } + }) + if isSelect { + tk.UsedPartitions(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } else { + tk.MustExec(tt) + } + } +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesEq(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("CREATE DATABASE TestRangePartitionBoundaries") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundaries") + tk.MustExec("USE TestRangePartitionBoundaries") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000)); +`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesNe(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("CREATE DATABASE TestRangePartitionBoundariesNe") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesNe") + tk.MustExec("USE TestRangePartitionBoundariesNe") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenM(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenM") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenM") + tk.MustExec("USE TestRangePartitionBoundariesBetweenM") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenS(c *C) { + c.Skip("unstable, skip it and fix it before 20210624") + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenS") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenS") + tk.MustExec("USE TestRangePartitionBoundariesBetweenS") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesLtM(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("create database TestRangePartitionBoundariesLtM") + defer tk.MustExec("drop database TestRangePartitionBoundariesLtM") + tk.MustExec("use TestRangePartitionBoundariesLtM") + tk.MustExec("drop table if exists t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesLtS(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("create database TestRangePartitionBoundariesLtS") + defer tk.MustExec("drop database TestRangePartitionBoundariesLtS") + tk.MustExec("use TestRangePartitionBoundariesLtS") + tk.MustExec("drop table if exists t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) diff --git a/executor/table_reader.go b/executor/table_reader.go index 712a17ab72f54..bafd9d1c394df 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -90,6 +91,24 @@ type TableReaderExecutor struct { virtualColumnRetFieldTypes []*types.FieldType // batchCop indicates whether use super batch coprocessor request, only works for TiFlash engine. batchCop bool + + // extraPIDColumnIndex is used for partition reader to add an extra partition ID column. + extraPIDColumnIndex offsetOptional +} + +// offsetOptional may be a positive integer, or invalid. +type offsetOptional int + +func newOffset(i int) offsetOptional { + return offsetOptional(i + 1) +} + +func (i offsetOptional) valid() bool { + return i != 0 +} + +func (i offsetOptional) value() int { + return int(i - 1) } // Open initialzes necessary variables for using this executor. @@ -181,9 +200,26 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return err } + // When 'select ... for update' work on a partitioned table, the table reader should + // add the partition ID as an extra column. The SelectLockExec need this information + // to construct the lock key. + physicalID := getPhysicalTableID(e.table) + if e.extraPIDColumnIndex.valid() { + fillExtraPIDColumn(req, e.extraPIDColumnIndex.value(), physicalID) + } + return nil } +func fillExtraPIDColumn(req *chunk.Chunk, extraPIDColumnIndex int, physicalID int64) { + numRows := req.NumRows() + pidColumn := chunk.NewColumn(types.NewFieldType(mysql.TypeLonglong), numRows) + for i := 0; i < numRows; i++ { + pidColumn.AppendInt64(physicalID) + } + req.SetCol(extraPIDColumnIndex, pidColumn) +} + // Close implements the Executor Close interface. func (e *TableReaderExecutor) Close() error { var err error diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 985e8ff0bf61f..9aa2ef6840726 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2039,6 +2039,7 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P Lock: p.Lock, TblID2Handle: p.tblID2Handle, PartitionedTable: p.partitionedTable, + ExtraPIDInfo: p.extraPIDInfo, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) return []PhysicalPlan{lock}, true } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 1ecdfc06c9767..cdf2b89613685 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3088,7 +3088,14 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("LOCK IN SHARE MODE") return nil, err } +<<<<<<< HEAD p = b.buildSelectLock(p, sel.LockTp) +======= + p, err = b.buildSelectLock(p, sel.LockInfo) + if err != nil { + return nil, err + } +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) } b.handleHelper.popMap() b.handleHelper.pushMap(nil) @@ -3213,6 +3220,41 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { } } +<<<<<<< HEAD +======= +// addExtraPIDColumn add an extra PID column for partition table. +// 'select ... for update' on a partition table need to know the partition ID +// to construct the lock key, so this column is added to the chunk row. +func (ds *DataSource) addExtraPIDColumn(info *extraPIDInfo) { + pidCol := &expression.Column{ + RetType: types.NewFieldType(mysql.TypeLonglong), + UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), + ID: model.ExtraPidColID, + OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPartitionIdName), + } + + ds.Columns = append(ds.Columns, model.NewExtraPartitionIDColInfo()) + schema := ds.Schema() + schema.Append(pidCol) + ds.names = append(ds.names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.TableInfo().Name, + ColName: model.ExtraPartitionIdName, + OrigColName: model.ExtraPartitionIdName, + }) + ds.TblCols = append(ds.TblCols, pidCol) + + info.Columns = append(info.Columns, pidCol) + info.TblIDs = append(info.TblIDs, ds.TableInfo().ID) + return +} + +var ( + pseudoEstimationNotAvailable = metrics.PseudoEstimation.WithLabelValues("nodata") + pseudoEstimationOutdate = metrics.PseudoEstimation.WithLabelValues("outdate") +) + +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) // getStatsTable gets statistics information for a table specified by "tableID". // A pseudo statistics table is returned in any of the following scenario: // 1. tidb-server started and statistics handle has not been initialized. @@ -3945,7 +3987,16 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( // buildSelectLock is an optimization that can reduce RPC call. // We only need do this optimization for single table update which is the most common case. // When TableRefs.Right is nil, it is single table update. +<<<<<<< HEAD p = b.buildSelectLock(p, ast.SelectLockForUpdate) +======= + p, err = b.buildSelectLock(p, &ast.SelectLockInfo{ + LockType: ast.SelectLockForUpdate, + }) + if err != nil { + return nil, err + } +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) } } @@ -4238,7 +4289,16 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( } if b.ctx.GetSessionVars().TxnCtx.IsPessimistic { if !delete.IsMultiTable { +<<<<<<< HEAD p = b.buildSelectLock(p, ast.SelectLockForUpdate) +======= + p, err = b.buildSelectLock(p, &ast.SelectLockInfo{ + LockType: ast.SelectLockForUpdate, + }) + if err != nil { + return nil, err + } +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) } } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index eec168145c183..3d275edd33f2c 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -963,6 +963,16 @@ type LogicalLimit struct { limitHints limitHintInfo } +// extraPIDInfo is used by SelectLock on partitioned table, the TableReader need +// to return the partition id column. +// Because SelectLock has to used that partition id to encode the lock key. +// the child of SelectLock may be Join, so that table can be multiple extra PID columns. +// fields are for each of the table, and TblIDs are the corresponding table IDs. +type extraPIDInfo struct { + Columns []*expression.Column + TblIDs []int64 +} + // LogicalLock represents a select lock plan. type LogicalLock struct { baseLogicalPlan @@ -970,6 +980,9 @@ type LogicalLock struct { Lock ast.SelectLockType tblID2Handle map[int64][]*expression.Column partitionedTable []table.PartitionedTable + // extraPIDInfo is used when it works on partition table, the child executor + // need to return an extra partition ID column in the chunk row. + extraPIDInfo } // WindowFrame represents a window function frame. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 3f2ebbb42e4c7..c0f8820d5f03f 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -277,6 +277,8 @@ type PhysicalTableScan struct { IsGlobalRead bool // The table scan may be a partition, rather than a real table. + // TODO: clean up this field. After we support dynamic partitioning, table scan + // works on the whole partition table, and `isPartition` is not used. isPartition bool // KeepOrder is true, if sort data by scanning pkcol, KeepOrder bool @@ -479,6 +481,7 @@ type PhysicalLock struct { TblID2Handle map[int64][]*expression.Column PartitionedTable []table.PartitionedTable + ExtraPIDInfo extraPIDInfo } // PhysicalLimit is the physical operator of Limit. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 223fe9c7ecdd2..9709d3179a41a 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1046,14 +1046,51 @@ func removeIgnoredPaths(paths, ignoredPaths []*util.AccessPath, tblInfo *model.T return remainedPaths } +<<<<<<< HEAD func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock ast.SelectLockType) *LogicalLock { +======= +func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock *ast.SelectLockInfo) (*LogicalLock, error) { +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) selectLock := LogicalLock{ Lock: lock, tblID2Handle: b.handleHelper.tailMap(), partitionedTable: b.partitionedTable, }.Init(b.ctx) selectLock.SetChildren(src) - return selectLock + + if len(b.partitionedTable) > 0 { + // If a chunk row is read from a partitioned table, which partition the row + // comes from is unknown. With the existence of Join, the situation could be + // even worse: SelectLock have to know the `pid` to construct the lock key. + // To solve the problem, an extra `pid` column is add to the schema, and the + // DataSource need to return the `pid` information in the chunk row. + err := addExtraPIDColumnToDataSource(src, &selectLock.extraPIDInfo) + if err != nil { + return nil, err + } + // TODO: Dynamic partition mode does not support adding extra pid column to the data source. + // (Because one table reader can read from multiple partitions, which partition a chunk row comes from is unknown) + // So we have to use the old "rewrite to union" way here, set `flagPartitionProcessor` flag for that. + b.optFlag = b.optFlag | flagPartitionProcessor + } + return selectLock, nil +} + +func addExtraPIDColumnToDataSource(p LogicalPlan, info *extraPIDInfo) error { + switch raw := p.(type) { + case *DataSource: + raw.addExtraPIDColumn(info) + return nil + default: + var err error + for _, child := range p.Children() { + err = addExtraPIDColumnToDataSource(child, info) + if err != nil { + return err + } + } + } + return nil } func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 7bc633bdfdb3d..c065422afed52 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -359,9 +359,8 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column) error { } if len(p.partitionedTable) > 0 { - // If the children include partitioned tables, do not prune columns. - // Because the executor needs the partitioned columns to calculate the lock key. - return p.children[0].PruneColumns(p.Schema().Columns) + // If the children include partitioned tables, there is an extra partition ID column. + parentUsedCols = append(parentUsedCols, p.extraPIDInfo.Columns...) } for _, cols := range p.tblID2Handle { diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index be40d793e77bb..eee9fbf36d43d 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -14,11 +14,11 @@ package core import ( "context" - "errors" "fmt" "sort" "strings" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -113,8 +113,474 @@ func generateHashPartitionExpr(t table.Table, ctx sessionctx.Context, columns [] return exprs[0], nil } +<<<<<<< HEAD func (s *partitionProcessor) pruneHashPartition(ds *DataSource, pi *model.PartitionInfo) (LogicalPlan, error) { pe, err := generateHashPartitionExpr(ds.table, ds.ctx, ds.TblCols, ds.names) +======= +func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, + conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, []expression.Expression, error) { + pi := tbl.Meta().Partition + pe, err := generateHashPartitionExpr(ctx, pi, columns, names) + if err != nil { + return nil, nil, err + } + partIdx := expression.ExtractColumns(pe) + colLen := make([]int, 0, len(partIdx)) + for i := 0; i < len(partIdx); i++ { + partIdx[i].Index = i + colLen = append(colLen, types.UnspecifiedLength) + } + detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partIdx, colLen) + if err != nil { + return nil, nil, err + } + ranges := detachedResult.Ranges + used := make([]int, 0, len(ranges)) + for _, r := range ranges { + if r.IsPointNullable(ctx.GetSessionVars().StmtCtx) { + if !r.HighVal[0].IsNull() { + if len(r.HighVal) != len(partIdx) { + used = []int{-1} + break + } + } + highLowVals := make([]types.Datum, 0, len(r.HighVal)+len(r.LowVal)) + highLowVals = append(highLowVals, r.HighVal...) + highLowVals = append(highLowVals, r.LowVal...) + pos, isNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(highLowVals).ToRow()) + if err != nil { + return nil, nil, err + } + if isNull { + pos = 0 + } + idx := math.Abs(pos % int64(pi.Num)) + if len(partitionNames) > 0 && !s.findByName(partitionNames, pi.Definitions[idx].Name.L) { + continue + } + used = append(used, int(idx)) + } else { + // processing hash partition pruning. eg: + // create table t2 (a int, b bigint, index (a), index (b)) partition by hash(a) partitions 10; + // desc select * from t2 where t2.a between 10 and 15; + // determine whether the partition key is int + if col, ok := pe.(*expression.Column); ok && col.RetType.EvalType() == types.ETInt { + numPartitions := len(pi.Definitions) + + posHigh, highIsNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) + if err != nil { + return nil, nil, err + } + + posLow, lowIsNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(r.LowVal).ToRow()) + if err != nil { + return nil, nil, err + } + + // consider whether the range is closed or open + if r.LowExclude { + posLow++ + } + if r.HighExclude { + posHigh-- + } + rangeScalar := posHigh - posLow + + // if range is less than the number of partitions, there will be unused partitions we can prune out. + if rangeScalar < int64(numPartitions) && !highIsNull && !lowIsNull { + for i := posLow; i <= posHigh; i++ { + idx := math.Abs(i % int64(pi.Num)) + if len(partitionNames) > 0 && !s.findByName(partitionNames, pi.Definitions[idx].Name.L) { + continue + } + used = append(used, int(idx)) + } + continue + } + + // issue:#22619 + if col.RetType.Tp == mysql.TypeBit { + // maximum number of partitions is 8192 + if col.RetType.Flen > 0 && col.RetType.Flen < int(math.Log2(ddl.PartitionCountLimit)) { + // all possible hash values + maxUsedPartitions := 1 << col.RetType.Flen + if maxUsedPartitions < numPartitions { + for i := 0; i < maxUsedPartitions; i++ { + used = append(used, i) + } + continue + } + } + } + } + + used = []int{FullRange} + break + } + } + if len(partitionNames) > 0 && len(used) == 1 && used[0] == FullRange { + or := partitionRangeOR{partitionRange{0, len(pi.Definitions)}} + return s.convertToIntSlice(or, pi, partitionNames), nil, nil + } + sort.Ints(used) + ret := used[:0] + for i := 0; i < len(used); i++ { + if i == 0 || used[i] != used[i-1] { + ret = append(ret, used[i]) + } + } + return ret, detachedResult.RemainedConds, nil +} + +func (s *partitionProcessor) convertToIntSlice(or partitionRangeOR, pi *model.PartitionInfo, partitionNames []model.CIStr) []int { + if len(or) == 1 && or[0].start == 0 && or[0].end == len(pi.Definitions) { + if len(partitionNames) == 0 { + return []int{FullRange} + } + } + ret := make([]int, 0, len(or)) + for i := 0; i < len(or); i++ { + for pos := or[i].start; pos < or[i].end; pos++ { + if len(partitionNames) > 0 && !s.findByName(partitionNames, pi.Definitions[pos].Name.L) { + continue + } + ret = append(ret, pos) + } + } + return ret +} + +func convertToRangeOr(used []int, pi *model.PartitionInfo) partitionRangeOR { + if len(used) == 1 && used[0] == -1 { + return fullRange(len(pi.Definitions)) + } + ret := make(partitionRangeOR, 0, len(used)) + for _, i := range used { + ret = append(ret, partitionRange{i, i + 1}) + } + return ret +} + +func (s *partitionProcessor) pruneHashPartition(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, + conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { + used, _, err := s.findUsedPartitions(ctx, tbl, partitionNames, conds, columns, names) + if err != nil { + return nil, err + } + return used, nil +} + +// reconstructTableColNames reconstructs FieldsNames according to ds.TblCols. +// ds.names may not match ds.TblCols since ds.names is pruned while ds.TblCols contains all original columns. +// please see https://github.com/pingcap/tidb/issues/22635 for more details. +func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types.FieldName, error) { + names := make([]*types.FieldName, 0, len(ds.TblCols)) + colsInfo := ds.table.FullHiddenColsAndVisibleCols() + colsInfoMap := make(map[int64]*table.Column, len(colsInfo)) + for _, c := range colsInfo { + colsInfoMap[c.ID] = c + } + for _, colExpr := range ds.TblCols { + if colExpr.ID == model.ExtraHandleID { + names = append(names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.tableInfo.Name, + ColName: model.ExtraHandleName, + OrigColName: model.ExtraHandleName, + }) + continue + } + if colExpr.ID == model.ExtraPidColID { + names = append(names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.tableInfo.Name, + ColName: model.ExtraPartitionIdName, + OrigColName: model.ExtraPartitionIdName, + }) + continue + } + if colInfo, found := colsInfoMap[colExpr.ID]; found { + names = append(names, &types.FieldName{ + DBName: ds.DBName, + TblName: ds.tableInfo.Name, + ColName: colInfo.Name, + OrigTblName: ds.tableInfo.Name, + OrigColName: colInfo.Name, + Hidden: colInfo.Hidden, + }) + continue + } + return nil, errors.Trace(fmt.Errorf("information of column %v is not found", colExpr.String())) + } + return names, nil +} + +func (s *partitionProcessor) processHashPartition(ds *DataSource, pi *model.PartitionInfo) (LogicalPlan, error) { + names, err := s.reconstructTableColNames(ds) + if err != nil { + return nil, err + } + used, err := s.pruneHashPartition(ds.SCtx(), ds.table, ds.partitionNames, ds.allConds, ds.TblCols, names) + if err != nil { + return nil, err + } + if used != nil { + return s.makeUnionAllChildren(ds, pi, convertToRangeOr(used, pi)) + } + tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.blockOffset) + tableDual.schema = ds.Schema() + return tableDual, nil +} + +// listPartitionPruner uses to prune partition for list partition. +type listPartitionPruner struct { + *partitionProcessor + ctx sessionctx.Context + pi *model.PartitionInfo + partitionNames []model.CIStr + colIDToUniqueID map[int64]int64 + fullRange map[int]struct{} + listPrune *tables.ForListPruning +} + +func newListPartitionPruner(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, + s *partitionProcessor, conds []expression.Expression, pruneList *tables.ForListPruning) *listPartitionPruner { + colIDToUniqueID := make(map[int64]int64) + for _, cond := range conds { + condCols := expression.ExtractColumns(cond) + for _, c := range condCols { + colIDToUniqueID[c.ID] = c.UniqueID + } + } + fullRange := make(map[int]struct{}) + fullRange[FullRange] = struct{}{} + return &listPartitionPruner{ + partitionProcessor: s, + ctx: ctx, + pi: tbl.Meta().Partition, + partitionNames: partitionNames, + colIDToUniqueID: colIDToUniqueID, + fullRange: fullRange, + listPrune: pruneList, + } +} + +func (l *listPartitionPruner) locatePartition(cond expression.Expression) (tables.ListPartitionLocation, bool, error) { + switch sf := cond.(type) { + case *expression.Constant: + b, err := sf.Value.ToBool(l.ctx.GetSessionVars().StmtCtx) + if err == nil && b == 0 { + // A constant false expression. + return nil, false, nil + } + case *expression.ScalarFunction: + switch sf.FuncName.L { + case ast.LogicOr: + dnfItems := expression.FlattenDNFConditions(sf) + return l.locatePartitionByDNFCondition(dnfItems) + case ast.LogicAnd: + cnfItems := expression.FlattenCNFConditions(sf) + return l.locatePartitionByCNFCondition(cnfItems) + } + return l.locatePartitionByColumn(sf) + } + return nil, true, nil +} + +func (l *listPartitionPruner) locatePartitionByCNFCondition(conds []expression.Expression) (tables.ListPartitionLocation, bool, error) { + if len(conds) == 0 { + return nil, true, nil + } + countFull := 0 + helper := tables.NewListPartitionLocationHelper() + for _, cond := range conds { + cnfLoc, isFull, err := l.locatePartition(cond) + if err != nil { + return nil, false, err + } + if isFull { + countFull++ + continue + } + if cnfLoc.IsEmpty() { + // No partition for intersection, just return 0 partition. + return nil, false, nil + } + if !helper.Intersect(cnfLoc) { + return nil, false, nil + } + } + if countFull == len(conds) { + return nil, true, nil + } + return helper.GetLocation(), false, nil +} + +func (l *listPartitionPruner) locatePartitionByDNFCondition(conds []expression.Expression) (tables.ListPartitionLocation, bool, error) { + if len(conds) == 0 { + return nil, true, nil + } + helper := tables.NewListPartitionLocationHelper() + for _, cond := range conds { + dnfLoc, isFull, err := l.locatePartition(cond) + if err != nil || isFull { + return nil, isFull, err + } + helper.Union(dnfLoc) + } + return helper.GetLocation(), false, nil +} + +// locatePartitionByColumn uses to locate partition by the one of the list columns value. +// Such as: partition by list columns(a,b) (partition p0 values in ((1,1),(2,2)), partition p1 values in ((6,6),(7,7))); +// and if the condition is `a=1`, then we can use `a=1` and the expression `(a in (1,2))` to locate partition `p0`. +func (l *listPartitionPruner) locatePartitionByColumn(cond *expression.ScalarFunction) (tables.ListPartitionLocation, bool, error) { + condCols := expression.ExtractColumns(cond) + if len(condCols) != 1 { + return nil, true, nil + } + var colPrune *tables.ForListColumnPruning + for _, cp := range l.listPrune.ColPrunes { + if cp.ExprCol.ID == condCols[0].ID { + colPrune = cp + } + } + if colPrune == nil { + return nil, true, nil + } + return l.locateColumnPartitionsByCondition(cond, colPrune) +} + +func (l *listPartitionPruner) locateColumnPartitionsByCondition(cond expression.Expression, colPrune *tables.ForListColumnPruning) (tables.ListPartitionLocation, bool, error) { + ranges, err := l.detachCondAndBuildRange([]expression.Expression{cond}, colPrune.ExprCol) + if err != nil { + return nil, false, err + } + + sc := l.ctx.GetSessionVars().StmtCtx + helper := tables.NewListPartitionLocationHelper() + for _, r := range ranges { + if len(r.LowVal) != 1 || len(r.HighVal) != 1 { + return nil, true, nil + } + var locations []tables.ListPartitionLocation + if r.IsPointNullable(sc) { + location, err := colPrune.LocatePartition(sc, r.HighVal[0]) + if err != nil { + return nil, false, err + } + locations = []tables.ListPartitionLocation{location} + } else { + locations, err = colPrune.LocateRanges(sc, r) + if err != nil { + return nil, false, nil + } + } + for _, location := range locations { + if len(l.partitionNames) > 0 { + for _, pg := range location { + if l.findByName(l.partitionNames, l.pi.Definitions[pg.PartIdx].Name.L) { + helper.UnionPartitionGroup(pg) + } + } + } else { + helper.Union(location) + } + } + } + return helper.GetLocation(), false, nil +} + +func (l *listPartitionPruner) detachCondAndBuildRange(conds []expression.Expression, exprCols ...*expression.Column) ([]*ranger.Range, error) { + cols := make([]*expression.Column, 0, len(exprCols)) + colLen := make([]int, 0, len(exprCols)) + for _, c := range exprCols { + c = c.Clone().(*expression.Column) + if uniqueID, ok := l.colIDToUniqueID[c.ID]; ok { + c.UniqueID = uniqueID + } + cols = append(cols, c) + colLen = append(colLen, types.UnspecifiedLength) + } + + detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(l.ctx, conds, cols, colLen) + if err != nil { + return nil, err + } + return detachedResult.Ranges, nil +} + +func (l *listPartitionPruner) findUsedListColumnsPartitions(conds []expression.Expression) (map[int]struct{}, error) { + if len(conds) == 0 { + return l.fullRange, nil + } + location, isFull, err := l.locatePartitionByCNFCondition(conds) + if err != nil { + return nil, err + } + if isFull { + return l.fullRange, nil + } + used := make(map[int]struct{}, len(location)) + for _, pg := range location { + used[pg.PartIdx] = struct{}{} + } + return used, nil +} + +func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expression) (map[int]struct{}, error) { + if len(conds) == 0 { + return l.fullRange, nil + } + exprCols := l.listPrune.PruneExprCols + pruneExpr := l.listPrune.PruneExpr + ranges, err := l.detachCondAndBuildRange(conds, exprCols...) + if err != nil { + return nil, err + } + used := make(map[int]struct{}, len(ranges)) + for _, r := range ranges { + if r.IsPointNullable(l.ctx.GetSessionVars().StmtCtx) { + if len(r.HighVal) != len(exprCols) && !r.HighVal[0].IsNull() { + // For the list partition, if the first argument is null, + // then the list partition expression should also be null. + return l.fullRange, nil + } + value, isNull, err := pruneExpr.EvalInt(l.ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) + if err != nil { + return nil, err + } + partitionIdx := l.listPrune.LocatePartition(value, isNull) + if partitionIdx == -1 { + continue + } + if len(l.partitionNames) > 0 && !l.findByName(l.partitionNames, l.pi.Definitions[partitionIdx].Name.L) { + continue + } + used[partitionIdx] = struct{}{} + } else { + return l.fullRange, nil + } + } + return used, nil +} + +func (s *partitionProcessor) findUsedListPartitions(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, + conds []expression.Expression) ([]int, error) { + pi := tbl.Meta().Partition + partExpr, err := tbl.(partitionTable).PartitionExpr() + if err != nil { + return nil, err + } + + listPruner := newListPartitionPruner(ctx, tbl, partitionNames, s, conds, partExpr.ForListPruning) + var used map[int]struct{} + if partExpr.ForListPruning.ColPrunes == nil { + used, err = listPruner.findUsedListPartitions(conds) + } else { + used, err = listPruner.findUsedListColumnsPartitions(conds) + } +>>>>>>> 0490590b0... planner,executor: fix 'select ...(join on partition table) for update' panic (#21148) if err != nil { return nil, err }