Skip to content

Commit

Permalink
Merge branch 'release-4.0' into release-4.0-09ebd0d703c1
Browse files Browse the repository at this point in the history
  • Loading branch information
lzmhhh123 authored Sep 13, 2021
2 parents 6739c74 + 842cb0c commit 105d8ce
Show file tree
Hide file tree
Showing 39 changed files with 942 additions and 192 deletions.
16 changes: 12 additions & 4 deletions executor/aggfuncs/func_count_distinct.go
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,10 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
p := (*partialResult4CountWithDistinct)(pr)

encodedBytes := make([]byte, 0)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)

Expand All @@ -304,7 +308,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return
}
Expand All @@ -325,7 +329,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,

// evalAndEncode eval one row with an expression and encode value to bytes.
func evalAndEncode(
sctx sessionctx.Context, arg expression.Expression,
sctx sessionctx.Context, arg expression.Expression, collator collate.Collator,
row chunk.Row, buf, encodedBytes []byte,
) (_ []byte, isNull bool, err error) {
switch tp := arg.GetType().EvalType(); tp {
Expand Down Expand Up @@ -377,7 +381,7 @@ func evalAndEncode(
if err != nil || isNull {
break
}
encodedBytes = codec.EncodeCompactBytes(encodedBytes, hack.Slice(val))
encodedBytes = codec.EncodeCompactBytes(encodedBytes, collator.Key(val))
default:
return nil, false, errors.Errorf("unsupported column type for encode %d", tp)
}
Expand Down Expand Up @@ -756,13 +760,17 @@ func (e *approxCountDistinctOriginal) UpdatePartialResult(sctx sessionctx.Contex
encodedBytes := make([]byte, 0)
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}

for _, row := range rowsInGroup {
var hasNull, isNull bool
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return
}
Expand Down
21 changes: 16 additions & 5 deletions executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/set"
)

Expand Down Expand Up @@ -175,18 +175,24 @@ func (e *groupConcatDistinct) ResetPartialResult(pr PartialResult) {
func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) {
p := (*partialResult4GroupConcatDistinct)(pr)
v, isNull := "", false

collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}

for _, row := range rowsInGroup {
p.valsBuf.Reset()
p.encodeBytesBuffer = p.encodeBytesBuffer[:0]
for _, arg := range e.args {
for i, arg := range e.args {
v, isNull, err = arg.EvalString(sctx, row)
if err != nil {
return err
}
if isNull {
break
}
p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, hack.Slice(v))
p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, collators[i].Key(v))
p.valsBuf.WriteString(v)
}
if isNull {
Expand Down Expand Up @@ -478,18 +484,23 @@ func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context,
p := (*partialResult4GroupConcatOrderDistinct)(pr)
p.topN.sctx = sctx
v, isNull := "", false
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}

for _, row := range rowsInGroup {
buffer := new(bytes.Buffer)
p.encodeBytesBuffer = p.encodeBytesBuffer[:0]
for _, arg := range e.args {
for i, arg := range e.args {
v, isNull, err = arg.EvalString(sctx, row)
if err != nil {
return err
}
if isNull {
break
}
p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, hack.Slice(v))
p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, collators[i].Key(v))
buffer.WriteString(v)
}
if isNull {
Expand Down
13 changes: 13 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -648,6 +648,19 @@ func (s *testSuite1) TestDefaultValForAnalyze(c *C) {
"└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false"))
}

func (s *testSerialSuite2) TestIssue27429(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table test.t(id int, value varchar(20) charset utf8mb4 collate utf8mb4_general_ci, value1 varchar(20) charset utf8mb4 collate utf8mb4_bin)")
tk.MustExec("insert into test.t values (1, 'abc', 'abc '),(4, 'Abc', 'abc'),(3,'def', 'def ');")

tk.MustQuery("select upper(group_concat(distinct value order by 1)) from test.t;").Check(testkit.Rows("ABC,DEF"))
tk.MustQuery("select upper(group_concat(distinct value)) from test.t;").Check(testkit.Rows("ABC,DEF"))
}

func (s *testSerialSuite2) TestIssue20874(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
Expand Down
9 changes: 9 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7107,3 +7107,12 @@ func (s *testSerialSuite1) TestIssue24210(c *C) {
err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError")
c.Assert(err, IsNil)
}

func (s *testSuite) TestIssue26532(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustQuery("select greatest(cast(\"2020-01-01 01:01:01\" as datetime), cast(\"2019-01-01 01:01:01\" as datetime) )union select null;").Sort().Check(testkit.Rows("2020-01-01 01:01:01", "<nil>"))
tk.MustQuery("select least(cast(\"2020-01-01 01:01:01\" as datetime), cast(\"2019-01-01 01:01:01\" as datetime) )union select null;").Sort().Check(testkit.Rows("2019-01-01 01:01:01", "<nil>"))
tk.MustQuery("select greatest(\"2020-01-01 01:01:01\" ,\"2019-01-01 01:01:01\" )union select null;").Sort().Check(testkit.Rows("2020-01-01 01:01:01", "<nil>"))
tk.MustQuery("select least(\"2020-01-01 01:01:01\" , \"2019-01-01 01:01:01\" )union select null;").Sort().Check(testkit.Rows("2019-01-01 01:01:01", "<nil>"))
}
17 changes: 17 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1309,3 +1309,20 @@ func (s *testSuite9) TestBinaryLiteralInsertToSet(c *C) {
tk.MustExec("insert into bintest(h) values(0x61)")
tk.MustQuery("select * from bintest").Check(testkit.Rows("a"))
}

func (s *testSuite9) TestIssue26762(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(c1 date);")
_, err := tk.Exec("insert into t1 values('2020-02-31');")
c.Assert(err.Error(), Equals, `[table:1366]Incorrect date value: '2020-02-31' for column 'c1' at row 1`)

tk.MustExec("set @@sql_mode='ALLOW_INVALID_DATES';")
tk.MustExec("insert into t1 values('2020-02-31');")
tk.MustQuery("select * from t1").Check(testkit.Rows("2020-02-31"))

tk.MustExec("set @@sql_mode='STRICT_TRANS_TABLES';")
_, err = tk.Exec("insert into t1 values('2020-02-31');")
c.Assert(err.Error(), Equals, `[table:1366]Incorrect date value: '2020-02-31' for column 'c1' at row 1`)
}
9 changes: 5 additions & 4 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, funcName string, args []Expressi
if ctx == nil {
return baseBuiltinFunc{}, errors.New("unexpected nil session ctx")
}
if err := checkIllegalMixCollation(funcName, args, retType); err != nil {
if err := CheckIllegalMixCollation(funcName, args, retType); err != nil {
return baseBuiltinFunc{}, err
}
derivedCharset, derivedCollate := DeriveCollationFromExprs(ctx, args...)
Expand All @@ -113,7 +113,8 @@ var (
coerString = []string{"EXPLICIT", "NONE", "IMPLICIT", "SYSCONST", "COERCIBLE", "NUMERIC", "IGNORABLE"}
)

func checkIllegalMixCollation(funcName string, args []Expression, evalType types.EvalType) error {
// CheckIllegalMixCollation check the if the aggregate expression is legal.
func CheckIllegalMixCollation(funcName string, args []Expression, evalType types.EvalType) error {
if len(args) < 2 {
return nil
}
Expand All @@ -132,7 +133,7 @@ func illegalMixCollationErr(funcName string, args []Expression) error {
case 2:
return collate.ErrIllegalMix2Collation.GenWithStackByArgs(args[0].GetType().Collate, coerString[args[0].Coercibility()], args[1].GetType().Collate, coerString[args[1].Coercibility()], funcName)
case 3:
return collate.ErrIllegalMix3Collation.GenWithStackByArgs(args[0].GetType().Collate, coerString[args[0].Coercibility()], args[1].GetType().Collate, coerString[args[1].Coercibility()], args[0].GetType().Collate, coerString[args[2].Coercibility()], funcName)
return collate.ErrIllegalMix3Collation.GenWithStackByArgs(args[0].GetType().Collate, coerString[args[0].Coercibility()], args[1].GetType().Collate, coerString[args[1].Coercibility()], args[2].GetType().Collate, coerString[args[2].Coercibility()], funcName)
default:
return collate.ErrIllegalMixCollation.GenWithStackByArgs(funcName)
}
Expand Down Expand Up @@ -170,7 +171,7 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Ex
}
}

if err = checkIllegalMixCollation(funcName, args, retType); err != nil {
if err = CheckIllegalMixCollation(funcName, args, retType); err != nil {
return
}

Expand Down
15 changes: 15 additions & 0 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -485,9 +485,23 @@ func (c *greatestFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
sig = &builtinGreatestTimeSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_GreatestTime)
}
sig.getRetTp().Flen, sig.getRetTp().Decimal = fixFlenAndDecimalForGreatestAndLeast(args)
return sig, nil
}

func fixFlenAndDecimalForGreatestAndLeast(args []Expression) (flen, decimal int) {
for _, arg := range args {
argFlen, argDecimal := arg.GetType().Flen, arg.GetType().Decimal
if argFlen > flen {
flen = argFlen
}
if argDecimal > decimal {
decimal = argDecimal
}
}
return flen, decimal
}

type builtinGreatestIntSig struct {
baseBuiltinFunc
}
Expand Down Expand Up @@ -702,6 +716,7 @@ func (c *leastFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi
sig = &builtinLeastTimeSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_LeastTime)
}
sig.getRetTp().Flen, sig.getRetTp().Decimal = fixFlenAndDecimalForGreatestAndLeast(args)
return sig, nil
}

Expand Down
9 changes: 8 additions & 1 deletion expression/builtin_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,14 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
}
fieldTp.Decimal, fieldTp.Flen = decimal, flen
if fieldTp.EvalType().IsStringKind() && !isBinaryStr {
fieldTp.Charset, fieldTp.Collate = charset.CharsetUTF8MB4, charset.CollationUTF8MB4
fieldTp.Charset, fieldTp.Collate = DeriveCollationFromExprs(ctx, args...)
if fieldTp.Charset == charset.CharsetBin && fieldTp.Collate == charset.CollationBin {
// When args are Json and Numerical type(eg. Int), the fieldTp is String.
// Both their charset/collation is binary, but the String need a default charset/collation.
fieldTp.Charset, fieldTp.Collate = charset.GetDefaultCharsetAndCollate()
}
} else {
fieldTp.Charset, fieldTp.Collate = charset.CharsetBin, charset.CollationBin
}
if isBinaryFlag {
fieldTp.Flag |= mysql.BinaryFlag
Expand Down
2 changes: 1 addition & 1 deletion expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp
sf.FuncName.L == ast.If ||
sf.FuncName.L == ast.Case ||
sf.FuncName.L == ast.NullEQ) {
return false, false, cond
return false, true, cond
}
for idx, expr := range sf.GetArgs() {
if src.Equal(nil, expr) {
Expand Down
2 changes: 1 addition & 1 deletion expression/expr_to_pb_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -558,7 +558,7 @@ func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) {
pbExprs, err := ExpressionsToPBList(sc, controlFuncs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4101,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
"null",
Expand Down
Loading

0 comments on commit 105d8ce

Please sign in to comment.