Skip to content

Commit

Permalink
util/ranger: move cut prefix logic for prefix index to ealier stage (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
time-and-fate committed Feb 19, 2024
1 parent def88be commit 26af280
Show file tree
Hide file tree
Showing 5 changed files with 80 additions and 97 deletions.
12 changes: 4 additions & 8 deletions pkg/util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,7 +337,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi
// Therefore, we need to calculate pointRanges separately so that it can be used to append tail ranges in considerDNF branch.
// See https://github.com/pingcap/tidb/issues/26029 for details.
var pointRanges Ranges
if hasPrefix(d.lengths) && fixPrefixColRange(ranges, d.lengths, tpSlice) {
if hasPrefix(d.lengths) {
if d.mergeConsecutive {
pointRanges = make(Ranges, 0, len(ranges))
for _, ran := range ranges {
Expand Down Expand Up @@ -630,9 +630,9 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex
collator := collate.GetCollator(cols[offset].GetType().GetCollate())
if mergedAccesses[offset] == nil {
mergedAccesses[offset] = accesses[offset]
points[offset] = rb.build(accesses[offset], collator)
points[offset] = rb.build(accesses[offset], collator, lengths[offset])
}
points[offset] = rb.intersection(points[offset], rb.build(cond, collator), collator)
points[offset] = rb.intersection(points[offset], rb.build(cond, collator, lengths[offset]), collator)
if len(points[offset]) == 0 { // Early termination if false expression found
if expression.MaybeOverOptimized4PlanCache(sctx, conditions) {
// `a>@x and a<@y` --> `invalid-range if @x>=@y`
Expand Down Expand Up @@ -772,7 +772,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression
if shouldReserve {
hasResidual = true
}
points := rb.build(item, collate.GetCollator(newTpSlice[0].GetCollate()))
points := rb.build(item, collate.GetCollator(newTpSlice[0].GetCollate()), d.lengths[0])
// TODO: restrict the mem usage of ranges
ranges, rangeFallback, err := points2Ranges(d.sctx, points, newTpSlice[0], d.rangeMaxSize)
if err != nil {
Expand Down Expand Up @@ -804,10 +804,6 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression
}
}

// Take prefix index into consideration.
if hasPrefix(d.lengths) {
fixPrefixColRange(totalRanges, d.lengths, newTpSlice)
}
totalRanges, err := UnionRanges(d.sctx, totalRanges, d.mergeConsecutive)
if err != nil {
return nil, nil, nil, false, errors.Trace(err)
Expand Down
75 changes: 47 additions & 28 deletions pkg/util/ranger/points.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func (r *pointSorter) Less(i, j int) bool {

func rangePointLess(sc *stmtctx.StatementContext, a, b *point, collator collate.Collator) (bool, error) {
if a.value.Kind() == types.KindMysqlEnum && b.value.Kind() == types.KindMysqlEnum {
return rangePointEnumLess(sc, a, b)
return rangePointEnumLess(a, b)
}
cmp, err := a.value.Compare(sc, &b.value, collator)
if cmp != 0 {
Expand All @@ -116,7 +116,7 @@ func rangePointLess(sc *stmtctx.StatementContext, a, b *point, collator collate.
return rangePointEqualValueLess(a, b), errors.Trace(err)
}

func rangePointEnumLess(_ *stmtctx.StatementContext, a, b *point) (bool, error) {
func rangePointEnumLess(a, b *point) (bool, error) {
cmp := cmp.Compare(a.value.GetInt64(), b.value.GetInt64())
if cmp != 0 {
return cmp < 0, nil
Expand Down Expand Up @@ -187,12 +187,18 @@ type builder struct {
sc *stmtctx.StatementContext
}

func (r *builder) build(expr expression.Expression, collator collate.Collator) []*point {
// build converts Expression on one column into point, which can be further built into Range.
// The input collator is used for intersection/union between points, which corresponds to AND/OR in the expression. Since
// our (*Datum).Compare(), which is used there, needs an explicit collator input to handle comparison for string and bytes,
// we pass it down from here.
// If the input prefixLen is not types.UnspecifiedLength, it means it's for a prefix column in a prefix index. In such
// cases, we should cut the prefix and adjust the exclusiveness. Ref: cutPrefixForPoints().
func (r *builder) build(expr expression.Expression, collator collate.Collator, prefixLen int) []*point {
switch x := expr.(type) {
case *expression.Column:
return r.buildFromColumn()
case *expression.ScalarFunction:
return r.buildFromScalarFunc(x, collator)
return r.buildFromScalarFunc(x, collator, prefixLen)
case *expression.Constant:
return r.buildFromConstant(x)
}
Expand Down Expand Up @@ -233,7 +239,7 @@ func (*builder) buildFromColumn() []*point {
return []*point{startPoint1, endPoint1, startPoint2, endPoint2}
}

func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point {
func (r *builder) buildFromBinOp(expr *expression.ScalarFunction, prefixLen int) []*point {
// This has been checked that the binary operation is comparison operation, and one of
// the operand is column name expression.
var (
Expand Down Expand Up @@ -339,40 +345,43 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point {
return handleEnumFromBinOp(r.sc, ft, value, op)
}

var res []*point
switch op {
case ast.NullEQ:
if value.IsNull() {
return []*point{{start: true}, {}} // [null, null]
res = []*point{{start: true}, {}} // [null, null]
break
}
fallthrough
case ast.EQ:
startPoint := &point{value: value, start: true}
endPoint := &point{value: value}
return []*point{startPoint, endPoint}
res = []*point{startPoint, endPoint}
case ast.NE:
startPoint1 := &point{value: types.MinNotNullDatum(), start: true}
endPoint1 := &point{value: value, excl: true}
startPoint2 := &point{value: value, start: true, excl: true}
endPoint2 := &point{value: types.MaxValueDatum()}
return []*point{startPoint1, endPoint1, startPoint2, endPoint2}
res = []*point{startPoint1, endPoint1, startPoint2, endPoint2}
case ast.LT:
startPoint := &point{value: types.MinNotNullDatum(), start: true}
endPoint := &point{value: value, excl: true}
return []*point{startPoint, endPoint}
res = []*point{startPoint, endPoint}
case ast.LE:
startPoint := &point{value: types.MinNotNullDatum(), start: true}
endPoint := &point{value: value}
return []*point{startPoint, endPoint}
res = []*point{startPoint, endPoint}
case ast.GT:
startPoint := &point{value: value, start: true, excl: true}
endPoint := &point{value: types.MaxValueDatum()}
return []*point{startPoint, endPoint}
res = []*point{startPoint, endPoint}
case ast.GE:
startPoint := &point{value: value, start: true}
endPoint := &point{value: types.MaxValueDatum()}
return []*point{startPoint, endPoint}
res = []*point{startPoint, endPoint}
}
return nil
cutPrefixForPoints(res, prefixLen, ft)
return res
}

// handleUnsignedCol handles the case when unsigned column meets negative value.
Expand Down Expand Up @@ -554,11 +563,12 @@ func (*builder) buildFromIsFalse(_ *expression.ScalarFunction, isNot int) []*poi
return []*point{startPoint, endPoint}
}

func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) {
func (r *builder) buildFromIn(expr *expression.ScalarFunction, prefixLen int) ([]*point, bool) {
list := expr.GetArgs()[1:]
rangePoints := make([]*point, 0, len(list)*2)
hasNull := false
colCollate := expr.GetArgs()[0].GetType().GetCollate()
ft := expr.GetArgs()[0].GetType()
colCollate := ft.GetCollate()
for _, e := range list {
v, ok := e.(*expression.Constant)
if !ok {
Expand Down Expand Up @@ -630,10 +640,12 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool)
if curPos > 0 {
curPos++
}
return rangePoints[:curPos], hasNull
rangePoints = rangePoints[:curPos]
cutPrefixForPoints(rangePoints, prefixLen, ft)
return rangePoints, hasNull
}

func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*point {
func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction, prefixLen int) []*point {
_, collation := expr.CharsetAndCollation()
if !collate.CompatibleCollate(expr.GetArgs()[0].GetType().GetCollate(), collation) {
return getFullRange()
Expand All @@ -652,7 +664,8 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po
if pattern == "" {
startPoint := &point{value: types.NewStringDatum(""), start: true}
endPoint := &point{value: types.NewStringDatum("")}
return []*point{startPoint, endPoint}
res := []*point{startPoint, endPoint}
return res
}
lowValue := make([]byte, 0, len(pattern))
edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(chunk.Row{})
Expand Down Expand Up @@ -698,10 +711,15 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po
}
if isExactMatch {
val := types.NewCollationStringDatum(string(lowValue), tpOfPattern.GetCollate())
return []*point{{value: val, start: true}, {value: val}}
startPoint := &point{value: val, start: true}
endPoint := &point{value: val}
res := []*point{startPoint, endPoint}
cutPrefixForPoints(res, prefixLen, tpOfPattern)
return res
}
startPoint := &point{start: true, excl: exclude}
startPoint.value.SetBytesAsString(lowValue, tpOfPattern.GetCollate(), uint32(tpOfPattern.GetFlen()))
cutPrefixForPoints([]*point{startPoint}, prefixLen, tpOfPattern)
highValue := make([]byte, len(lowValue))
copy(highValue, lowValue)
endPoint := &point{excl: true}
Expand Down Expand Up @@ -730,7 +748,7 @@ func isPadSpaceCollation(collation string) bool {
return collation != charset.CollationBin
}

func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point {
func (r *builder) buildFromNot(expr *expression.ScalarFunction, prefixLen int) []*point {
switch n := expr.FuncName.L; n {
case ast.IsTruthWithoutNull:
return r.buildFromIsTrue(expr, 1, false)
Expand All @@ -743,7 +761,7 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point {
isUnsignedIntCol bool
nonNegativePos int
)
rangePoints, hasNull := r.buildFromIn(expr)
rangePoints, hasNull := r.buildFromIn(expr, types.UnspecifiedLength)
if hasNull {
return nil
}
Expand All @@ -769,6 +787,7 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point {
// Append the interval (last element, max value].
retRangePoints = append(retRangePoints, &point{value: previousValue, start: true, excl: true})
retRangePoints = append(retRangePoints, &point{value: types.MaxValueDatum()})
cutPrefixForPoints(retRangePoints, prefixLen, expr.GetArgs()[0].GetType())
return retRangePoints
case ast.Like:
// Pattern not like is not supported.
Expand All @@ -785,31 +804,31 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point {
return getFullRange()
}

func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator collate.Collator) []*point {
func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator collate.Collator, prefixLen int) []*point {
switch op := expr.FuncName.L; op {
case ast.GE, ast.GT, ast.LT, ast.LE, ast.EQ, ast.NE, ast.NullEQ:
return r.buildFromBinOp(expr)
return r.buildFromBinOp(expr, prefixLen)
case ast.LogicAnd:
return r.intersection(r.build(expr.GetArgs()[0], collator), r.build(expr.GetArgs()[1], collator), collator)
return r.intersection(r.build(expr.GetArgs()[0], collator, prefixLen), r.build(expr.GetArgs()[1], collator, prefixLen), collator)
case ast.LogicOr:
return r.union(r.build(expr.GetArgs()[0], collator), r.build(expr.GetArgs()[1], collator), collator)
return r.union(r.build(expr.GetArgs()[0], collator, prefixLen), r.build(expr.GetArgs()[1], collator, prefixLen), collator)
case ast.IsTruthWithoutNull:
return r.buildFromIsTrue(expr, 0, false)
case ast.IsTruthWithNull:
return r.buildFromIsTrue(expr, 0, true)
case ast.IsFalsity:
return r.buildFromIsFalse(expr, 0)
case ast.In:
retPoints, _ := r.buildFromIn(expr)
retPoints, _ := r.buildFromIn(expr, prefixLen)
return retPoints
case ast.Like:
return r.newBuildFromPatternLike(expr)
return r.newBuildFromPatternLike(expr, prefixLen)
case ast.IsNull:
startPoint := &point{start: true}
endPoint := &point{}
return []*point{startPoint, endPoint}
case ast.UnaryNot:
return r.buildFromNot(expr.GetArgs()[0].(*expression.ScalarFunction))
return r.buildFromNot(expr.GetArgs()[0].(*expression.ScalarFunction), prefixLen)
}

return nil
Expand Down
82 changes: 25 additions & 57 deletions pkg/util/ranger/ranger.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,7 +414,7 @@ func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx.
rangePoints := getFullRange()
for _, cond := range accessConditions {
collator := collate.GetCollator(tp.GetCollate())
rangePoints = rb.intersection(rangePoints, rb.build(cond, collator), collator)
rangePoints = rb.intersection(rangePoints, rb.build(cond, collator, colLen), collator)
if rb.err != nil {
return nil, nil, nil, errors.Trace(rb.err)
}
Expand All @@ -438,17 +438,6 @@ func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx.
return ranges, nil, accessConditions, nil
}
if colLen != types.UnspecifiedLength {
for _, ran := range ranges {
// If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false.
// For example, `col_varchar > 'xx'` should be converted to range [xx, +inf) when the prefix index length of
// `col_varchar` is 2. Otherwise we would miss values like 'xxx' if we execute (xx, +inf) index range scan.
if CutDatumByPrefixLen(&ran.LowVal[0], colLen, tp) || ReachPrefixLen(&ran.LowVal[0], colLen, tp) {
ran.LowExclude = false
}
if CutDatumByPrefixLen(&ran.HighVal[0], colLen, tp) {
ran.HighExclude = false
}
}
ranges, err = UnionRanges(sctx, ranges, true)
if err != nil {
return nil, nil, nil, err
Expand Down Expand Up @@ -492,7 +481,7 @@ func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAn
)
for i := 0; i < eqAndInCount; i++ {
// Build ranges for equal or in access conditions.
point := rb.build(accessConds[i], collate.GetCollator(newTp[i].GetCollate()))
point := rb.build(accessConds[i], collate.GetCollator(newTp[i].GetCollate()), d.lengths[i])
if rb.err != nil {
return nil, nil, nil, errors.Trace(rb.err)
}
Expand All @@ -513,7 +502,7 @@ func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAn
// Build rangePoints for non-equal access conditions.
for i := eqAndInCount; i < len(accessConds); i++ {
collator := collate.GetCollator(newTp[eqAndInCount].GetCollate())
rangePoints = rb.intersection(rangePoints, rb.build(accessConds[i], collator), collator)
rangePoints = rb.intersection(rangePoints, rb.build(accessConds[i], collator, d.lengths[eqAndInCount]), collator)
if rb.err != nil {
return nil, nil, nil, errors.Trace(rb.err)
}
Expand Down Expand Up @@ -543,11 +532,9 @@ func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, eqAndInCoun

// Take prefix index into consideration.
if hasPrefix(d.lengths) {
if fixPrefixColRange(ranges, d.lengths, newTp) {
ranges, err = UnionRanges(d.sctx, ranges, d.mergeConsecutive)
if err != nil {
return nil, nil, nil, errors.Trace(err)
}
ranges, err = UnionRanges(d.sctx, ranges, d.mergeConsecutive)
if err != nil {
return nil, nil, nil, errors.Trace(err)
}
}

Expand Down Expand Up @@ -618,47 +605,28 @@ func hasPrefix(lengths []int) bool {
return false
}

// fixPrefixColRange checks whether the range of one column exceeds the length and needs to be cut.
// It specially handles the last column of each range point. If the last one need to be cut, it will
// change the exclude status of that point and return `true` to tell
// that we need do a range merging since that interval may have intersection.
// e.g. if the interval is (-inf -inf, a xxxxx), (a xxxxx, +inf +inf) and the length of the last column is 3,
//
// then we'll change it to (-inf -inf, a xxx], [a xxx, +inf +inf). You can see that this two interval intersect,
// so we need a merge operation.
//
// Q: only checking the last column to decide whether the endpoint's exclude status needs to be reset is enough?
// A: Yes, suppose that the interval is (-inf -inf, a xxxxx b) and only the second column needs to be cut.
//
// The result would be (-inf -inf, a xxx b) if the length of it is 3. Obviously we only need to care about the data
// whose the first two key is `a` and `xxx`. It read all data whose index value begins with `a` and `xxx` and the third
// value less than `b`, covering the values begin with `a` and `xxxxx` and the third value less than `b` perfectly.
// So in this case we don't need to reset its exclude status. The right endpoint case can be proved in the same way.
func fixPrefixColRange(ranges Ranges, lengths []int, tp []*types.FieldType) bool {
var hasCut bool
for _, ran := range ranges {
lowTail := len(ran.LowVal) - 1
for i := 0; i < lowTail; i++ {
hasCut = CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) || hasCut
}
lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail])
// If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false.
// For example, `col_varchar > 'xx'` should be converted to range [xx, +inf) when the prefix index length of
// `col_varchar` is 2. Otherwise we would miss values like 'xxx' if we execute (xx, +inf) index range scan.
if lowCut || ReachPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail]) {
ran.LowExclude = false
}
highTail := len(ran.HighVal) - 1
for i := 0; i < highTail; i++ {
hasCut = CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) || hasCut
// cutPrefixForPoints cuts the prefix of points according to the prefix length of the prefix index.
// It may modify the point.value and point.excl. The modification is in-place.
// This function doesn't require the start and end points to be paired in the input.
func cutPrefixForPoints(points []*point, length int, tp *types.FieldType) {
if length == types.UnspecifiedLength {
return
}
for _, p := range points {
if p == nil {
continue
}
highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail])
if highCut {
ran.HighExclude = false
cut := CutDatumByPrefixLen(&p.value, length, tp)
// In two cases, we need to convert the exclusive point to an inclusive point.
// case 1: we actually cut the value to accommodate the prefix index.
if cut ||
// case 2: the value is already equal to the prefix index.
// For example, col_varchar > 'xx' should be converted to range [xx, +inf) when the prefix index length of
// `col_varchar` is 2. Otherwise, we would miss values like 'xxx' if we execute (xx, +inf) index range scan.
(p.start && ReachPrefixLen(&p.value, length, tp)) {
p.excl = false
}
hasCut = hasCut || lowCut || highCut
}
return hasCut
}

// CutDatumByPrefixLen cuts the datum according to the prefix length.
Expand Down
Loading

0 comments on commit 26af280

Please sign in to comment.