Skip to content

Commit

Permalink
executor: fix the behavior when index join meet prefix index (pingcap…
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros committed Jul 15, 2019
1 parent 14d29c3 commit 93307c4
Show file tree
Hide file tree
Showing 6 changed files with 53 additions and 14 deletions.
9 changes: 9 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1738,6 +1738,13 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin)
if defaultValues == nil {
defaultValues = make([]types.Datum, len(innerTypes))
}
hasPrefixCol := false
for _, l := range v.IdxColLens {
if l != types.UnspecifiedLength {
hasPrefixCol = true
break
}
}
e := &IndexLookUpJoin{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), outerExec),
outerCtx: outerCtx{
Expand All @@ -1747,6 +1754,8 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin)
innerCtx: innerCtx{
readerBuilder: &dataReaderBuilder{Plan: innerPlan, executorBuilder: b},
rowTypes: innerTypes,
colLens: v.IdxColLens,
hasPrefixCol: hasPrefixCol,
},
workerWg: new(sync.WaitGroup),
joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes),
Expand Down
12 changes: 12 additions & 0 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@ type innerCtx struct {
readerBuilder *dataReaderBuilder
rowTypes []*types.FieldType
keyCols []int
colLens []int
hasPrefixCol bool
}

type lookUpJoinTask struct {
Expand Down Expand Up @@ -495,6 +497,16 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi
}
// Store the encoded lookup key in chunk, so we can use it to lookup the matched inners directly.
task.encodedLookUpKeys.AppendBytes(0, keyBuf)
if iw.hasPrefixCol {
for i := range iw.outerCtx.keyCols {
// If it's a prefix column. Try to fix it.
if iw.colLens[i] != types.UnspecifiedLength {
ranger.CutDatumByPrefixLen(&dLookUpKey[i], iw.colLens[i], iw.rowTypes[iw.keyCols[i]])
}
}
// dLookUpKey is sorted and deduplicated at sortAndDedupLookUpContents.
// So we don't need to do it here.
}
lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: task.outerResult.GetRow(i)})
}

Expand Down
8 changes: 8 additions & 0 deletions executor/index_lookup_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,3 +152,11 @@ func (s *testSuite) TestIndexJoinOverflow(c *C) {
tk.MustExec(`create table t2(a int unsigned, index idx(a));`)
tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a;`).Check(testkit.Rows())
}

func (s *testSuite2) TestIssue11061(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1(c varchar(30), index ix_c(c(10)))")
tk.MustExec("insert into t1 (c) values('7_chars'), ('13_characters')")
tk.MustQuery("SELECT /*+ TIDB_INLJ(t1) */ SUM(LENGTH(c)) FROM t1 WHERE c IN (SELECT t1.c FROM t1)").Check(testkit.Rows("20"))
}
18 changes: 13 additions & 5 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,8 +322,15 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int)

// When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range
// is generated during execution time.
func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerIdx int, innerPlan PhysicalPlan,
ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCmpFuncManager) []PhysicalPlan {
func (p *LogicalJoin) constructIndexJoin(
prop *property.PhysicalProperty,
outerIdx int,
innerPlan PhysicalPlan,
ranges []*ranger.Range,
keyOff2IdxOff []int,
lens []int,
compareFilters *ColWithCmpFuncManager,
) []PhysicalPlan {
joinType := p.JoinType
outerSchema := p.children[outerIdx].Schema()
var (
Expand Down Expand Up @@ -373,6 +380,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerI
DefaultValues: p.DefaultValues,
innerPlan: innerPlan,
KeyOff2IdxOff: newKeyOff,
IdxColLens: lens,
Ranges: ranges,
CompareFilters: compareFilters,
}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...)
Expand Down Expand Up @@ -431,7 +439,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou
innerPlan := p.constructInnerTableScan(ds, pkCol, outerJoinKeys, us)
// Since the primary key means one value corresponding to exact one row, this will always be a no worse one
// comparing to other index.
return p.constructIndexJoin(prop, outerIdx, innerPlan, nil, keyOff2IdxOff, nil)
return p.constructIndexJoin(prop, outerIdx, innerPlan, nil, keyOff2IdxOff, nil, nil)
}
}
helper := &indexJoinBuildHelper{join: p}
Expand All @@ -455,10 +463,10 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou
keyOff2IdxOff[keyOff] = idxOff
}
}
idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo)
idxCols, lens := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo)
rangeInfo := helper.buildRangeDecidedByInformation(idxCols, outerJoinKeys)
innerPlan := p.constructInnerIndexScan(ds, helper.chosenIndexInfo, helper.chosenRemained, outerJoinKeys, us, rangeInfo)
return p.constructIndexJoin(prop, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager)
return p.constructIndexJoin(prop, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, lens, helper.lastColManager)
}
return nil
}
Expand Down
2 changes: 2 additions & 0 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -232,6 +232,8 @@ type PhysicalIndexJoin struct {
Ranges []*ranger.Range
// KeyOff2IdxOff maps the offsets in join key to the offsets in the index.
KeyOff2IdxOff []int
// IdxColLens stores the length of each index column.
IdxColLens []int
// CompareFilters stores the filters for last column if those filters need to be evaluated during execution.
// e.g. select * from t where t.a = t1.a and t.b > t1.b and t.b < t1.b+10
// If there's index(t.a, t.b). All the filters can be used to construct index range but t.b > t1.b and t.b < t1.b=10
Expand Down
18 changes: 9 additions & 9 deletions util/ranger/ranger.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,10 +263,10 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat
}
if colLen != types.UnspecifiedLength {
for _, ran := range ranges {
if fixRangeDatum(&ran.LowVal[0], colLen, tp) {
if CutDatumByPrefixLen(&ran.LowVal[0], colLen, tp) {
ran.LowExclude = false
}
if fixRangeDatum(&ran.HighVal[0], colLen, tp) {
if CutDatumByPrefixLen(&ran.HighVal[0], colLen, tp) {
ran.HighExclude = false
}
}
Expand Down Expand Up @@ -425,17 +425,17 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo
for _, ran := range ranges {
lowTail := len(ran.LowVal) - 1
for i := 0; i < lowTail; i++ {
fixRangeDatum(&ran.LowVal[i], lengths[i], tp[i])
CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i])
}
lowCut := fixRangeDatum(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail])
lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail])
if lowCut {
ran.LowExclude = false
}
highTail := len(ran.HighVal) - 1
for i := 0; i < highTail; i++ {
fixRangeDatum(&ran.HighVal[i], lengths[i], tp[i])
CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i])
}
highCut := fixRangeDatum(&ran.HighVal[highTail], lengths[highTail], tp[highTail])
highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail])
if highCut {
ran.HighExclude = false
}
Expand All @@ -444,9 +444,9 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo
return hasCut
}

func fixRangeDatum(v *types.Datum, length int, tp *types.FieldType) bool {
// If this column is prefix and the prefix length is smaller than the range, cut it.
// In case of UTF8, prefix should be cut by characters rather than bytes
// CutDatumByPrefixLen cuts the datum according to the prefix length.
// If it's UTF8 encoded, we will cut it by characters rather than bytes.
func CutDatumByPrefixLen(v *types.Datum, length int, tp *types.FieldType) bool {
if v.Kind() == types.KindString || v.Kind() == types.KindBytes {
colCharset := tp.Charset
colValue := v.GetBytes()
Expand Down

0 comments on commit 93307c4

Please sign in to comment.