Skip to content

Commit

Permalink
executor: consider the collation in vecGroupChecker (#14859)
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Feb 21, 2020
1 parent 27b280d commit 848dd01
Show file tree
Hide file tree
Showing 3 changed files with 101 additions and 6 deletions.
4 changes: 2 additions & 2 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -1213,9 +1213,9 @@ func (e *vecGroupChecker) evalGroupItemsAndResolveGroups(item expression.Express
firstRowDatum.SetMysqlJSON(col.GetJSON(0).Copy())
lastRowDatum.SetMysqlJSON(col.GetJSON(numRows - 1).Copy())
case types.ETString:
previousKey := col.GetString(0)
previousKey := codec.ConvertByCollationStr(col.GetString(0), tp)
for i := 1; i < numRows; i++ {
key := col.GetString(i)
key := codec.ConvertByCollationStr(col.GetString(i), tp)
isNull := col.IsNull(i)
if e.sameGroup[i] {
if isNull != previousIsNull || previousKey != key {
Expand Down
88 changes: 88 additions & 0 deletions executor/collation_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/mock"
)

var _ = SerialSuites(&testCollationSuite{})

type testCollationSuite struct {
}

func (s *testCollationSuite) TestVecGroupChecker(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)

tp := &types.FieldType{Tp: mysql.TypeVarchar}
col0 := &expression.Column{
RetType: tp,
Index: 0,
}
ctx := mock.NewContext()
groupChecker := newVecGroupChecker(ctx, []expression.Expression{col0})

chk := chunk.New([]*types.FieldType{tp}, 6, 6)
chk.Reset()
chk.Column(0).AppendString("aaa")
chk.Column(0).AppendString("AAA")
chk.Column(0).AppendString("😜")
chk.Column(0).AppendString("😃")
chk.Column(0).AppendString("À")
chk.Column(0).AppendString("A")

tp.Collate = "bin"
groupChecker.reset()
_, err := groupChecker.splitIntoGroups(chk)
c.Assert(err, IsNil)
for i := 0; i < 6; i++ {
b, e := groupChecker.getNextGroup()
c.Assert(b, Equals, i)
c.Assert(e, Equals, i+1)
}
c.Assert(groupChecker.isExhausted(), IsTrue)

tp.Collate = "utf8_general_ci"
groupChecker.reset()
_, err = groupChecker.splitIntoGroups(chk)
c.Assert(err, IsNil)
for i := 0; i < 3; i++ {
b, e := groupChecker.getNextGroup()
c.Assert(b, Equals, i*2)
c.Assert(e, Equals, i*2+2)
}
c.Assert(groupChecker.isExhausted(), IsTrue)

// test padding
tp.Collate = "utf8_bin"
tp.Flen = 6
chk.Reset()
chk.Column(0).AppendString("a")
chk.Column(0).AppendString("a ")
chk.Column(0).AppendString("a ")
groupChecker.reset()
_, err = groupChecker.splitIntoGroups(chk)
c.Assert(err, IsNil)
b, e := groupChecker.getNextGroup()
c.Assert(b, Equals, 0)
c.Assert(e, Equals, 3)
c.Assert(groupChecker.isExhausted(), IsTrue)
}
15 changes: 11 additions & 4 deletions util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -308,7 +308,7 @@ func encodeHashChunkRowIdx(sc *stmtctx.StatementContext, row chunk.Row, tp *type
case mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob:
flag = compactBytesFlag
b = row.GetBytes(idx)
b = convertByCollation(b, tp)
b = ConvertByCollation(b, tp)
case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp:
flag = uintFlag
t := row.GetTime(idx)
Expand Down Expand Up @@ -446,7 +446,7 @@ func HashChunkSelected(sc *stmtctx.StatementContext, h []hash.Hash64, chk *chunk
} else {
buf[0] = compactBytesFlag
b = column.GetBytes(i)
b = convertByCollation(b, tp)
b = ConvertByCollation(b, tp)
}

// As the golang doc described, `Hash.Write` never returns an error.
Expand Down Expand Up @@ -1166,7 +1166,7 @@ func HashGroupKey(sc *stmtctx.StatementContext, n int, col *chunk.Column, buf []
if col.IsNull(i) {
buf[i] = append(buf[i], NilFlag)
} else {
buf[i] = encodeBytes(buf[i], convertByCollation(col.GetBytes(i), ft), false)
buf[i] = encodeBytes(buf[i], ConvertByCollation(col.GetBytes(i), ft), false)
}
}
default:
Expand All @@ -1175,7 +1175,14 @@ func HashGroupKey(sc *stmtctx.StatementContext, n int, col *chunk.Column, buf []
return buf, nil
}

func convertByCollation(raw []byte, tp *types.FieldType) []byte {
// ConvertByCollation converts these bytes according to its collation.
func ConvertByCollation(raw []byte, tp *types.FieldType) []byte {
collator := collate.GetCollator(tp.Collate)
return collator.Key(string(hack.String(raw)), collate.NewCollatorOption(tp.Flen))
}

// ConvertByCollationStr converts this string according to its collation.
func ConvertByCollationStr(str string, tp *types.FieldType) string {
collator := collate.GetCollator(tp.Collate)
return string(hack.String(collator.Key(str, collate.NewCollatorOption(tp.Flen))))
}

0 comments on commit 848dd01

Please sign in to comment.