Skip to content

Commit

Permalink
*: replace compareDatum by compare (#30421)
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 committed Dec 6, 2021
1 parent 9d307b0 commit d3c6045
Show file tree
Hide file tree
Showing 9 changed files with 36 additions and 15 deletions.
9 changes: 8 additions & 1 deletion executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,7 @@ type topNRows struct {
// ('---', 'ccc') should be poped from heap, so '-' should be appended to result.
// eg: 'aaa---bbb---ccc' -> 'aaa---bbb-'
isSepTruncated bool
collators []collate.Collator
}

func (h topNRows) Len() int {
Expand All @@ -302,7 +303,7 @@ func (h topNRows) Len() int {
func (h topNRows) Less(i, j int) bool {
n := len(h.rows[i].byItems)
for k := 0; k < n; k++ {
ret, err := h.rows[i].byItems[k].CompareDatum(h.sctx.GetSessionVars().StmtCtx, h.rows[j].byItems[k])
ret, err := h.rows[i].byItems[k].Compare(h.sctx.GetSessionVars().StmtCtx, h.rows[j].byItems[k], h.collators[k])
if err != nil {
h.err = err
return false
Expand Down Expand Up @@ -411,8 +412,10 @@ func (e *groupConcatOrder) AppendFinalResult2Chunk(sctx sessionctx.Context, pr P

func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int64) {
desc := make([]bool, len(e.byItems))
ctors := make([]collate.Collator, 0, len(e.byItems))
for i, byItem := range e.byItems {
desc[i] = byItem.Desc
ctors = append(ctors, collate.GetCollator(byItem.Expr.GetType().Collate))
}
p := &partialResult4GroupConcatOrder{
topN: &topNRows{
Expand All @@ -421,6 +424,7 @@ func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int6
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
isSepTruncated: false,
collators: ctors,
},
}
return PartialResult(p), DefPartialResult4GroupConcatOrderSize + DefTopNRowsSize
Expand Down Expand Up @@ -513,8 +517,10 @@ func (e *groupConcatDistinctOrder) AppendFinalResult2Chunk(sctx sessionctx.Conte

func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDelta int64) {
desc := make([]bool, len(e.byItems))
ctors := make([]collate.Collator, 0, len(e.byItems))
for i, byItem := range e.byItems {
desc[i] = byItem.Desc
ctors = append(ctors, collate.GetCollator(byItem.Expr.GetType().Collate))
}
valSet, setSize := set.NewStringSetWithMemoryUsage()
p := &partialResult4GroupConcatOrderDistinct{
Expand All @@ -524,6 +530,7 @@ func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDe
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
isSepTruncated: false,
collators: ctors,
},
valSet: valSet,
}
Expand Down
6 changes: 6 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/cteutil"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -1059,6 +1060,11 @@ func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannerco
reader = sel.children[0]
}

us.collators = make([]collate.Collator, 0, len(us.columns))
for _, tp := range retTypes(us) {
us.collators = append(us.collators, collate.GetCollator(tp.Collate))
}

switch x := reader.(type) {
case *TableReaderExecutor:
us.desc = x.desc
Expand Down
6 changes: 4 additions & 2 deletions executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
)

// UnionScanExec merges the rows from dirty table and the rows from distsql request.
Expand Down Expand Up @@ -59,6 +60,7 @@ type UnionScanExec struct {

// cacheTable not nil means it's reading from cached table.
cacheTable kv.MemBuffer
collators []collate.Collator
}

// Open implements the Executor Open interface.
Expand Down Expand Up @@ -273,13 +275,13 @@ func (us *UnionScanExec) compare(a, b []types.Datum) (int, error) {
for _, colOff := range us.usedIndex {
aColumn := a[colOff]
bColumn := b[colOff]
cmp, err := aColumn.CompareDatum(sc, &bColumn)
cmp, err := aColumn.Compare(sc, &bColumn, us.collators[colOff])
if err != nil {
return 0, err
}
if cmp != 0 {
return cmp, nil
}
}
return us.belowHandleCols.Compare(a, b)
return us.belowHandleCols.Compare(a, b, us.collators)
}
5 changes: 3 additions & 2 deletions expression/aggregation/aggregation.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tipb/go-tipb"
)

Expand Down Expand Up @@ -68,9 +69,9 @@ func NewDistAggFunc(expr *tipb.Expr, fieldTps []*types.FieldType, sc *stmtctx.St
case tipb.ExprType_GroupConcat:
return &concatFunction{aggFunction: newAggFunc(ast.AggFuncGroupConcat, args, false)}, nil
case tipb.ExprType_Max:
return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMax, args, false), isMax: true}, nil
return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMax, args, false), isMax: true, ctor: collate.GetCollator(args[0].GetType().Collate)}, nil
case tipb.ExprType_Min:
return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMin, args, false)}, nil
return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMin, args, false), ctor: collate.GetCollator(args[0].GetType().Collate)}, nil
case tipb.ExprType_First:
return &firstRowFunction{aggFunction: newAggFunc(ast.AggFuncFirstRow, args, false)}, nil
case tipb.ExprType_Agg_BitOr:
Expand Down
5 changes: 3 additions & 2 deletions expression/aggregation/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
)

// AggFuncDesc describes an aggregation function signature, only used in planner.
Expand Down Expand Up @@ -230,9 +231,9 @@ func (a *AggFuncDesc) GetAggFunc(ctx sessionctx.Context) Aggregation {
}
return &concatFunction{aggFunction: aggFunc, maxLen: maxLen}
case ast.AggFuncMax:
return &maxMinFunction{aggFunction: aggFunc, isMax: true}
return &maxMinFunction{aggFunction: aggFunc, isMax: true, ctor: collate.GetCollator(a.Args[0].GetType().Collate)}
case ast.AggFuncMin:
return &maxMinFunction{aggFunction: aggFunc, isMax: false}
return &maxMinFunction{aggFunction: aggFunc, isMax: false, ctor: collate.GetCollator(a.Args[0].GetType().Collate)}
case ast.AggFuncFirstRow:
return &firstRowFunction{aggFunction: aggFunc}
case ast.AggFuncBitOr:
Expand Down
4 changes: 3 additions & 1 deletion expression/aggregation/max_min.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,13 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
)

type maxMinFunction struct {
aggFunction
isMax bool
ctor collate.Collator
}

// GetResult implements Aggregation interface.
Expand All @@ -49,7 +51,7 @@ func (mmf *maxMinFunction) Update(evalCtx *AggEvaluateContext, sc *stmtctx.State
return nil
}
var c int
c, err = evalCtx.Value.CompareDatum(sc, &value)
c, err = evalCtx.Value.Compare(sc, &value, mmf.ctor)
if err != nil {
return err
}
Expand Down
11 changes: 6 additions & 5 deletions planner/core/handle_cols.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
)

// HandleCols is the interface that holds handle columns.
Expand All @@ -48,7 +49,7 @@ type HandleCols interface {
// NumCols returns the number of columns.
NumCols() int
// Compare compares two datum rows by handle order.
Compare(a, b []types.Datum) (int, error)
Compare(a, b []types.Datum, ctors []collate.Collator) (int, error)
// GetFieldTypes return field types of columns.
GetFieldsTypes() []*types.FieldType
}
Expand Down Expand Up @@ -145,11 +146,11 @@ func (cb *CommonHandleCols) String() string {
}

// Compare implements the kv.HandleCols interface.
func (cb *CommonHandleCols) Compare(a, b []types.Datum) (int, error) {
for _, col := range cb.columns {
func (cb *CommonHandleCols) Compare(a, b []types.Datum, ctors []collate.Collator) (int, error) {
for i, col := range cb.columns {
aDatum := &a[col.Index]
bDatum := &b[col.Index]
cmp, err := aDatum.CompareDatum(cb.sc, bDatum)
cmp, err := aDatum.Compare(cb.sc, bDatum, ctors[i])
if err != nil {
return 0, err
}
Expand Down Expand Up @@ -237,7 +238,7 @@ func (ib *IntHandleCols) NumCols() int {
}

// Compare implements the kv.HandleCols interface.
func (ib *IntHandleCols) Compare(a, b []types.Datum) (int, error) {
func (ib *IntHandleCols) Compare(a, b []types.Datum, ctors []collate.Collator) (int, error) {
aInt := a[ib.col.Index].GetInt64()
bInt := b[ib.col.Index].GetInt64()
if aInt == bInt {
Expand Down
3 changes: 2 additions & 1 deletion util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,7 +448,8 @@ func allSinglePoints(sc *stmtctx.StatementContext, points []*point) []*point {
if !left.start || right.start || left.excl || right.excl {
return nil
}
cmp, err := left.value.CompareDatum(sc, &right.value)
// Since the point's collations are equal to the column's collation, we can use any of them.
cmp, err := left.value.Compare(sc, &right.value, collate.GetCollator(left.value.Collation()))
if err != nil || cmp != 0 {
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion util/ranger/points.go
Original file line number Diff line number Diff line change
Expand Up @@ -467,7 +467,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val
}

d := types.NewCollateMysqlEnumDatum(tmpEnum, ft.Collate)
if v, err := d.CompareDatum(sc, &val); err == nil {
if v, err := d.Compare(sc, &val, collate.GetCollator(ft.Collate)); err == nil {
switch op {
case ast.LT:
if v < 0 {
Expand Down

0 comments on commit d3c6045

Please sign in to comment.