Skip to content

Commit

Permalink
*: code preparations for supporting build range for like function on …
Browse files Browse the repository at this point in the history
…new collation columns (#48972)

ref #48181
  • Loading branch information
time-and-fate authored Nov 30, 2023
1 parent 1c6e0ee commit 598ccad
Show file tree
Hide file tree
Showing 16 changed files with 1,377 additions and 98 deletions.
6 changes: 4 additions & 2 deletions pkg/executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -540,11 +540,13 @@ func EncodeUniqueIndexValuesForKey(ctx sessionctx.Context, tblInfo *model.TableI
colInfo := tblInfo.Columns[idxInfo.Columns[i].Offset]
// table.CastValue will append 0x0 if the string value's length is smaller than the BINARY column's length.
// So we don't use CastValue for string value for now.
// TODO: merge two if branch.
// TODO: The first if branch should have been removed, because the functionality of set the collation of the datum
// have been moved to util/ranger (normal path) and getNameValuePairs/getPointGetValue (fast path). But this change
// will be cherry-picked to a hotfix, so we choose to be a bit conservative and keep this for now.
if colInfo.GetType() == mysql.TypeString || colInfo.GetType() == mysql.TypeVarString || colInfo.GetType() == mysql.TypeVarchar {
var str string
str, err = idxVals[i].ToString()
idxVals[i].SetString(str, colInfo.FieldType.GetCollate())
idxVals[i].SetString(str, idxVals[i].Collation())
} else if colInfo.GetType() == mysql.TypeEnum && (idxVals[i].Kind() == types.KindString || idxVals[i].Kind() == types.KindBytes || idxVals[i].Kind() == types.KindBinaryLiteral) {
var str string
var e types.Enum
Expand Down
15 changes: 14 additions & 1 deletion pkg/planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -1438,7 +1438,15 @@ func getNameValuePairs(ctx sessionctx.Context, tbl *model.TableInfo, tblName mod
col := model.FindColumnInfo(tbl.Cols(), colName.Name.Name.L)
if col == nil { // Handling the case when the column is _tidb_rowid.
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: types.NewFieldType(mysql.TypeLonglong), value: d, con: con}), false
} else if col.GetType() == mysql.TypeString && col.GetCollate() == charset.CollationBin { // This type we needn't to pad `\0` in here.
}

// As in buildFromBinOp in util/ranger, when we build key from the expression to do range scan or point get on
// a string column, we should set the collation of the string datum to collation of the column.
if col.FieldType.EvalType() == types.ETString && (d.Kind() == types.KindString || d.Kind() == types.KindBinaryLiteral) {
d.SetString(d.GetString(), col.FieldType.GetCollate())
}

if col.GetType() == mysql.TypeString && col.GetCollate() == charset.CollationBin { // This type we needn't to pad `\0` in here.
return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: &col.FieldType, value: d, con: con}), false
}
if !checkCanConvertInPointGet(col, d) {
Expand Down Expand Up @@ -1468,6 +1476,11 @@ func getPointGetValue(stmtCtx *stmtctx.StatementContext, col *model.ColumnInfo,
if !checkCanConvertInPointGet(col, *d) {
return nil
}
// As in buildFromBinOp in util/ranger, when we build key from the expression to do range scan or point get on
// a string column, we should set the collation of the string datum to collation of the column.
if col.FieldType.EvalType() == types.ETString && (d.Kind() == types.KindString || d.Kind() == types.KindBinaryLiteral) {
d.SetString(d.GetString(), col.FieldType.GetCollate())
}
dVal, err := d.ConvertTo(stmtCtx.TypeCtx(), &col.FieldType)
if err != nil {
return nil
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/testdata/index_merge_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,8 @@
"Plan": [
"Selection 1.42 root eq(test.t8.s5, \"test,2\")",
"└─IndexMerge 0.59 root type: intersection",
" ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(0x616263,+inf], keep order:false, stats:pseudo",
" ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(0x636261,+inf], keep order:false, stats:pseudo",
" ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(\"abc\",+inf], keep order:false, stats:pseudo",
" ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(\"cba\",+inf], keep order:false, stats:pseudo",
" ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t8, index:is4(s4) range:[\"aA\",\"aA\"], [\"??\",\"??\"], keep order:false, stats:pseudo",
" └─Selection(Probe) 0.59 cop[tikv] gt(test.t8.s3, \"cba\"), like(test.t8.s1, \"啊A%\", 92)",
" └─TableRowIDScan 2.22 cop[tikv] table:t8 keep order:false, stats:pseudo"
Expand Down
12 changes: 6 additions & 6 deletions pkg/util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ func unionColumnValues(lhs, rhs []*valueInfo) []*valueInfo {
// detachCNFCondAndBuildRangeForIndex will detach the index filters from table filters. These conditions are connected with `and`
// It will first find the point query column and then extract the range query column.
// considerDNF is true means it will try to extract access conditions from the DNF expressions.
func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expression.Expression, tpSlice []*types.FieldType, considerDNF bool) (*DetachRangeResult, error) {
func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expression.Expression, newTpSlice []*types.FieldType, considerDNF bool) (*DetachRangeResult, error) {
var (
eqCount int
ranges Ranges
Expand All @@ -316,7 +316,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi
return res, nil
}
var remainedConds []expression.Expression
ranges, accessConds, remainedConds, err = d.buildRangeOnColsByCNFCond(tpSlice, len(accessConds), accessConds)
ranges, accessConds, remainedConds, err = d.buildRangeOnColsByCNFCond(newTpSlice, len(accessConds), accessConds)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -452,7 +452,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi
}
// `eqOrInCount` must be 0 when coming here.
res.AccessConds, res.RemainedConds = detachColumnCNFConditions(d.sctx, newConditions, checker)
ranges, res.AccessConds, remainedConds, err = d.buildCNFIndexRange(tpSlice, 0, res.AccessConds)
ranges, res.AccessConds, remainedConds, err = d.buildCNFIndexRange(newTpSlice, 0, res.AccessConds)
if err != nil {
return nil, err
}
Expand All @@ -473,7 +473,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi
accessConds = append(accessConds, cond)
// TODO: if it's prefix column, we need to add cond to filterConds?
}
ranges, accessConds, remainedConds, err = d.buildCNFIndexRange(tpSlice, eqOrInCount, accessConds)
ranges, accessConds, remainedConds, err = d.buildCNFIndexRange(newTpSlice, eqOrInCount, accessConds)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -608,7 +608,7 @@ func extractValueInfo(expr expression.Expression) *valueInfo {
func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column,
lengths []int) ([]expression.Expression, []expression.Expression, []expression.Expression, []*valueInfo, bool) {
var filters []expression.Expression
rb := builder{ctx: sctx}
rb := builder{sctx: sctx}
accesses := make([]expression.Expression, len(cols))
points := make([][]*point, len(cols))
mergedAccesses := make([]expression.Expression, len(cols))
Expand Down Expand Up @@ -715,7 +715,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression
optPrefixIndexSingleScan: d.sctx.GetSessionVars().OptPrefixIndexSingleScan,
ctx: d.sctx,
}
rb := builder{ctx: d.sctx}
rb := builder{sctx: d.sctx}
dnfItems := expression.FlattenDNFConditions(condition)
newAccessItems := make([]expression.Expression, 0, len(dnfItems))
var totalRanges Ranges
Expand Down
56 changes: 38 additions & 18 deletions pkg/util/ranger/points.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,8 +183,8 @@ func NullRange() Ranges {

// builder is the range builder struct.
type builder struct {
err error
ctx sessionctx.Context
err error
sctx sessionctx.Context
}

// build converts Expression on one column into point, which can be further built into Range.
Expand All @@ -193,7 +193,11 @@ type builder struct {
// 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 {
func (r *builder) build(
expr expression.Expression,
collator collate.Collator,
prefixLen int,
) []*point {
switch x := expr.(type) {
case *expression.Column:
return r.buildFromColumn()
Expand All @@ -207,7 +211,7 @@ func (r *builder) build(expr expression.Expression, collator collate.Collator, p
}

func (r *builder) buildFromConstant(expr *expression.Constant) []*point {
dt, err := expr.Eval(r.ctx, chunk.Row{})
dt, err := expr.Eval(r.sctx, chunk.Row{})
if err != nil {
r.err = err
return nil
Expand All @@ -216,7 +220,7 @@ func (r *builder) buildFromConstant(expr *expression.Constant) []*point {
return nil
}

tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx()
tc := r.sctx.GetSessionVars().StmtCtx.TypeCtx()
val, err := dt.ToBool(tc)
if err != nil {
r.err = err
Expand All @@ -240,7 +244,10 @@ func (*builder) buildFromColumn() []*point {
return []*point{startPoint1, endPoint1, startPoint2, endPoint2}
}

func (r *builder) buildFromBinOp(expr *expression.ScalarFunction, prefixLen int) []*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 All @@ -250,7 +257,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction, prefixLen int)
ft *types.FieldType
)

tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx()
tc := r.sctx.GetSessionVars().StmtCtx.TypeCtx()
// refineValueAndOp refines the constant datum and operator:
// 1. for string type since we may eval the constant to another collation instead of its own collation.
// 2. for year type since 2-digit year value need adjustment, see https://dev.mysql.com/doc/refman/5.6/en/year.html
Expand Down Expand Up @@ -292,7 +299,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction, prefixLen int)
var ok bool
if col, ok = expr.GetArgs()[0].(*expression.Column); ok {
ft = col.RetType
value, err = expr.GetArgs()[1].Eval(r.ctx, chunk.Row{})
value, err = expr.GetArgs()[1].Eval(r.sctx, chunk.Row{})
if err != nil {
return nil
}
Expand All @@ -303,7 +310,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction, prefixLen int)
return nil
}
ft = col.RetType
value, err = expr.GetArgs()[0].Eval(r.ctx, chunk.Row{})
value, err = expr.GetArgs()[0].Eval(r.sctx, chunk.Row{})
if err != nil {
return nil
}
Expand Down Expand Up @@ -565,20 +572,23 @@ func (*builder) buildFromIsFalse(_ *expression.ScalarFunction, isNot int) []*poi
return []*point{startPoint, endPoint}
}

func (r *builder) buildFromIn(expr *expression.ScalarFunction, prefixLen int) ([]*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
ft := expr.GetArgs()[0].GetType()
colCollate := ft.GetCollate()
tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx()
tc := r.sctx.GetSessionVars().StmtCtx.TypeCtx()
for _, e := range list {
v, ok := e.(*expression.Constant)
if !ok {
r.err = ErrUnsupportedType.GenWithStack("expr:%v is not constant", e)
return getFullRange(), hasNull
}
dt, err := v.Eval(r.ctx, chunk.Row{})
dt, err := v.Eval(r.sctx, chunk.Row{})
if err != nil {
r.err = ErrUnsupportedType.GenWithStack("expr:%v is not evaluated", e)
return getFullRange(), hasNull
Expand Down Expand Up @@ -648,12 +658,15 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction, prefixLen int) ([
return rangePoints, hasNull
}

func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction, prefixLen int) []*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()
}
pdt, err := expr.GetArgs()[1].(*expression.Constant).Eval(r.ctx, chunk.Row{})
pdt, err := expr.GetArgs()[1].(*expression.Constant).Eval(r.sctx, chunk.Row{})
tpOfPattern := expr.GetArgs()[0].GetType()
if err != nil {
r.err = errors.Trace(err)
Expand All @@ -671,7 +684,7 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction, prefi
return res
}
lowValue := make([]byte, 0, len(pattern))
edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(r.ctx, chunk.Row{})
edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(r.sctx, chunk.Row{})
if err != nil {
r.err = errors.Trace(err)
return getFullRange()
Expand Down Expand Up @@ -751,7 +764,10 @@ func isPadSpaceCollation(collation string) bool {
return collation != charset.CollationBin
}

func (r *builder) buildFromNot(expr *expression.ScalarFunction, prefixLen int) []*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 Down Expand Up @@ -807,7 +823,11 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction, prefixLen int) [
return getFullRange()
}

func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator collate.Collator, prefixLen int) []*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, prefixLen)
Expand Down Expand Up @@ -848,7 +868,7 @@ func (r *builder) union(a, b []*point, collator collate.Collator) []*point {
func (r *builder) mergeSorted(a, b []*point, collator collate.Collator) []*point {
ret := make([]*point, 0, len(a)+len(b))
i, j := 0, 0
tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx()
tc := r.sctx.GetSessionVars().StmtCtx.TypeCtx()
for i < len(a) && j < len(b) {
less, err := rangePointLess(tc, a[i], b[j], collator)
if err != nil {
Expand Down
Loading

0 comments on commit 598ccad

Please sign in to comment.