Skip to content

Commit 9008d23

Browse files
authored
*: remove ExtraPidCol and replace it with ExtraPhysTblIDCol (#53974)
close #53929
1 parent a5a1b67 commit 9008d23

29 files changed

+121
-135
lines changed

pkg/executor/builder.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -683,7 +683,7 @@ func (b *executorBuilder) buildCleanupIndex(v *plannercore.CleanupIndex) exec.Ex
683683
sessCtx := e.Ctx().GetSessionVars().StmtCtx
684684
e.handleCols = buildHandleColsForExec(sessCtx, tblInfo, e.columns)
685685
if e.index.Meta().Global {
686-
e.columns = append(e.columns, model.NewExtraPartitionIDColInfo())
686+
e.columns = append(e.columns, model.NewExtraPhysTblIDColInfo())
687687
}
688688
return e
689689
}
@@ -5133,7 +5133,7 @@ func NewRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model
51335133
}
51345134
defVal := func(i int, chk *chunk.Chunk) error {
51355135
if reqCols[i].ID < 0 {
5136-
// model.ExtraHandleID, ExtraPidColID, ExtraPhysTblID... etc
5136+
// model.ExtraHandleID, ExtraPhysTblID... etc
51375137
// Don't set the default value for that column.
51385138
chk.AppendNull(i)
51395139
return nil

pkg/executor/delete.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -130,7 +130,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error {
130130

131131
datumRow := make([]types.Datum, 0, len(fields))
132132
for i, field := range fields {
133-
if columns[i].ID == model.ExtraPidColID || columns[i].ID == model.ExtraPhysTblID {
133+
if columns[i].ID == model.ExtraPhysTblID {
134134
continue
135135
}
136136

pkg/executor/distsql.go

+2-4
Original file line numberDiff line numberDiff line change
@@ -581,24 +581,22 @@ func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize in
581581

582582
func (e *IndexLookUpExecutor) needPartitionHandle(tp getHandleType) (bool, error) {
583583
var col *expression.Column
584-
var needPartitionHandle, hasExtraCol bool
584+
var needPartitionHandle bool
585585
if tp == getHandleFromIndex {
586586
cols := e.idxPlans[0].Schema().Columns
587587
outputOffsets := e.dagPB.OutputOffsets
588588
col = cols[outputOffsets[len(outputOffsets)-1]]
589589
// For indexScan, need partitionHandle when global index or keepOrder with partitionTable
590590
needPartitionHandle = e.index.Global || e.partitionTableMode && e.keepOrder
591-
hasExtraCol = col.ID == model.ExtraPhysTblID || col.ID == model.ExtraPidColID
592591
} else {
593592
cols := e.tblPlans[0].Schema().Columns
594593
outputOffsets := e.tableRequest.OutputOffsets
595594
col = cols[outputOffsets[len(outputOffsets)-1]]
596595

597596
// For TableScan, need partitionHandle in `indexOrder` when e.keepOrder == true or execute `admin check [table|index]` with global index
598597
needPartitionHandle = ((e.index.Global || e.partitionTableMode) && e.keepOrder) || (e.index.Global && e.checkIndexValue != nil)
599-
// no ExtraPidColID here, because TableScan shouldn't contain them.
600-
hasExtraCol = col.ID == model.ExtraPhysTblID
601598
}
599+
hasExtraCol := col.ID == model.ExtraPhysTblID
602600

603601
// There will be two needPartitionHandle != hasExtraCol situations.
604602
// Only `needPartitionHandle` == true and `hasExtraCol` == false are not allowed.

pkg/executor/index_merge_reader.go

+5-5
Original file line numberDiff line numberDiff line change
@@ -628,7 +628,6 @@ func (w *partialTableWorker) needPartitionHandle() (bool, error) {
628628
col := cols[outputOffsets[len(outputOffsets)-1]]
629629

630630
needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0
631-
// no ExtraPidColID here, because a clustered index couldn't be a global index.
632631
hasExtraCol := col.ID == model.ExtraPhysTblID
633632

634633
// There will be two needPartitionHandle != hasExtraCol situations.
@@ -1698,24 +1697,25 @@ func syncErr(ctx context.Context, finished <-chan struct{}, errCh chan<- *indexM
16981697
}
16991698

17001699
// needPartitionHandle indicates whether we need create a partitionHandle or not.
1701-
// If the schema from planner part contains ExtraPidColID or ExtraPhysTblID,
1700+
// If the schema from planner part contains ExtraPhysTblID,
17021701
// we need create a partitionHandle, otherwise create a normal handle.
17031702
// In TableRowIDScan, the partitionHandle will be used to create key ranges.
17041703
func (w *partialIndexWorker) needPartitionHandle() (bool, error) {
17051704
cols := w.plan[0].Schema().Columns
17061705
outputOffsets := w.dagPB.OutputOffsets
17071706
col := cols[outputOffsets[len(outputOffsets)-1]]
17081707

1709-
needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0
1710-
hasExtraCol := col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID
1708+
is := w.plan[0].(*plannercore.PhysicalIndexScan)
1709+
needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0 || is.Index.Global
1710+
hasExtraCol := col.ID == model.ExtraPhysTblID
17111711

17121712
// There will be two needPartitionHandle != hasExtraCol situations.
17131713
// Only `needPartitionHandle` == true and `hasExtraCol` == false are not allowed.
17141714
// `ExtraPhysTblID` will be used in `SelectLock` when `needPartitionHandle` == false and `hasExtraCol` == true.
17151715
if needPartitionHandle && !hasExtraCol {
17161716
return needPartitionHandle, errors.Errorf("Internal error, needPartitionHandle != ret")
17171717
}
1718-
return needPartitionHandle || (col.ID == model.ExtraPidColID), nil
1718+
return needPartitionHandle, nil
17191719
}
17201720

17211721
func (w *partialIndexWorker) fetchHandles(

pkg/executor/partition_table_test.go

+29
Original file line numberDiff line numberDiff line change
@@ -2407,3 +2407,32 @@ func TestIssue31024(t *testing.T) {
24072407

24082408
tk2.MustExec("rollback")
24092409
}
2410+
2411+
func TestGlobalIndexWithSelectLock(t *testing.T) {
2412+
store := testkit.CreateMockStore(t)
2413+
2414+
tk1 := testkit.NewTestKit(t, store)
2415+
tk1.MustExec("set tidb_enable_global_index = true")
2416+
tk1.MustExec("use test")
2417+
tk1.MustExec("create table t(a int, b int, unique index(b), primary key(a)) partition by hash(a) partitions 5;")
2418+
tk1.MustExec("insert into t values (1,1),(2,2),(3,3),(4,4),(5,5);")
2419+
tk1.MustExec("begin")
2420+
tk1.MustExec("select * from t use index(b) where b = 2 order by b limit 1 for update;")
2421+
2422+
tk2 := testkit.NewTestKit(t, store)
2423+
tk2.MustExec("use test")
2424+
2425+
ch := make(chan int, 10)
2426+
go func() {
2427+
// Check the key is locked.
2428+
tk2.MustExec("update t set b = 6 where b = 2")
2429+
ch <- 1
2430+
}()
2431+
2432+
time.Sleep(50 * time.Millisecond)
2433+
ch <- 0
2434+
tk1.MustExec("commit")
2435+
2436+
require.Equal(t, <-ch, 0)
2437+
require.Equal(t, <-ch, 1)
2438+
}

pkg/expression/explain.go

+3-3
Original file line numberDiff line numberDiff line change
@@ -37,11 +37,11 @@ func (expr *ScalarFunction) explainInfo(ctx EvalContext, normalized bool) string
3737
intest.Assert(normalized || ctx != nil)
3838
var buffer bytes.Buffer
3939
fmt.Fprintf(&buffer, "%s(", expr.FuncName.L)
40-
// convert `in(_tidb_pid, -1)` to `in(_tidb_pid, dual)` whether normalized equals to true or false.
40+
// convert `in(_tidb_tid, -1)` to `in(_tidb_tid, dual)` whether normalized equals to true or false.
4141
if expr.FuncName.L == ast.In {
4242
args := expr.GetArgs()
43-
if len(args) == 2 && args[0].ExplainNormalizedInfo() == model.ExtraPartitionIdName.L && args[1].(*Constant).Value.GetInt64() == -1 {
44-
buffer.WriteString(model.ExtraPartitionIdName.L + ", dual)")
43+
if len(args) == 2 && strings.HasSuffix(args[0].ExplainNormalizedInfo(), model.ExtraPhysTblIdName.L) && args[1].(*Constant).Value.GetInt64() == -1 {
44+
buffer.WriteString(args[0].ExplainNormalizedInfo() + ", dual)")
4545
return buffer.String()
4646
}
4747
}

pkg/parser/model/model.go

+5-21
Original file line numberDiff line numberDiff line change
@@ -383,13 +383,12 @@ func IsIndexPrefixCovered(tbInfo *TableInfo, index *IndexInfo, cols ...CIStr) bo
383383
// for use of execution phase.
384384
const ExtraHandleID = -1
385385

386-
// ExtraPidColID is the column ID of column which store the partitionID decoded in global index values.
387-
const ExtraPidColID = -2
386+
// Deprecated: Use ExtraPhysTblID instead.
387+
// const ExtraPidColID = -2
388388

389389
// ExtraPhysTblID is the column ID of column that should be filled in with the physical table id.
390390
// Primarily used for table partition dynamic prune mode, to return which partition (physical table id) the row came from.
391-
// Using a dedicated id for this, since in the future ExtraPidColID and ExtraPhysTblID may be used for the same request.
392-
// Must be after ExtraPidColID!
391+
// If used with a global index, the partition ID decoded from the key value will be filled in.
393392
const ExtraPhysTblID = -3
394393

395394
// ExtraRowChecksumID is the column ID of column which holds the row checksum info.
@@ -435,8 +434,8 @@ const (
435434
// ExtraHandleName is the name of ExtraHandle Column.
436435
var ExtraHandleName = NewCIStr("_tidb_rowid")
437436

438-
// ExtraPartitionIdName is the name of ExtraPartitionId Column.
439-
var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive
437+
// Deprecated: Use ExtraPhysTblIdName instead.
438+
// var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive
440439

441440
// ExtraPhysTblIdName is the name of ExtraPhysTblID Column.
442441
var ExtraPhysTblIdName = NewCIStr("_tidb_tid") //nolint:revive
@@ -923,21 +922,6 @@ func NewExtraHandleColInfo() *ColumnInfo {
923922
return colInfo
924923
}
925924

926-
// NewExtraPartitionIDColInfo mocks a column info for extra partition id column.
927-
func NewExtraPartitionIDColInfo() *ColumnInfo {
928-
colInfo := &ColumnInfo{
929-
ID: ExtraPidColID,
930-
Name: ExtraPartitionIdName,
931-
}
932-
colInfo.SetType(mysql.TypeLonglong)
933-
flen, decimal := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong)
934-
colInfo.SetFlen(flen)
935-
colInfo.SetDecimal(decimal)
936-
colInfo.SetCharset(charset.CharsetBin)
937-
colInfo.SetCollate(charset.CollationBin)
938-
return colInfo
939-
}
940-
941925
// NewExtraPhysTblIDColInfo mocks a column info for extra partition id column.
942926
func NewExtraPhysTblIDColInfo() *ColumnInfo {
943927
colInfo := &ColumnInfo{

pkg/planner/core/find_best_task.go

+17-21
Original file line numberDiff line numberDiff line change
@@ -1423,15 +1423,6 @@ func (ds *DataSource) FindBestTask(prop *property.PhysicalProperty, planCounter
14231423
}
14241424
}
14251425
if canConvertPointGet {
1426-
// If the schema contains ExtraPidColID, do not convert to point get.
1427-
// Because the point get executor can not handle the extra partition ID column now.
1428-
// I.e. Global Index is used
1429-
for _, col := range ds.schema.Columns {
1430-
if col.ID == model.ExtraPidColID {
1431-
canConvertPointGet = false
1432-
break
1433-
}
1434-
}
14351426
if path != nil && path.Index != nil && path.Index.Global {
14361427
// Don't convert to point get during ddl
14371428
// TODO: Revisit truncate partition and global index
@@ -2148,6 +2139,15 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub
21482139
}
21492140
}
21502141

2142+
var extraPhysTblCol *expression.Column
2143+
// If `dataSouceSchema` contains `model.ExtraPhysTblID`, we should add it into `indexScan.schema`
2144+
for _, col := range is.dataSourceSchema.Columns {
2145+
if col.ID == model.ExtraPhysTblID {
2146+
extraPhysTblCol = col.Clone().(*expression.Column)
2147+
break
2148+
}
2149+
}
2150+
21512151
if isDoubleRead || is.Index.Global {
21522152
// If it's double read case, the first index must return handle. So we should add extra handle column
21532153
// if there isn't a handle column.
@@ -2161,23 +2161,19 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub
21612161
})
21622162
}
21632163
}
2164-
// If it's global index, handle and PidColID columns has to be added, so that needed pids can be filtered.
2165-
if is.Index.Global {
2164+
// If it's global index, handle and PhysTblID columns has to be added, so that needed pids can be filtered.
2165+
if is.Index.Global && extraPhysTblCol == nil {
21662166
indexCols = append(indexCols, &expression.Column{
21672167
RetType: types.NewFieldType(mysql.TypeLonglong),
2168-
ID: model.ExtraPidColID,
2168+
ID: model.ExtraPhysTblID,
21692169
UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(),
2170-
OrigName: model.ExtraPartitionIdName.O,
2170+
OrigName: model.ExtraPhysTblIdName.O,
21712171
})
21722172
}
21732173
}
21742174

2175-
// If `dataSouceSchema` contains `model.ExtraPhysTblID`, we should add it into `indexScan.schema`
2176-
for _, col := range is.dataSourceSchema.Columns {
2177-
if col.ID == model.ExtraPhysTblID {
2178-
indexCols = append(indexCols, col.Clone().(*expression.Column))
2179-
break
2180-
}
2175+
if extraPhysTblCol != nil {
2176+
indexCols = append(indexCols, extraPhysTblCol)
21812177
}
21822178

21832179
is.SetSchema(expression.NewSchema(indexCols...))
@@ -2189,14 +2185,14 @@ func (is *PhysicalIndexScan) addSelectionConditionForGlobalIndex(p *DataSource,
21892185
}
21902186
args := make([]expression.Expression, 0, len(p.partitionNames)+1)
21912187
for _, col := range is.schema.Columns {
2192-
if col.ID == model.ExtraPidColID {
2188+
if col.ID == model.ExtraPhysTblID {
21932189
args = append(args, col.Clone())
21942190
break
21952191
}
21962192
}
21972193

21982194
if len(args) != 1 {
2199-
return nil, errors.Errorf("Can't find column %s in schema %s", model.ExtraPartitionIdName.O, is.schema)
2195+
return nil, errors.Errorf("Can't find column %s in schema %s", model.ExtraPhysTblIdName.O, is.schema)
22002196
}
22012197

22022198
// For SQL like 'select x from t partition(p0, p1) use index(idx)',

pkg/planner/core/logical_plan_builder.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -3892,7 +3892,7 @@ func unfoldWildStar(field *ast.SelectField, outputName types.NameSlice, column [
38923892
}
38933893
if (dbName.L == "" || dbName.L == name.DBName.L) &&
38943894
(tblName.L == "" || tblName.L == name.TblName.L) &&
3895-
col.ID != model.ExtraHandleID && col.ID != model.ExtraPidColID && col.ID != model.ExtraPhysTblID {
3895+
col.ID != model.ExtraHandleID && col.ID != model.ExtraPhysTblID {
38963896
colName := &ast.ColumnNameExpr{
38973897
Name: &ast.ColumnName{
38983898
Schema: name.DBName,

pkg/planner/core/plan_to_pb.go

-2
Original file line numberDiff line numberDiff line change
@@ -465,8 +465,6 @@ func (p *PhysicalIndexScan) ToPB(_ *base.BuildPBContext, _ kv.StoreType) (*tipb.
465465
columns = append(columns, model.NewExtraHandleColInfo())
466466
} else if col.ID == model.ExtraPhysTblID {
467467
columns = append(columns, model.NewExtraPhysTblIDColInfo())
468-
} else if col.ID == model.ExtraPidColID {
469-
columns = append(columns, model.NewExtraPartitionIDColInfo())
470468
} else {
471469
columns = append(columns, FindColumnInfoByID(tableColumns, col.ID))
472470
}

pkg/planner/core/rule_column_pruning.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -356,7 +356,7 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt
356356
parentUsedCols = append(parentUsedCols, p.handleCols.GetCol(i))
357357
}
358358
for _, col := range p.Schema().Columns {
359-
if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID {
359+
if col.ID == model.ExtraPhysTblID {
360360
parentUsedCols = append(parentUsedCols, col)
361361
}
362362
}

pkg/planner/core/rule_partition_processor.go

-9
Original file line numberDiff line numberDiff line change
@@ -471,15 +471,6 @@ func (*partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types.Fi
471471
})
472472
continue
473473
}
474-
if colExpr.ID == model.ExtraPidColID {
475-
names = append(names, &types.FieldName{
476-
DBName: ds.DBName,
477-
TblName: ds.tableInfo.Name,
478-
ColName: model.ExtraPartitionIdName,
479-
OrigColName: model.ExtraPartitionIdName,
480-
})
481-
continue
482-
}
483474
if colExpr.ID == model.ExtraPhysTblID {
484475
names = append(names, &types.FieldName{
485476
DBName: ds.DBName,

pkg/store/mockstore/unistore/cophandler/closure_exec.go

+3-8
Original file line numberDiff line numberDiff line change
@@ -305,13 +305,6 @@ func (e *closureExecutor) initIdxScanCtx(idxScan *tipb.IndexScan) {
305305
lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1]
306306
}
307307

308-
// Here it is required that ExtraPidColID
309-
// is after all other columns except ExtraPhysTblID
310-
if lastColumn.GetColumnId() == model.ExtraPidColID {
311-
e.idxScanCtx.columnLen--
312-
lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1]
313-
}
314-
315308
if len(e.idxScanCtx.primaryColumnIds) == 0 {
316309
if lastColumn.GetPkHandle() {
317310
if mysql.HasUnsignedFlag(uint(lastColumn.GetFlag())) {
@@ -932,7 +925,9 @@ func (e *closureExecutor) indexScanProcessCore(key, value []byte) error {
932925
}
933926
// Add ExtraPhysTblID if requested
934927
// Assumes it is always last!
935-
if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID {
928+
// If we need pid, it already filled by above loop. Because `DecodeIndexKV` func will return pid in `values`.
929+
// The following if statement is to fill in the tid when we needed it.
930+
if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID && len(e.columnInfos) >= len(values) {
936931
tblID := tablecodec.DecodeTableID(key)
937932
chk.AppendInt64(len(e.columnInfos)-1, tblID)
938933
}

pkg/store/mockstore/unistore/cophandler/mpp.go

-4
Original file line numberDiff line numberDiff line change
@@ -138,10 +138,6 @@ func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error
138138
*physTblIDColIdx = numIdxCols
139139
lastCol = pb.Columns[numIdxCols-1]
140140
}
141-
if lastCol.GetColumnId() == model.ExtraPidColID {
142-
numIdxCols--
143-
lastCol = pb.Columns[numIdxCols-1]
144-
}
145141

146142
hdlStatus := tablecodec.HandleDefault
147143
if len(primaryColIds) == 0 {

pkg/store/mockstore/unistore/cophandler/mpp_exec.go

+4-1
Original file line numberDiff line numberDiff line change
@@ -313,7 +313,10 @@ func (e *indexScanExec) Process(key, value []byte) error {
313313
}
314314
}
315315
}
316-
if e.physTblIDColIdx != nil {
316+
317+
// If we need pid, it already filled by above loop. Because `DecodeIndexKV` func will return pid in `values`.
318+
// The following if statement is to fill in the tid when we needed it.
319+
if e.physTblIDColIdx != nil && *e.physTblIDColIdx >= len(values) {
317320
tblID := tablecodec.DecodeTableID(key)
318321
e.chk.AppendInt64(*e.physTblIDColIdx, tblID)
319322
}

tests/integrationtest/r/globalindex/aggregate.result

+2-2
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ id estRows task access object operator info
1919
HashAgg 1.00 root NULL funcs:count(Column#9)->Column#4, funcs:max(Column#10)->Column#5, funcs:min(Column#11)->Column#6
2020
└─IndexReader 1.00 root partition:p0 index:HashAgg
2121
└─HashAgg 1.00 cop[tikv] NULL funcs:count(1)->Column#9, funcs:max(globalindex__aggregate.p.id)->Column#10, funcs:min(globalindex__aggregate.p.id)->Column#11
22-
└─Selection 10000.00 cop[tikv] NULL in(_tidb_pid, pid0)
22+
└─Selection 10000.00 cop[tikv] NULL in(_tidb_tid, tid0)
2323
└─IndexFullScan 10000.00 cop[tikv] table:p, index:idx(id) keep order:false, stats:pseudo
2424
select count(*), max(id), min(id) from p partition(p0) use index(idx);
2525
count(*) max(id) min(id)
@@ -41,7 +41,7 @@ explain format='brief' select avg(id), max(id), min(id) from p partition(p0) use
4141
id estRows task access object operator info
4242
HashAgg 8000.00 root NULL group by:globalindex__aggregate.p.c, funcs:avg(Column#9, Column#10)->Column#4, funcs:max(Column#11)->Column#5, funcs:min(Column#12)->Column#6
4343
└─IndexLookUp 8000.00 root partition:p0 NULL
44-
├─Selection(Build) 10000.00 cop[tikv] NULL in(_tidb_pid, pid0)
44+
├─Selection(Build) 10000.00 cop[tikv] NULL in(_tidb_tid, tid0)
4545
│ └─IndexFullScan 10000.00 cop[tikv] table:p, index:idx(id) keep order:false, stats:pseudo
4646
└─HashAgg(Probe) 8000.00 cop[tikv] NULL group by:globalindex__aggregate.p.c, funcs:count(globalindex__aggregate.p.id)->Column#9, funcs:sum(globalindex__aggregate.p.id)->Column#10, funcs:max(globalindex__aggregate.p.id)->Column#11, funcs:min(globalindex__aggregate.p.id)->Column#12
4747
└─TableRowIDScan 10000.00 cop[tikv] table:p keep order:false, stats:pseudo

tests/integrationtest/r/globalindex/expression_index.result

+1-1
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ explain format='brief' select * from t partition(p0) use index(idx) where lower(
3131
id estRows task access object operator info
3232
Projection 3333.33 root NULL globalindex__expression_index.t.a, globalindex__expression_index.t.b
3333
└─IndexLookUp 3333.33 root partition:p0 NULL
34-
├─Selection(Build) 3333.33 cop[tikv] NULL in(_tidb_pid, pid0)
34+
├─Selection(Build) 3333.33 cop[tikv] NULL in(_tidb_tid, tid0)
3535
│ └─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(lower(`b`)) range:("c",+inf], keep order:false, stats:pseudo
3636
└─TableRowIDScan(Probe) 3333.33 cop[tikv] table:t keep order:false, stats:pseudo
3737
select * from t partition(p0) use index(idx) where lower(b) > 'c';

0 commit comments

Comments
 (0)