*: 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:
@ -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
|
||||
}
|
||||
|
||||
@ -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++
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
@ -9,7 +9,7 @@ go_test(
|
||||
"main_test.go",
|
||||
],
|
||||
flaky = True,
|
||||
shard_count = 2,
|
||||
shard_count = 4,
|
||||
deps = [
|
||||
"//pkg/testkit",
|
||||
"//pkg/util/logutil",
|
||||
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user