*: support for partition table and common handle for index lookup push down (#64839)

ref pingcap/tidb#62575, close pingcap/tidb#64594, close pingcap/tidb#64595
This commit is contained in:
Chao Wang
2025-12-04 17:07:23 +08:00
committed by GitHub
parent c5cd3e6a16
commit 5ee17f4a1d
14 changed files with 1019 additions and 244 deletions

View File

@ -415,6 +415,13 @@ func (r *selectResult) fetchRespWithIntermediateResults(ctx context.Context, int
return errors.Trace(err)
}
respSize := int64(r.selectResp.Size())
atomic.StoreInt64(&r.selectRespSize, respSize)
r.memConsume(respSize)
if err := r.selectResp.Error; err != nil {
return dbterror.ClassTiKV.Synthesize(terror.ErrCode(err.Code), err.Msg)
}
if len(r.selectResp.IntermediateOutputs) != len(intermediateOutputTypes) {
return errors.Errorf(
"The length of intermediate output types %d mismatches the length of got intermediate outputs %d."+
@ -422,14 +429,8 @@ func (r *selectResult) fetchRespWithIntermediateResults(ctx context.Context, int
len(intermediateOutputTypes), len(r.selectResp.IntermediateOutputs),
)
}
r.intermediateOutputTypes = intermediateOutputTypes
respSize := int64(r.selectResp.Size())
atomic.StoreInt64(&r.selectRespSize, respSize)
r.memConsume(respSize)
if err := r.selectResp.Error; err != nil {
return dbterror.ClassTiKV.Synthesize(terror.ErrCode(err.Code), err.Msg)
}
if err = r.ctx.SQLKiller.HandleSignal(); err != nil {
return err
}

View File

@ -738,7 +738,9 @@ func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize in
func (e *IndexLookUpExecutor) needPartitionHandle(tp getHandleType) (bool, error) {
if e.indexLookUpPushDown {
// For index lookup push down, partition table is not supported
// For index lookup push down, needPartitionHandle should always return false because
// global index or keep order for partition table is not supported now.
intest.Assert(!e.index.Global && !e.keepOrder)
return false, nil
}
@ -817,8 +819,21 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, initBatchSiz
e.byItems = nil
}
var tps []*types.FieldType
tblScanIdxForRewritePartitionID := -1
if e.indexLookUpPushDown {
tps = e.RetFieldTypes()
if e.partitionTableMode {
for idx, executor := range e.dagPB.Executors {
if executor.Tp == tipb.ExecType_TypeTableScan {
tblScanIdxForRewritePartitionID = idx
break
}
}
if tblScanIdxForRewritePartitionID < 0 {
intest.Assert(false)
return errors.New("cannot find table scan executor in for partition index lookup push down")
}
}
} else {
tps = e.getRetTpsForIndexReader()
}
@ -840,7 +855,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, initBatchSiz
worker.batchSize = e.calculateBatchSize(initBatchSize, worker.maxBatchSize)
results := make([]distsql.SelectResult, 0, len(kvRanges))
for _, kvRange := range kvRanges {
for idx, kvRange := range kvRanges {
// check if executor is closed
finished := false
select {
@ -851,6 +866,13 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, initBatchSiz
if finished {
break
}
if tblScanIdxForRewritePartitionID >= 0 {
// We should set the TblScan's TableID to the partition physical ID to make sure
// the push-down index lookup can encode the table handle key correctly.
e.dagPB.Executors[tblScanIdxForRewritePartitionID].TblScan.TableId = e.prunedPartitions[idx].GetPhysicalID()
}
var builder distsql.RequestBuilder
builder.SetDAGRequest(e.dagPB).
SetStartTS(e.startTS).
@ -1237,7 +1259,8 @@ func (w *indexWorker) fetchHandles(ctx context.Context, results selectResultList
}
taskID := 0
for i := 0; i < len(results); {
result := results[i]
curResultIdx := i
result := results[curResultIdx]
if w.PushedLimit != nil && w.scannedKeys >= w.PushedLimit.Count+w.PushedLimit.Offset {
break
}
@ -1284,7 +1307,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, results selectResultList
}
tableLookUpTask = w.buildTableTask(taskID, handles, retChunk)
if w.idxLookup.partitionTableMode {
tableLookUpTask.partitionTable = w.idxLookup.prunedPartitions[i]
tableLookUpTask.partitionTable = w.idxLookup.prunedPartitions[curResultIdx]
}
taskID++
}

View File

@ -17,6 +17,7 @@ package executor_test
import (
"context"
"fmt"
"hash/fnv"
"math/rand"
"strconv"
"strings"
@ -765,6 +766,166 @@ func TestBuildProjectionForIndexJoinPanic(t *testing.T) {
require.ErrorContains(t, err, "buildProjectionForIndexJoinPanic")
}
type IndexLookUpPushDownRunVerifier struct {
*testing.T
tk *testkit.TestKit
tableName string
indexName string
primaryRows []int
hitRate any
msg string
}
type RunSelectWithCheckResult struct {
SQL string
Rows [][]any
AnalyzeRows [][]any
}
func (t *IndexLookUpPushDownRunVerifier) RunSelectWithCheck(where string, skip, limit int) RunSelectWithCheckResult {
require.NotNil(t, t.tk)
require.NotEmpty(t, t.tableName)
require.NotEmpty(t, t.indexName)
require.NotEmpty(t, t.primaryRows)
require.GreaterOrEqual(t, skip, 0)
if skip > 0 {
require.GreaterOrEqual(t, limit, 0)
}
var hitRate int
if r, ok := t.hitRate.(*rand.Rand); ok {
hitRate = r.Intn(11)
} else {
hitRate, ok = t.hitRate.(int)
require.True(t, ok)
}
message := fmt.Sprintf("%s, hitRate: %d, where: %s, limit: %d", t.msg, hitRate, where, limit)
injectHandleFilter := func(h kv.Handle) bool {
if hitRate >= 10 {
return true
}
h64a := fnv.New64a()
_, err := h64a.Write(h.Encoded())
require.NoError(t, err)
return h64a.Sum64()%10 < uint64(hitRate)
}
var injectCalled atomic.Bool
require.NoError(t, failpoint.EnableCall("github.com/pingcap/tidb/pkg/store/mockstore/unistore/cophandler/inject-index-lookup-handle-filter", func(f *func(kv.Handle) bool) {
*f = injectHandleFilter
injectCalled.Store(true)
}))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/mockstore/unistore/cophandler/inject-index-lookup-handle-filter"))
}()
var sb strings.Builder
sb.WriteString(fmt.Sprintf("select /*+ index_lookup_pushdown(%s, %s)*/ * from %s where ", t.tableName, t.indexName, t.tableName))
sb.WriteString(where)
if skip > 0 {
sb.WriteString(fmt.Sprintf(" limit %d, %d", skip, limit))
} else if limit >= 0 {
sb.WriteString(fmt.Sprintf(" limit %d", limit))
}
// make sure the query uses index lookup
analyzeSQL := "explain analyze " + sb.String()
injectCalled.Store(false)
analyzeResult := t.tk.MustQuery(analyzeSQL)
require.True(t, injectCalled.Load(), message)
require.Contains(t, analyzeResult.String(), "LocalIndexLookUp", analyzeSQL+"\n"+analyzeResult.String())
// get actual result
injectCalled.Store(false)
rs := t.tk.MustQuery(sb.String())
actual := rs.Rows()
require.True(t, injectCalled.Load(), message)
idSets := make(map[string]struct{}, len(actual))
for _, row := range actual {
var primaryKey strings.Builder
require.Greater(t, len(t.primaryRows), 0)
for i, idx := range t.primaryRows {
if i > 0 {
primaryKey.WriteString("#")
}
primaryKey.WriteString(row[idx].(string))
}
id := primaryKey.String()
_, dup := idSets[id]
require.False(t, dup, "dupID: "+id+", "+message)
idSets[row[0].(string)] = struct{}{}
}
// use table scan
matchCondList := t.tk.MustQuery(fmt.Sprintf("select /*+ use_index(%s) */* from %s where "+where, t.tableName, t.tableName)).Rows()
if limit == 0 || skip >= len(matchCondList) {
require.Len(t, actual, 0, message)
} else if limit < 0 {
// no limit two results should have same members
require.ElementsMatch(t, matchCondList, actual, message)
} else {
expectRowCnt := limit
if skip+limit > len(matchCondList) {
expectRowCnt = len(matchCondList) - skip
}
require.Len(t, actual, expectRowCnt, message)
require.Subset(t, matchCondList, actual, message)
}
// check in analyze the index is lookup locally
message = fmt.Sprintf("%s\n%s\n%s", message, analyzeSQL, analyzeResult.String())
analyzeVerified := false
localIndexLookUpIndex := -1
totalIndexScanCnt := -1
localIndexLookUpRowCnt := -1
analyzeRows := analyzeResult.Rows()
metTableRowIDScan := false
for i, row := range analyzeRows {
if strings.Contains(row[0].(string), "LocalIndexLookUp") {
localIndexLookUpIndex = i
continue
}
if strings.Contains(row[0].(string), "TableRowIDScan") && strings.Contains(row[3].(string), "cop[tikv]") {
var err error
if !metTableRowIDScan {
localIndexLookUpRowCnt, err = strconv.Atoi(row[2].(string))
require.NoError(t, err, message)
require.GreaterOrEqual(t, localIndexLookUpRowCnt, 0)
if hitRate == 0 {
require.Zero(t, localIndexLookUpRowCnt, message)
}
// check actRows for LocalIndexLookUp
require.Equal(t, analyzeRows[localIndexLookUpIndex][2], row[2], message)
// get index scan row count
totalIndexScanCnt, err = strconv.Atoi(analyzeRows[localIndexLookUpIndex+1][2].(string))
require.NoError(t, err, message)
require.GreaterOrEqual(t, totalIndexScanCnt, localIndexLookUpRowCnt)
if hitRate >= 10 {
require.Equal(t, localIndexLookUpRowCnt, totalIndexScanCnt)
}
metTableRowIDScan = true
continue
}
tidbIndexLookUpRowCnt, err := strconv.Atoi(row[2].(string))
require.NoError(t, err, message)
if limit < 0 {
require.Equal(t, totalIndexScanCnt, localIndexLookUpRowCnt+tidbIndexLookUpRowCnt, message)
} else {
require.LessOrEqual(t, localIndexLookUpRowCnt+tidbIndexLookUpRowCnt, totalIndexScanCnt, message)
}
analyzeVerified = true
break
}
}
require.True(t, analyzeVerified, analyzeResult.String())
return RunSelectWithCheckResult{
SQL: sb.String(),
Rows: actual,
AnalyzeRows: analyzeRows,
}
}
func TestIndexLookUpPushDownExec(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
@ -773,6 +934,16 @@ func TestIndexLookUpPushDownExec(t *testing.T) {
seed := time.Now().UnixNano()
logutil.BgLogger().Info("Run TestIndexLookUpPushDownExec with seed", zap.Int64("seed", seed))
r := rand.New(rand.NewSource(seed))
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: "t",
indexName: "a",
primaryRows: []int{0},
hitRate: r,
msg: fmt.Sprintf("seed: %d", seed),
}
batch := 100
total := batch * 20
indexValEnd := 100
@ -787,94 +958,110 @@ func TestIndexLookUpPushDownExec(t *testing.T) {
tk.MustExec("insert into t values " + strings.Join(values, ","))
}
runSelectWithCheck := func(where string, skip, limit int) {
require.GreaterOrEqual(t, skip, 0)
if skip > 0 {
require.GreaterOrEqual(t, limit, 0)
v.RunSelectWithCheck("1", 0, -1)
v.RunSelectWithCheck("1", 0, r.Intn(total*2))
v.RunSelectWithCheck("1", total/2, r.Intn(total))
v.RunSelectWithCheck("1", total-10, 20)
v.RunSelectWithCheck("1", total, 10)
v.RunSelectWithCheck("1", 10, 0)
v.RunSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, 25)
v.RunSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, r.Intn(100)+1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, r.Intn(100)+1)
start := randIndexVal()
v.RunSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, -1)
start = randIndexVal()
v.RunSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, r.Intn(50)+1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, r.Intn(50)+1)
}
func TestIndexLookUpPushDownPartitionExec(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// int handle
tk.MustExec("create table tp1 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(b) CLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE (b) (\n" +
" PARTITION p0 VALUES LESS THAN (100),\n" +
" PARTITION p1 VALUES LESS THAN (200),\n" +
" PARTITION p2 VALUES LESS THAN (300),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
// common handle
tk.MustExec("create table tp2 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(a, b) CLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE COLUMNS (a) (\n" +
" PARTITION p0 VALUES LESS THAN ('c'),\n" +
" PARTITION p1 VALUES LESS THAN ('e'),\n" +
" PARTITION p2 VALUES LESS THAN ('g'),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
// extra handle
tk.MustExec("create table tp3 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(a, b) NONCLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE COLUMNS (a) (\n" +
" PARTITION p0 VALUES LESS THAN ('c'),\n" +
" PARTITION p1 VALUES LESS THAN ('e'),\n" +
" PARTITION p2 VALUES LESS THAN ('g'),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
tableNames := []string{"tp1", "tp2", "tp3"}
// prepare data
for _, tableName := range tableNames {
tk.MustExec("insert into " + tableName + " values " +
"('a', 10, 1, 100), " +
"('b', 20, 2, 200), " +
"('c', 110, 3, 300), " +
"('d', 120, 4, 400), " +
"('e', 210, 5, 500), " +
"('f', 220, 6, 600), " +
"('g', 330, 5, 700), " +
"('h', 340, 5, 800), " +
"('i', 450, 5, 900), " +
"('j', 550, 6, 1000) ",
)
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: tableName,
indexName: "c",
primaryRows: []int{0, 1},
msg: tableName,
}
hitRate := r.Intn(11)
message := fmt.Sprintf("seed: %d, hitRate: %d, where: %s, limit: %d", seed, hitRate, where, limit)
filterMap := make([]bool, total)
for i := 0; i < total; i++ {
filterMap[i] = r.Intn(10) < hitRate
if tableName == "tp1" {
v.primaryRows = []int{1}
}
injectHandleFilter := func(h kv.Handle) bool {
return filterMap[h.IntValue()]
}
var injectCalled atomic.Bool
require.NoError(t, failpoint.EnableCall("github.com/pingcap/tidb/pkg/store/mockstore/unistore/cophandler/inject-index-lookup-handle-filter", func(f *func(kv.Handle) bool) {
*f = injectHandleFilter
injectCalled.Store(true)
}))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/mockstore/unistore/cophandler/inject-index-lookup-handle-filter"))
}()
// use index lookup push down
injectCalled.Store(false)
var sb strings.Builder
sb.WriteString("select /*+ index_lookup_pushdown(t, a)*/ * from t where ")
sb.WriteString(where)
if skip > 0 {
sb.WriteString(fmt.Sprintf(" limit %d, %d", skip, limit))
} else if limit >= 0 {
sb.WriteString(fmt.Sprintf(" limit %d", limit))
}
// make sure the query uses index lookup
analyzeResult := tk.MustQuery("explain analyze " + sb.String())
require.Contains(t, analyzeResult.String(), "LocalIndexLookUp", analyzeResult.String())
// get actual result
rs := tk.MustQuery(sb.String())
require.True(t, injectCalled.Load(), message)
actual := rs.Rows()
idSets := make(map[string]struct{}, len(actual))
for _, row := range actual {
id := row[0].(string)
_, dup := idSets[id]
require.False(t, dup, "dupID: "+id+", "+message)
idSets[row[0].(string)] = struct{}{}
}
// use table scan
injectCalled.Store(false)
matchCondList := tk.MustQuery("select /*+ use_index(t) */* from t where " + where + " order by id").Rows()
require.False(t, injectCalled.Load(), message)
if limit == 0 || skip >= len(matchCondList) {
require.Len(t, actual, 0, message)
} else if limit < 0 {
// no limit two results should have same members
require.ElementsMatch(t, matchCondList, actual, message)
} else {
expectRowCnt := limit
if skip+limit > len(matchCondList) {
expectRowCnt = len(matchCondList) - skip
}
require.Len(t, actual, expectRowCnt, message)
require.Subset(t, matchCondList, actual, message)
for _, hitRate := range []int{0, 5, 10} {
v.hitRate = hitRate
v.RunSelectWithCheck("1", 0, -1)
}
}
runSelectWithCheck("1", 0, -1)
runSelectWithCheck("1", 0, r.Intn(total*2))
runSelectWithCheck("1", total/2, r.Intn(total))
runSelectWithCheck("1", total-10, 20)
runSelectWithCheck("1", total, 10)
runSelectWithCheck("1", 10, 0)
runSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, 25)
runSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, r.Intn(100)+1)
runSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, r.Intn(100)+1)
start := randIndexVal()
runSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, -1)
start = randIndexVal()
runSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, r.Intn(50)+1)
runSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, r.Intn(50)+1)
}

View File

@ -45,7 +45,7 @@ func resetPlanIDRecursively(ctx base.PlanContext, p base.PhysicalPlan) {
}
func buildPushDownIndexLookUpPlan(
ctx base.PlanContext, indexPlan base.PhysicalPlan, tablePlan base.PhysicalPlan,
ctx base.PlanContext, indexPlan base.PhysicalPlan, tablePlan base.PhysicalPlan, isCommonHandle bool,
) (indexLookUpPlan base.PhysicalPlan, err error) {
tablePlan, err = tablePlan.Clone(ctx)
if err != nil {
@ -53,10 +53,17 @@ func buildPushDownIndexLookUpPlan(
}
resetPlanIDRecursively(ctx, tablePlan)
var indexHandleOffsets []uint32
if !isCommonHandle {
// - If common handle, we don't need to set the indexHandleOffsets to build the common handle key
// which can be read from the index value directly.
// - If int handle, it is the last column in the index schema.
indexHandleOffsets = []uint32{uint32(indexPlan.Schema().Len()) - 1}
}
tableScanPlan, parentOfTableScan := detachRootTableScanPlan(tablePlan)
indexLookUpPlan = PhysicalLocalIndexLookUp{
// Only int handle is supported now, so the handle is always the last column of index schema.
IndexHandleOffsets: []uint32{uint32(indexPlan.Schema().Len()) - 1},
IndexHandleOffsets: indexHandleOffsets,
}.Init(ctx, indexPlan, tableScanPlan, tablePlan.QueryBlockOffset())
if parentOfTableScan != nil {

View File

@ -223,7 +223,7 @@ func (p *PhysicalIndexLookUpReader) tryPushDownLookUp(ctx base.PlanContext) {
return
}
indexLookUpPlan, err := buildPushDownIndexLookUpPlan(ctx, p.IndexPlan, p.TablePlan)
indexLookUpPlan, err := buildPushDownIndexLookUpPlan(ctx, p.IndexPlan, p.TablePlan, len(p.CommonHandleCols) > 0)
if err != nil {
// This should not happen, but if it happens, we just log a warning and continue to use the original plan.
intest.AssertNoError(err)

View File

@ -1185,10 +1185,10 @@ func isTiKVIndexByName(idxName ast.CIStr, indexInfo *model.IndexInfo, tblInfo *m
func checkIndexLookUpPushDownSupported(ctx base.PlanContext, tblInfo *model.TableInfo, index *model.IndexInfo) bool {
unSupportedReason := ""
sessionVars := ctx.GetSessionVars()
if tblInfo.IsCommonHandle {
unSupportedReason = "common handle table is not supported"
} else if tblInfo.Partition != nil {
unSupportedReason = "partition table is not supported"
if tblInfo.IsCommonHandle && tblInfo.CommonHandleVersion < 1 {
unSupportedReason = "common handle table with old encoding version is not supported"
} else if index.Global {
unSupportedReason = "the global index in partition table is not supported"
} else if tblInfo.TempTableType != model.TempTableNone {
unSupportedReason = "temporary table is not supported"
} else if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable {

View File

@ -64,6 +64,9 @@ type mppExecBuilder struct {
ndvs []int64
paging *coprocessor.KeyRange
pagingSize uint64
// is building the index lookup
inBuildIndexLookUp bool
}
func (b *mppExecBuilder) buildMPPTableScan(pb *tipb.TableScan) (*tableScanExec, error) {
@ -174,31 +177,45 @@ func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error
})
}
var commonHandleKeyIdx *int
if b.inBuildIndexLookUp && len(pb.PrimaryColumnIds) > 0 {
// For index scan in index lookup, if the table has common handle, we need to
// return the common handle key to indexLookUpExec
idx := len(fieldTypes)
commonHandleKeyIdx = &idx
fieldTypes = append(fieldTypes, types.NewFieldType(mysql.TypeVarchar))
}
var prevVals [][]byte
if b.dagReq.GetCollectRangeCounts() {
prevVals = make([][]byte, numIdxCols)
}
idxScan := &indexScanExec{
baseMPPExec: baseMPPExec{sctx: b.sctx, fieldTypes: fieldTypes},
startTS: b.dagCtx.startTS,
kvRanges: ranges,
dbReader: b.dbReader,
lockStore: b.dagCtx.lockStore,
resolvedLocks: b.dagCtx.resolvedLocks,
counts: b.counts,
ndvs: b.ndvs,
prevVals: prevVals,
colInfos: colInfos,
numIdxCols: numIdxCols,
hdlStatus: hdlStatus,
desc: pb.Desc,
physTblIDColIdx: physTblIDColIdx,
paging: b.paging,
baseMPPExec: baseMPPExec{sctx: b.sctx, fieldTypes: fieldTypes},
startTS: b.dagCtx.startTS,
kvRanges: ranges,
dbReader: b.dbReader,
lockStore: b.dagCtx.lockStore,
resolvedLocks: b.dagCtx.resolvedLocks,
counts: b.counts,
ndvs: b.ndvs,
prevVals: prevVals,
colInfos: colInfos,
numIdxCols: numIdxCols,
hdlStatus: hdlStatus,
desc: pb.Desc,
physTblIDColIdx: physTblIDColIdx,
commonHandleKeyIdx: commonHandleKeyIdx,
paging: b.paging,
}
return idxScan, nil
}
func (b *mppExecBuilder) buildIndexLookUp(pb *tipb.IndexLookUp) (*indexLookUpExec, error) {
b.inBuildIndexLookUp = true
defer func() {
b.inBuildIndexLookUp = false
}()
if len(pb.Children) != 2 {
return nil, errors.New("IndexLookUp should have two children")
}
@ -212,10 +229,6 @@ func (b *mppExecBuilder) buildIndexLookUp(pb *tipb.IndexLookUp) (*indexLookUpExe
}
tblScanPB := pb.Children[1].TblScan
if len(tblScanPB.PrimaryColumnIds) > 0 {
return nil, errors.New("common handle not supported")
}
indexScanChild, err := b.buildMPPExecutor(pb.Children[0])
if err != nil {
return nil, err
@ -244,7 +257,7 @@ func (b *mppExecBuilder) buildIndexLookUp(pb *tipb.IndexLookUp) (*indexLookUpExe
keyspaceID: b.dagCtx.keyspaceID,
indexHandleOffsets: pb.IndexHandleOffsets,
tblScanPB: tblScanPB,
isCommonHandle: false,
isCommonHandle: len(tblScanPB.PrimaryColumnIds) > 0,
extraReaderProvider: b.dbReader.ExtraDbReaderProvider,
buildTableScan: func(reader *dbreader.DBReader, ranges []kv.KeyRange) (*tableScanExec, error) {
copRanges := make([]*coprocessor.KeyRange, len(ranges))

View File

@ -296,6 +296,8 @@ type indexScanExec struct {
// if ExtraPhysTblIDCol is requested, fill in the physical table id in this column position
physTblIDColIdx *int
// if common handle key is requested, fill the common handle in this column
commonHandleKeyIdx *int
// This is used to update the paging range result, updated in next().
paging *coprocessor.KeyRange
chunkLastProcessedKeys []kv.Key
@ -352,6 +354,20 @@ func (e *indexScanExec) Process(key, value []byte) error {
tblID := tablecodec.DecodeTableID(decodedKey)
e.chk.AppendInt64(*e.physTblIDColIdx, tblID)
}
// If we need common handle key, we should fill it here.
if e.commonHandleKeyIdx != nil && *e.commonHandleKeyIdx >= len(values) {
h, err := tablecodec.DecodeIndexHandle(decodedKey, value, e.numIdxCols)
if err != nil {
return err
}
commonHandle, ok := h.(*kv.CommonHandle)
if !ok {
return errors.New("common handle expected")
}
e.chk.AppendBytes(*e.commonHandleKeyIdx, commonHandle.Encoded())
}
if e.chk.IsFull() {
e.chunks = append(e.chunks, e.chk)
if e.paging != nil {
@ -530,9 +546,14 @@ func (e *indexLookUpExec) fetchTableScans() (tableScans []*tableScanExec, counts
for i := range chk.NumRows() {
row := chk.GetRow(i)
indexRows = append(indexRows, row)
handle, err := e.buildHandle(row)
if err != nil {
return nil, nil, nil, err
}
sortedHandles = append(sortedHandles, Handle{
IndexOrder: rowCnt,
Handle: e.buildIntHandle(row),
Handle: handle,
})
rowCnt++
}
@ -662,9 +683,12 @@ func (e *indexLookUpExec) regionContainsKey(r *metapb.Region, key []byte) bool {
(bytes.Compare(key, r.GetEndKey()) < 0 || len(r.GetEndKey()) == 0)
}
func (e *indexLookUpExec) buildIntHandle(row chunk.Row) kv.Handle {
func (e *indexLookUpExec) buildHandle(row chunk.Row) (kv.Handle, error) {
if e.isCommonHandle {
return kv.NewCommonHandle(row.GetBytes(row.Len() - 1))
}
i := row.GetInt64(int(e.indexHandleOffsets[0]))
return kv.IntHandle(i)
return kv.IntHandle(i), nil
}
func (e *indexLookUpExec) takeIntermediateResults() (ret []*chunk.Chunk) {

View File

@ -1,16 +1,11 @@
drop table if exists t1, t2, t3, t4, t5, t6, t7, tmp1, tmp2;
create table t1(id int primary key, a varchar(32), b int, c int, index i(a, b));
create table t2(a int, b int, c int, d int, primary key(a, b) CLUSTERED, index i(c));
create table t2(a varchar(32), b int, c int, d int, e int, primary key(a, b) CLUSTERED, index i(c), unique index u(e)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;
create table t3 (
id int primary key,
a int,
b int,
index a(a)
)
PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (1000),
PARTITION p1 VALUES LESS THAN (2000),
PARTITION p2 VALUES LESS THAN MAXVALUE
);
create table t4(id int primary key, a int, b int, index a(a));
alter table t4 cache;
@ -212,12 +207,107 @@ a b
313 30
214 40
115 50
insert into t2 values
('a', 1, 9010, 100, 90),
('A', 2, 8020, 200, 80),
('a', 3, 7030, 300, 70),
('b', 1, 6040, 400, 60),
('B', 2, 5050, 500, 50),
('b', 3, 4060, 600, 40),
('c', 1, 3070, 700, 30),
('C', 2, 2080, 800, 20),
('c', 3, 1090, 900, 10);
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2;
id estRows task access object operator info
TableReader_6 10000.00 root data:TableFullScan_5
└─TableFullScan_5 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, common handle table is not supported
IndexLookUp_7 10000.00 root
├─LocalIndexLookUp_9(Build) 10000.00 cop[tikv] index handle offsets:[]
│ ├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:t2, index:i(c) keep order:false, stats:pseudo
│ └─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(t2, i) */ * from t2;
a b c d e
a 1 9010 100 90
A 2 8020 200 80
a 3 7030 300 70
b 1 6040 400 60
B 2 5050 500 50
b 3 4060 600 40
c 1 3070 700 30
C 2 2080 800 20
c 3 1090 900 10
explain select /*+ index_lookup_pushdown(t2, u) */ * from t2;
id estRows task access object operator info
IndexLookUp_7 10000.00 root
├─LocalIndexLookUp_9(Build) 10000.00 cop[tikv] index handle offsets:[]
│ ├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:t2, index:u(e) keep order:false, stats:pseudo
│ └─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(t2, u) */ * from t2;
a b c d e
a 1 9010 100 90
A 2 8020 200 80
a 3 7030 300 70
b 1 6040 400 60
B 2 5050 500 50
b 3 4060 600 40
c 1 3070 700 30
C 2 2080 800 20
c 3 1090 900 10
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2 where b < 3 and a < 'c' limit 3;
id estRows task access object operator info
IndexLookUp_17 3.00 root limit embedded(offset:0, count:3)
├─LocalIndexLookUp_19(Build) 3.00 cop[tikv] index handle offsets:[]
│ ├─Limit_16(Build) 3.00 cop[tikv] offset:0, count:3
│ │ └─Selection_15 14.01 cop[tikv] lt(executor__index_lookup_pushdown.t2.a, "c"), lt(executor__index_lookup_pushdown.t2.b, 3)
│ │ └─IndexFullScan_13 126.89 cop[tikv] table:t2, index:i(c) keep order:false, stats:pseudo
│ └─TableRowIDScan_18(Probe) 3.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableRowIDScan_14(Probe) 0.00 cop[tikv] table:t2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(t2, i) */ * from t2 where b < 3 and a < 'c' limit 3;
a b c d e
A 2 8020 200 80
b 1 6040 400 60
B 2 5050 500 50
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2 order by a desc, b desc limit 4;
id estRows task access object operator info
TopN_9 4.00 root executor__index_lookup_pushdown.t2.a:desc, executor__index_lookup_pushdown.t2.b:desc, offset:0, count:4
└─IndexLookUp_18 4.00 root
├─LocalIndexLookUp_20(Build) 4.00 cop[tikv] index handle offsets:[]
│ ├─TopN_17(Build) 4.00 cop[tikv] executor__index_lookup_pushdown.t2.a:desc, executor__index_lookup_pushdown.t2.b:desc, offset:0, count:4
│ │ └─IndexFullScan_15 10000.00 cop[tikv] table:t2, index:i(c) keep order:false, stats:pseudo
│ └─TableRowIDScan_19(Probe) 4.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableRowIDScan_16(Probe) 0.00 cop[tikv] table:t2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(t2, i) */ * from t2 order by a desc, b desc limit 4;
a b c d e
c 3 1090 900 10
C 2 2080 800 20
c 1 3070 700 30
b 3 4060 600 40
create table t22(a varchar(32), b int, c int, d int, e int, primary key(a(2), b) CLUSTERED, index i(c)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;
insert into t22 values
('abc', 1, 9010, 100, 90),
('Abc', 2, 8020, 200, 80),
('abc', 3, 7030, 300, 70),
('bcd', 1, 6040, 400, 60),
('Bcd', 2, 5050, 500, 50),
('bcd', 3, 4060, 600, 40),
('cde', 1, 3070, 700, 30),
('Cde', 2, 2080, 800, 20),
('cde', 3, 1090, 900, 10);
explain select /*+ index_lookup_pushdown(t22, i) */ * from t22 where c > 2000 limit 5;
id estRows task access object operator info
IndexLookUp_16 5.00 root limit embedded(offset:0, count:5)
├─LocalIndexLookUp_18(Build) 5.00 cop[tikv] index handle offsets:[]
│ ├─Limit_15(Build) 5.00 cop[tikv] offset:0, count:5
│ │ └─IndexRangeScan_13 5.00 cop[tikv] table:t22, index:i(c) range:(2000,+inf], keep order:false, stats:pseudo
│ └─TableRowIDScan_17(Probe) 5.00 cop[tikv] table:t22 keep order:false, stats:pseudo
└─TableRowIDScan_14(Probe) 0.00 cop[tikv] table:t22 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(t22, i) */ * from t22 where c > 2000 limit 5;
a b c d e
bcd 1 6040 400 60
Bcd 2 5050 500 50
bcd 3 4060 600 40
cde 1 3070 700 30
Cde 2 2080 800 20
explain select /*+ index_lookup_pushdown(t1, i) */ * from t1 where a > 'a' order by a asc;
id estRows task access object operator info
IndexLookUp_18 3333.33 root
@ -225,13 +315,6 @@ IndexLookUp_18 3333.33 root
└─TableRowIDScan_17(Probe) 3333.33 cop[tikv] table:t1 keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, keep order is not supported.
explain select /*+ index_lookup_pushdown(t3, a) */ * from t3 where id < 100 and a = 1;
id estRows task access object operator info
TableReader_8 1.00 root partition:p0 data:Selection_7
└─Selection_7 1.00 cop[tikv] eq(executor__index_lookup_pushdown.t3.a, 1)
└─TableRangeScan_6 3333.33 cop[tikv] table:t3 range:[-inf,100), keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, partition table is not supported
explain select /*+ index_lookup_pushdown(t4, a) */ * from t4 where a < 100;
id estRows task access object operator info
UnionScan_7 3323.33 root lt(executor__index_lookup_pushdown.t4.a, 100)
@ -374,16 +457,17 @@ TableReader_8 10.00 root data:Selection_7
└─TableFullScan_6 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, multi-valued index is not supported
set @@tidb_replica_read='learner';
insert into t3 values(1, 2, 3);
explain select /*+ index_lookup_pushdown(t3, a) */ * from t3 use index(a);
id estRows task access object operator info
IndexLookUp_7 10000.00 root partition:all
IndexLookUp_7 10000.00 root
├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo
└─TableRowIDScan_6(Probe) 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, partition table is not supported
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, only leader read is supported
select /*+ index_lookup_pushdown(t3, a) */ * from t3 use index(a);
id a b
1 2 3
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, partition table is not supported
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, only leader read is supported

View File

@ -0,0 +1,103 @@
drop table if exists tp1, tp2;
create table tp1 (a int primary key, b int, c int, key b(b), key c(c) global) partition by hash(a) partitions 4;
insert into tp1 values (1, 10, 10), (2, 9, 20), (3, 8, 30), (4, 7, 40), (5, 6, 50), (6, 5, 60);
explain select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 and a > 1 limit 3;
id estRows task access object operator info
IndexLookUp_17 3.00 root partition:all limit embedded(offset:0, count:3)
├─LocalIndexLookUp_19(Build) 3.00 cop[tikv] index handle offsets:[1]
│ ├─Limit_16(Build) 3.00 cop[tikv] offset:0, count:3
│ │ └─Selection_15 14.05 cop[tikv] gt(executor__index_lookup_pushdown_partition.tp1.a, 1)
│ │ └─IndexRangeScan_13 42.14 cop[tikv] table:tp1, index:b(b) range:[-inf,10), keep order:false, stats:pseudo
│ └─TableRowIDScan_18(Probe) 3.00 cop[tikv] table:tp1 keep order:false, stats:pseudo
└─TableRowIDScan_14(Probe) 0.00 cop[tikv] table:tp1 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 and a > 1 limit 3;
a b c
4 7 40
5 6 50
2 9 20
explain select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 order by a limit 3;
id estRows task access object operator info
TopN_10 3.00 root executor__index_lookup_pushdown_partition.tp1.a, offset:0, count:3
└─IndexLookUp_19 3.00 root partition:all
├─LocalIndexLookUp_21(Build) 3.00 cop[tikv] index handle offsets:[1]
│ ├─TopN_18(Build) 3.00 cop[tikv] executor__index_lookup_pushdown_partition.tp1.a, offset:0, count:3
│ │ └─IndexRangeScan_16 3323.33 cop[tikv] table:tp1, index:b(b) range:[-inf,10), keep order:false, stats:pseudo
│ └─TableRowIDScan_20(Probe) 3.00 cop[tikv] table:tp1 keep order:false, stats:pseudo
└─TableRowIDScan_17(Probe) 0.00 cop[tikv] table:tp1 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 order by a limit 3;
a b c
2 9 20
3 8 30
4 7 40
explain select /*+ index_lookup_pushdown(tp1, c) */ * from tp1;
id estRows task access object operator info
TableReader_6 10000.00 root partition:all data:TableFullScan_5
└─TableFullScan_5 10000.00 cop[tikv] table:tp1 keep order:false, stats:pseudo
Level Code Message
Warning 1815 hint INDEX_LOOKUP_PUSHDOWN is inapplicable, the global index in partition table is not supported
create table tp2 (
id1 varchar(32),
id2 int,
a int,
b int,
primary key (id1, id2),
index a(a)
)
PARTITION BY RANGE COLUMNS (id1) (
PARTITION p0 VALUES LESS THAN ('c'),
PARTITION p1 VALUES LESS THAN ('e'),
PARTITION p2 VALUES LESS THAN ('g'),
PARTITION p3 VALUES LESS THAN MAXVALUE
);
insert into tp2 values
('a', 1, 99, 10),
('b', 2, 88, 20),
('c', 3, 77, 30),
('d', 4, 66, 40),
('e', 5, 55, 50),
('f', 6, 44, 60),
('g', 7, 33, 70),
('h', 8, 22, 80);
explain select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 where a > 33 limit 5;
id estRows task access object operator info
IndexLookUp_16 5.00 root partition:all limit embedded(offset:0, count:5)
├─LocalIndexLookUp_18(Build) 5.00 cop[tikv] index handle offsets:[1]
│ ├─Limit_15(Build) 5.00 cop[tikv] offset:0, count:5
│ │ └─IndexRangeScan_13 5.00 cop[tikv] table:tp2, index:a(a) range:(33,+inf], keep order:false, stats:pseudo
│ └─TableRowIDScan_17(Probe) 5.00 cop[tikv] table:tp2 keep order:false, stats:pseudo
└─TableRowIDScan_14(Probe) 0.00 cop[tikv] table:tp2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 where a > 33 limit 5;
id1 id2 a b
a 1 99 10
b 2 88 20
c 3 77 30
d 4 66 40
e 5 55 50
explain select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 partition (p1);
id estRows task access object operator info
IndexLookUp_7 10000.00 root partition:p1
├─LocalIndexLookUp_9(Build) 10000.00 cop[tikv] index handle offsets:[1]
│ ├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:tp2, index:a(a) keep order:false, stats:pseudo
│ └─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:tp2 keep order:false, stats:pseudo
└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:tp2 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 partition (p1);
id1 id2 a b
c 3 77 30
d 4 66 40
create table tp3 (a int, b int, c int, key b(b)) partition by hash(a) partitions 4;
insert into tp3 values (1, 10, 10), (2, 9, 20), (3, 8, 30), (4, 7, 40), (5, 6, 50), (6, 5, 60);
explain select /*+ index_lookup_pushdown(tp3, b) */ * from tp3;
id estRows task access object operator info
IndexLookUp_7 10000.00 root partition:all
├─LocalIndexLookUp_9(Build) 10000.00 cop[tikv] index handle offsets:[1]
│ ├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:tp3, index:b(b) keep order:false, stats:pseudo
│ └─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:tp3 keep order:false, stats:pseudo
└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:tp3 keep order:false, stats:pseudo
select /*+ index_lookup_pushdown(tp3, b) */ * from tp3;
a b c
4 7 40
1 10 10
5 6 50
2 9 20
6 5 60
3 8 30

View File

@ -1,16 +1,11 @@
drop table if exists t1, t2, t3, t4, t5, t6, t7, tmp1, tmp2;
create table t1(id int primary key, a varchar(32), b int, c int, index i(a, b));
create table t2(a int, b int, c int, d int, primary key(a, b) CLUSTERED, index i(c));
create table t2(a varchar(32), b int, c int, d int, e int, primary key(a, b) CLUSTERED, index i(c), unique index u(e)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;
create table t3 (
id int primary key,
a int,
b int,
index a(a)
)
PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (1000),
PARTITION p1 VALUES LESS THAN (2000),
PARTITION p2 VALUES LESS THAN MAXVALUE
);
create table t4(id int primary key, a int, b int, index a(a));
alter table t4 cache;
@ -76,13 +71,44 @@ insert into t6 values (511, 10), (412, 20), (313, 30), (214, 40), (115, 50);
explain select /*+ index_lookup_pushdown(t6, primary) */ * from t6;
select /*+ index_lookup_pushdown(t6, primary) */ * from t6;
# Test index lookup push down restrictions
## Common handle not supported
## test for common handle
insert into t2 values
('a', 1, 9010, 100, 90),
('A', 2, 8020, 200, 80),
('a', 3, 7030, 300, 70),
('b', 1, 6040, 400, 60),
('B', 2, 5050, 500, 50),
('b', 3, 4060, 600, 40),
('c', 1, 3070, 700, 30),
('C', 2, 2080, 800, 20),
('c', 3, 1090, 900, 10);
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2;
select /*+ index_lookup_pushdown(t2, i) */ * from t2;
explain select /*+ index_lookup_pushdown(t2, u) */ * from t2;
select /*+ index_lookup_pushdown(t2, u) */ * from t2;
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2 where b < 3 and a < 'c' limit 3;
select /*+ index_lookup_pushdown(t2, i) */ * from t2 where b < 3 and a < 'c' limit 3;
explain select /*+ index_lookup_pushdown(t2, i) */ * from t2 order by a desc, b desc limit 4;
select /*+ index_lookup_pushdown(t2, i) */ * from t2 order by a desc, b desc limit 4;
## test common handle with primary key prefix column
create table t22(a varchar(32), b int, c int, d int, e int, primary key(a(2), b) CLUSTERED, index i(c)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;
insert into t22 values
('abc', 1, 9010, 100, 90),
('Abc', 2, 8020, 200, 80),
('abc', 3, 7030, 300, 70),
('bcd', 1, 6040, 400, 60),
('Bcd', 2, 5050, 500, 50),
('bcd', 3, 4060, 600, 40),
('cde', 1, 3070, 700, 30),
('Cde', 2, 2080, 800, 20),
('cde', 3, 1090, 900, 10);
explain select /*+ index_lookup_pushdown(t22, i) */ * from t22 where c > 2000 limit 5;
select /*+ index_lookup_pushdown(t22, i) */ * from t22 where c > 2000 limit 5;
# Test index lookup push down restrictions
## Keep order not supported
explain select /*+ index_lookup_pushdown(t1, i) */ * from t1 where a > 'a' order by a asc;
## partition table not supported
explain select /*+ index_lookup_pushdown(t3, a) */ * from t3 where id < 100 and a = 1;
## cached table not supported
explain select /*+ index_lookup_pushdown(t4, a) */ * from t4 where a < 100;
## temporary tables are not supported
@ -136,6 +162,7 @@ explain select /*+ index_lookup_pushdown(t7, idx) */ * from t7 where (1 member o
--disable_warnings
## issue #64519, the index_lookup_pushdown hint should not affect other hints in a same index.
set @@tidb_replica_read='learner';
insert into t3 values(1, 2, 3);
--enable_warnings
explain select /*+ index_lookup_pushdown(t3, a) */ * from t3 use index(a);

View File

@ -0,0 +1,51 @@
drop table if exists tp1, tp2;
# int handle
create table tp1 (a int primary key, b int, c int, key b(b), key c(c) global) partition by hash(a) partitions 4;
insert into tp1 values (1, 10, 10), (2, 9, 20), (3, 8, 30), (4, 7, 40), (5, 6, 50), (6, 5, 60);
explain select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 and a > 1 limit 3;
select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 and a > 1 limit 3;
explain select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 order by a limit 3;
select /*+ index_lookup_pushdown(tp1, b) */ * from tp1 where b < 10 order by a limit 3;
# global index not supported
--enable_warnings
explain select /*+ index_lookup_pushdown(tp1, c) */ * from tp1;
--disable_warnings
# common handle
create table tp2 (
id1 varchar(32),
id2 int,
a int,
b int,
primary key (id1, id2),
index a(a)
)
PARTITION BY RANGE COLUMNS (id1) (
PARTITION p0 VALUES LESS THAN ('c'),
PARTITION p1 VALUES LESS THAN ('e'),
PARTITION p2 VALUES LESS THAN ('g'),
PARTITION p3 VALUES LESS THAN MAXVALUE
);
insert into tp2 values
('a', 1, 99, 10),
('b', 2, 88, 20),
('c', 3, 77, 30),
('d', 4, 66, 40),
('e', 5, 55, 50),
('f', 6, 44, 60),
('g', 7, 33, 70),
('h', 8, 22, 80);
explain select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 where a > 33 limit 5;
select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 where a > 33 limit 5;
# specify the partition
explain select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 partition (p1);
select /*+ index_lookup_pushdown(tp2, a) */ * from tp2 partition (p1);
# extra handle
create table tp3 (a int, b int, c int, key b(b)) partition by hash(a) partitions 4;
insert into tp3 values (1, 10, 10), (2, 9, 20), (3, 8, 30), (4, 7, 40), (5, 6, 50), (6, 5, 60);
explain select /*+ index_lookup_pushdown(tp3, b) */ * from tp3;
select /*+ index_lookup_pushdown(tp3, b) */ * from tp3;

View File

@ -9,7 +9,7 @@ go_test(
"main_test.go",
],
flaky = True,
shard_count = 2,
shard_count = 4,
deps = [
"//pkg/testkit",
"//pkg/util/logutil",

View File

@ -29,6 +29,130 @@ import (
"go.uber.org/zap"
)
type IndexLookUpPushDownRunVerifier struct {
*testing.T
tk *testkit.TestKit
tableName string
indexName string
primaryRows []int
msg string
}
type RunSelectWithCheckResult struct {
SQL string
Rows [][]any
AnalyzeRows [][]any
}
func (t *IndexLookUpPushDownRunVerifier) RunSelectWithCheck(where string, skip, limit int) RunSelectWithCheckResult {
require.NotNil(t, t.tk)
require.NotEmpty(t, t.tableName)
require.NotEmpty(t, t.indexName)
require.NotEmpty(t, t.primaryRows)
require.GreaterOrEqual(t, skip, 0)
if skip > 0 {
require.GreaterOrEqual(t, limit, 0)
}
message := fmt.Sprintf("%s, table: %s, where: %s, limit: %d", t.msg, t.tableName, where, limit)
var sb strings.Builder
sb.WriteString(fmt.Sprintf("select /*+ index_lookup_pushdown(%s, %s)*/ * from %s where ", t.tableName, t.indexName, t.tableName))
sb.WriteString(where)
if skip > 0 {
sb.WriteString(fmt.Sprintf(" limit %d, %d", skip, limit))
} else if limit >= 0 {
sb.WriteString(fmt.Sprintf(" limit %d", limit))
}
// make sure the query uses index lookup
analyzeSQL := "explain analyze " + sb.String()
analyzeResult := t.tk.MustQuery(analyzeSQL)
require.Contains(t, analyzeResult.String(), "LocalIndexLookUp", analyzeSQL+"\n"+analyzeResult.String())
// get actual result
rs := t.tk.MustQuery(sb.String())
actual := rs.Rows()
idSets := make(map[string]struct{}, len(actual))
for _, row := range actual {
var primaryKey strings.Builder
require.Greater(t, len(t.primaryRows), 0)
for i, idx := range t.primaryRows {
if i > 0 {
primaryKey.WriteString("#")
}
primaryKey.WriteString(row[idx].(string))
}
id := primaryKey.String()
_, dup := idSets[id]
require.False(t, dup, "dupID: "+id+", "+message)
idSets[row[0].(string)] = struct{}{}
}
// use table scan
matchCondList := t.tk.MustQuery(fmt.Sprintf("select /*+ use_index(%s) */* from %s where "+where, t.tableName, t.tableName)).Rows()
if limit == 0 || skip >= len(matchCondList) {
require.Len(t, actual, 0, message)
} else if limit < 0 {
// no limit two results should have same members
require.ElementsMatch(t, matchCondList, actual, message)
} else {
expectRowCnt := limit
if skip+limit > len(matchCondList) {
expectRowCnt = len(matchCondList) - skip
}
require.Len(t, actual, expectRowCnt, message)
require.Subset(t, matchCondList, actual, message)
}
// check in analyze the index is lookup locally
message = fmt.Sprintf("%s\n%s\n%s", message, analyzeSQL, analyzeResult.String())
analyzeVerified := false
localIndexLookUpIndex := -1
totalIndexScanCnt := -1
localIndexLookUpRowCnt := -1
analyzeRows := analyzeResult.Rows()
metTableRowIDScan := false
for i, row := range analyzeRows {
if strings.Contains(row[0].(string), "LocalIndexLookUp") {
localIndexLookUpIndex = i
continue
}
if strings.Contains(row[0].(string), "TableRowIDScan") && strings.Contains(row[3].(string), "cop[tikv]") {
var err error
if !metTableRowIDScan {
localIndexLookUpRowCnt, err = strconv.Atoi(row[2].(string))
require.NoError(t, err, message)
require.GreaterOrEqual(t, localIndexLookUpRowCnt, 0)
// check actRows for LocalIndexLookUp
require.Equal(t, analyzeRows[localIndexLookUpIndex][2], row[2], message)
// get index scan row count
totalIndexScanCnt, err = strconv.Atoi(analyzeRows[localIndexLookUpIndex+1][2].(string))
require.NoError(t, err, message)
require.GreaterOrEqual(t, totalIndexScanCnt, localIndexLookUpRowCnt)
metTableRowIDScan = true
continue
}
tidbIndexLookUpRowCnt, err := strconv.Atoi(row[2].(string))
require.NoError(t, err, message)
if limit < 0 {
require.Equal(t, totalIndexScanCnt, localIndexLookUpRowCnt+tidbIndexLookUpRowCnt, message)
} else {
require.LessOrEqual(t, localIndexLookUpRowCnt+tidbIndexLookUpRowCnt, totalIndexScanCnt, message)
}
analyzeVerified = true
break
}
}
require.True(t, analyzeVerified, analyzeResult.String())
return RunSelectWithCheckResult{
SQL: sb.String(),
Rows: actual,
AnalyzeRows: analyzeRows,
}
}
func TestRealTiKVIndexLookUpPushDown(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -51,95 +175,226 @@ func TestRealTiKVIndexLookUpPushDown(t *testing.T) {
tk.MustExec("insert into t values " + strings.Join(values, ","))
}
runSelectWithCheck := func(where string, skip, limit int) {
require.GreaterOrEqual(t, skip, 0)
if skip > 0 {
require.GreaterOrEqual(t, limit, 0)
}
message := fmt.Sprintf("seed: %d, where: %s, limit: %d", seed, where, limit)
var sb strings.Builder
sb.WriteString("select /*+ index_lookup_pushdown(t, a)*/ * from t where ")
sb.WriteString(where)
if skip > 0 {
sb.WriteString(fmt.Sprintf(" limit %d, %d", skip, limit))
} else if limit >= 0 {
sb.WriteString(fmt.Sprintf(" limit %d", limit))
}
// make sure the query uses index lookup
analyzeResult := tk.MustQuery("explain analyze " + sb.String())
require.Contains(t, analyzeResult.String(), "LocalIndexLookUp", analyzeResult.String())
// get actual result
rs := tk.MustQuery(sb.String())
actual := rs.Rows()
idSets := make(map[string]struct{}, len(actual))
for _, row := range actual {
id := row[0].(string)
_, dup := idSets[id]
require.False(t, dup, "dupID: "+id+", "+message)
idSets[row[0].(string)] = struct{}{}
}
// use table scan
matchCondList := tk.MustQuery("select /*+ use_index(t) */* from t where " + where + " order by id").Rows()
if limit == 0 || skip >= len(matchCondList) {
require.Len(t, actual, 0, message)
} else if limit < 0 {
// no limit two results should have same members
require.ElementsMatch(t, matchCondList, actual, message)
} else {
expectRowCnt := limit
if skip+limit > len(matchCondList) {
expectRowCnt = len(matchCondList) - skip
}
require.Len(t, actual, expectRowCnt, message)
require.Subset(t, matchCondList, actual, message)
}
// check in analyze the index is lookup locally
message = fmt.Sprintf("%s, %s", message, analyzeResult.String())
localIndexLookUpIndex := -1
analyzeVerified := false
analyzeRows := analyzeResult.Rows()
for i, row := range analyzeRows {
if strings.Contains(row[0].(string), "LocalIndexLookUp") {
localIndexLookUpIndex = i
continue
}
if strings.Contains(row[0].(string), "TableRowIDScan") && strings.Contains(row[3].(string), "cop[tikv]") {
localLookUpRowCnt, err := strconv.Atoi(row[2].(string))
require.NoError(t, err, message)
if len(actual) > 0 {
require.Greater(t, localLookUpRowCnt, 0, message)
}
require.Equal(t, analyzeRows[i-1][2], row[2], message)
require.Equal(t, analyzeRows[localIndexLookUpIndex][2], row[2], message)
analyzeVerified = true
break
}
}
require.True(t, analyzeVerified, analyzeResult.String())
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: "t",
indexName: "a",
primaryRows: []int{0},
msg: fmt.Sprintf("seed: %d", seed),
}
runSelectWithCheck("1", 0, -1)
runSelectWithCheck("1", 0, r.Intn(total*2))
runSelectWithCheck("1", total/2, r.Intn(total))
runSelectWithCheck("1", total-10, 20)
runSelectWithCheck("1", total, 10)
runSelectWithCheck("1", 10, 0)
runSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, 25)
runSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, r.Intn(100)+1)
runSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, r.Intn(100)+1)
v.RunSelectWithCheck("1", 0, -1)
v.RunSelectWithCheck("1", 0, r.Intn(total*2))
v.RunSelectWithCheck("1", total/2, r.Intn(total))
v.RunSelectWithCheck("1", total-10, 20)
v.RunSelectWithCheck("1", total, 10)
v.RunSelectWithCheck("1", 10, 0)
v.RunSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a = %d", randIndexVal()), 0, 25)
v.RunSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a < %d", randIndexVal()), 0, r.Intn(100)+1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d", randIndexVal()), 0, r.Intn(100)+1)
start := randIndexVal()
runSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, -1)
start = randIndexVal()
runSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, r.Intn(50)+1)
runSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, -1)
runSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, r.Intn(50)+1)
v.RunSelectWithCheck(fmt.Sprintf("a >= %d and a < %d", start, start+r.Intn(5)+1), 0, r.Intn(50)+1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, -1)
v.RunSelectWithCheck(fmt.Sprintf("a > %d and b < %d", randIndexVal(), r.Int63()), 0, r.Intn(50)+1)
}
func TestRealTiKVCommonHandleIndexLookUpPushDown(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
collations := [][]string{
{"binary", "binary"},
{"ascii", "ascii_bin"},
{"latin1", "latin1_bin"},
{"gbk", "gbk_bin"},
{"gbk", "gbk_chinese_ci"},
{"utf8mb4", "utf8mb4_bin"},
{"utf8mb4", "utf8mb4_general_ci"},
{"utf8mb4", "utf8mb4_unicode_ci"},
{"utf8mb4", "utf8mb4_0900_ai_ci"},
{"utf8mb4", "utf8mb4_0900_bin"},
}
prepareTable := func(v *IndexLookUpPushDownRunVerifier, uniqueIndex bool, charset, collation, primaryKey string) {
uniquePrefix := ""
if uniqueIndex {
uniquePrefix = "unique "
}
tk.MustExec("drop table if exists " + v.tableName)
tk.MustExec("create table " + v.tableName + " (" +
"id1 varchar(64), " +
"id2 bigint, " +
"a bigint, " +
"b bigint, " +
"primary key(" + primaryKey + "), " +
uniquePrefix + "index " + v.indexName + "(a)" +
") charset=" + charset + " collate=" + collation)
tk.MustExec("insert into " + v.tableName + " values " +
"('abcA', 1, 99, 199), " +
"('aBCE', 2, 98, 198), " +
"('ABdd', 1, 97, 197), " +
"('abdc', 2, 96, 196), " +
"('Defb', 1, 95, 195), " +
"('defa', 2, 94, 194), " +
"('efga', 1, 93, 193)",
)
}
for i, cs := range collations {
for j, unique := range []bool{true, false} {
charset := cs[0]
collation := cs[1]
caseName := fmt.Sprintf("%s-%s-unique-%v", charset, collation, unique)
t.Run(caseName, func(t *testing.T) {
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: fmt.Sprintf("t_common_handle_%d_%d", i, j),
indexName: "idx_a",
primaryRows: []int{0, 1},
msg: fmt.Sprintf("case: %s", caseName),
}
prepareTable(v, unique, charset, collation, "id1, id2")
v.RunSelectWithCheck("1", 0, -1)
v.RunSelectWithCheck("a > 93 and b < 199", 0, 10)
v.RunSelectWithCheck("a > 93 and b < 199 and id1 != 'abd'", 0, 10)
// check the TopN push down
result := v.RunSelectWithCheck("1 order by id2, id1", 0, 5)
require.Contains(t, result.AnalyzeRows[2][0], "LocalIndexLookUp")
require.Contains(t, result.AnalyzeRows[3][0], "TopN")
require.Equal(t, "cop[tikv]", result.AnalyzeRows[3][3])
if strings.Contains(collation, "_ci") {
require.Equal(t, [][]any{
{"abcA", "1", "99", "199"},
{"ABdd", "1", "97", "197"},
{"Defb", "1", "95", "195"},
{"efga", "1", "93", "193"},
{"aBCE", "2", "98", "198"},
}, result.Rows)
} else {
require.Equal(t, [][]any{
{"ABdd", "1", "97", "197"},
{"Defb", "1", "95", "195"},
{"abcA", "1", "99", "199"},
{"efga", "1", "93", "193"},
{"aBCE", "2", "98", "198"},
}, result.Rows)
}
})
}
// test prefix index column
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: "t_common_handle_prefix_primary_index",
indexName: "idx_a",
primaryRows: []int{0, 1},
msg: "case: t_common_handle_prefix_primary_index",
}
prepareTable(v, false, "utf8mb4", "utf8mb4_general_ci", "id1(3), id2")
v.RunSelectWithCheck("1", 0, -1)
// test two int column primary key
v = &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: "t_common_handle_two_int_pk",
indexName: "idx_a",
primaryRows: []int{0, 1},
msg: "case: t_common_handle_two_int_pk",
}
prepareTable(v, false, "utf8mb4", "utf8mb4_general_ci", "b, id2")
v.RunSelectWithCheck("1", 0, -1)
}
}
func TestRealTiKVPartitionIndexLookUpPushDown(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// int handle
tk.MustExec("create table tp1 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(b) CLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE (b) (\n" +
" PARTITION p0 VALUES LESS THAN (100),\n" +
" PARTITION p1 VALUES LESS THAN (200),\n" +
" PARTITION p2 VALUES LESS THAN (300),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
// common handle
tk.MustExec("create table tp2 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(a, b) CLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE COLUMNS (a) (\n" +
" PARTITION p0 VALUES LESS THAN ('c'),\n" +
" PARTITION p1 VALUES LESS THAN ('e'),\n" +
" PARTITION p2 VALUES LESS THAN ('g'),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
// extra handle
tk.MustExec("create table tp3 (\n" +
" a varchar(32),\n" +
" b int,\n" +
" c int,\n" +
" d int,\n" +
" primary key(a, b) NONCLUSTERED,\n" +
" index c(c)\n" +
")\n" +
"PARTITION BY RANGE COLUMNS (a) (\n" +
" PARTITION p0 VALUES LESS THAN ('c'),\n" +
" PARTITION p1 VALUES LESS THAN ('e'),\n" +
" PARTITION p2 VALUES LESS THAN ('g'),\n" +
" PARTITION p3 VALUES LESS THAN MAXVALUE\n" +
")")
tableNames := []string{"tp1", "tp2", "tp3"}
// prepare data
for _, tableName := range tableNames {
tk.MustExec("insert into " + tableName + " values " +
"('a', 10, 1, 100), " +
"('b', 20, 2, 200), " +
"('c', 110, 3, 300), " +
"('d', 120, 4, 400), " +
"('e', 210, 5, 500), " +
"('f', 220, 6, 600), " +
"('g', 330, 5, 700), " +
"('h', 340, 5, 800), " +
"('i', 450, 5, 900), " +
"('j', 550, 6, 1000) ",
)
v := &IndexLookUpPushDownRunVerifier{
T: t,
tk: tk,
tableName: tableName,
indexName: "c",
primaryRows: []int{0, 1},
msg: tableName,
}
if tableName == "tp1" {
v.primaryRows = []int{1}
}
v.RunSelectWithCheck("1", 0, -1)
}
}