diff --git a/expression/expression.go b/expression/expression.go index e487cf3a8ca5e..77c8bb0f57ad2 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -176,6 +176,352 @@ func EvalBool(ctx sessionctx.Context, exprList CNFExprs, row chunk.Row) (bool, b return true, false, nil } +<<<<<<< HEAD +======= +var ( + defaultChunkSize = 1024 + selPool = sync.Pool{ + New: func() interface{} { + return make([]int, defaultChunkSize) + }, + } + zeroPool = sync.Pool{ + New: func() interface{} { + return make([]int8, defaultChunkSize) + }, + } +) + +func allocSelSlice(n int) []int { + if n > defaultChunkSize { + return make([]int, n) + } + return selPool.Get().([]int) +} + +func deallocateSelSlice(sel []int) { + if cap(sel) <= defaultChunkSize { + selPool.Put(sel) + } +} + +func allocZeroSlice(n int) []int8 { + if n > defaultChunkSize { + return make([]int8, n) + } + return zeroPool.Get().([]int8) +} + +func deallocateZeroSlice(isZero []int8) { + if cap(isZero) <= defaultChunkSize { + zeroPool.Put(isZero) + } +} + +// VecEvalBool does the same thing as EvalBool but it works in a vectorized manner. +func VecEvalBool(ctx sessionctx.Context, exprList CNFExprs, input *chunk.Chunk, selected, nulls []bool) ([]bool, []bool, error) { + // If input.Sel() != nil, we will call input.SetSel(nil) to clear the sel slice in input chunk. + // After the function finished, then we reset the input.Sel(). + // The caller will handle the input.Sel() and selected slices. + defer input.SetSel(input.Sel()) + input.SetSel(nil) + + n := input.NumRows() + selected = selected[:0] + nulls = nulls[:0] + for i := 0; i < n; i++ { + selected = append(selected, false) + nulls = append(nulls, false) + } + + sel := allocSelSlice(n) + defer deallocateSelSlice(sel) + sel = sel[:0] + for i := 0; i < n; i++ { + sel = append(sel, i) + } + input.SetSel(sel) + + // In isZero slice, -1 means Null, 0 means zero, 1 means not zero + isZero := allocZeroSlice(n) + defer deallocateZeroSlice(isZero) + for _, expr := range exprList { + eType := expr.GetType().EvalType() + buf, err := globalColumnAllocator.get(eType, n) + if err != nil { + return nil, nil, err + } + + if err := EvalExpr(ctx, expr, input, buf); err != nil { + return nil, nil, err + } + + err = toBool(ctx.GetSessionVars().StmtCtx, eType, buf, sel, isZero) + if err != nil { + return nil, nil, err + } + + j := 0 + isEQCondFromIn := IsEQCondFromIn(expr) + for i := range sel { + if isZero[i] == -1 { + if eType != types.ETInt && !isEQCondFromIn { + continue + } + // In this case, we set this row to null and let it pass this filter. + // The null flag may be set to false later by other expressions in some cases. + nulls[sel[i]] = true + sel[j] = sel[i] + j++ + continue + } + + if isZero[i] == 0 { + continue + } + sel[j] = sel[i] // this row passes this filter + j++ + } + sel = sel[:j] + input.SetSel(sel) + globalColumnAllocator.put(buf) + } + + for _, i := range sel { + if !nulls[i] { + selected[i] = true + } + } + + return selected, nulls, nil +} + +func toBool(sc *stmtctx.StatementContext, eType types.EvalType, buf *chunk.Column, sel []int, isZero []int8) error { + switch eType { + case types.ETInt: + i64s := buf.Int64s() + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + if i64s[i] == 0 { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETReal: + f64s := buf.Float64s() + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + if f64s[i] == 0 { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETDuration: + d64s := buf.GoDurations() + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + if d64s[i] == 0 { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETDatetime, types.ETTimestamp: + t64s := buf.Times() + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + if t64s[i].IsZero() { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETString: + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + iVal, err := types.StrToFloat(sc, buf.GetString(i)) + if err != nil { + return err + } + if iVal == 0 { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETDecimal: + d64s := buf.Decimals() + for i := range sel { + if buf.IsNull(i) { + isZero[i] = -1 + } else { + if d64s[i].IsZero() { + isZero[i] = 0 + } else { + isZero[i] = 1 + } + } + } + case types.ETJson: + return errors.Errorf("cannot convert type json.BinaryJSON to bool") + } + return nil +} + +// EvalExpr evaluates this expr according to its type. +// And it selects the method for evaluating expression based on +// the environment variables and whether the expression can be vectorized. +func EvalExpr(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, result *chunk.Column) (err error) { + evalType := expr.GetType().EvalType() + if expr.Vectorized() && ctx.GetSessionVars().EnableVectorizedExpression { + switch evalType { + case types.ETInt: + err = expr.VecEvalInt(ctx, input, result) + case types.ETReal: + err = expr.VecEvalReal(ctx, input, result) + case types.ETDuration: + err = expr.VecEvalDuration(ctx, input, result) + case types.ETDatetime, types.ETTimestamp: + err = expr.VecEvalTime(ctx, input, result) + case types.ETString: + err = expr.VecEvalString(ctx, input, result) + case types.ETJson: + err = expr.VecEvalJSON(ctx, input, result) + case types.ETDecimal: + err = expr.VecEvalDecimal(ctx, input, result) + default: + err = errors.New(fmt.Sprintf("invalid eval type %v", expr.GetType().EvalType())) + } + } else { + ind, n := 0, input.NumRows() + iter := chunk.NewIterator4Chunk(input) + switch evalType { + case types.ETInt: + result.ResizeInt64(n, false) + i64s := result.Int64s() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalInt(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + i64s[ind] = value + } + ind++ + } + case types.ETReal: + result.ResizeFloat64(n, false) + f64s := result.Float64s() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalReal(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + f64s[ind] = value + } + ind++ + } + case types.ETDuration: + result.ResizeGoDuration(n, false) + d64s := result.GoDurations() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalDuration(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + d64s[ind] = value.Duration + } + ind++ + } + case types.ETDatetime, types.ETTimestamp: + result.ResizeTime(n, false) + t64s := result.Times() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalTime(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + t64s[ind] = value + } + ind++ + } + case types.ETString: + result.ReserveString(n) + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalString(ctx, it) + if err != nil { + return err + } + if isNull { + result.AppendNull() + } else { + result.AppendString(value) + } + } + case types.ETJson: + result.ReserveJSON(n) + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalJSON(ctx, it) + if err != nil { + return err + } + if isNull { + result.AppendNull() + } else { + result.AppendJSON(value) + } + } + case types.ETDecimal: + result.ResizeDecimal(n, false) + d64s := result.Decimals() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalDecimal(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + d64s[ind] = *value + } + ind++ + } + default: + err = errors.New(fmt.Sprintf("invalid eval type %v", expr.GetType().EvalType())) + } + } + return +} + +>>>>>>> 94011e6... expression: fix the issue that incorrect result for query that uses an AND operator on floats (#15927) // composeConditionWithBinaryOp composes condition with binary operator into a balance deep tree, which benefits a lot for pb decoder/encoder. func composeConditionWithBinaryOp(ctx sessionctx.Context, conditions []Expression, funcName string) Expression { length := len(conditions) diff --git a/expression/integration_test.go b/expression/integration_test.go index e145fc5f35cd5..dec1b24bf5123 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5014,3 +5014,707 @@ func (s *testIntegrationSuite) TestCacheRefineArgs(c *C) { tk.MustExec("set @p0='-184467440737095516167.1'") tk.MustQuery("execute stmt using @p0").Check(testkit.Rows("0")) } +<<<<<<< HEAD +======= + +func (s *testIntegrationSuite) TestCollation(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (utf8_bin_c varchar(10) charset utf8 collate utf8_bin, utf8_gen_c varchar(10) charset utf8 collate utf8_general_ci, bin_c binary, num_c int, " + + "abin char collate ascii_bin, lbin char collate latin1_bin, u4bin char collate utf8mb4_bin, u4ci char collate utf8mb4_general_ci)") + tk.MustExec("insert into t values ('a', 'b', 'c', 4, 'a', 'a', 'a', 'a')") + tk.MustQuery("select collation(null)").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(2)").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(2 + 'a')").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(2 + utf8_gen_c) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(2 + utf8_bin_c) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(utf8_bin_c, 2)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(concat(utf8_gen_c, 'abc')) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(concat(utf8_gen_c, null)) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(concat(utf8_gen_c, num_c)) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(concat(utf8_bin_c, utf8_gen_c)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(upper(utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(upper(utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(upper(bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(abin, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(lbin, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(utf8_bin_c, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(utf8_gen_c, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(u4bin, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(u4ci, bin_c)) from t").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(concat(abin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(concat(lbin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(concat(utf8_bin_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(concat(utf8_gen_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(concat(u4ci, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(concat(abin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(concat(lbin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(concat(utf8_bin_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(concat(utf8_gen_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(concat(abin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(concat(lbin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(concat(utf8_gen_c, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(concat(abin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(concat(lbin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) + tk.MustQuery("select collation(concat(abin, lbin)) from t").Check(testkit.Rows("latin1_bin")) + + tk.MustExec("set names utf8mb4 collate utf8mb4_bin") + tk.MustQuery("select collation('a')").Check(testkit.Rows("utf8mb4_bin")) + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") + tk.MustQuery("select collation('a')").Check(testkit.Rows("utf8mb4_general_ci")) + + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") + tk.MustExec("set @test_collate_var = 'a'") + tk.MustQuery("select collation(@test_collate_var)").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") + tk.MustExec("set @test_collate_var = 1") + tk.MustQuery("select collation(@test_collate_var)").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustExec("set @test_collate_var = concat(\"a\", \"b\" collate utf8mb4_bin)") + tk.MustQuery("select collation(@test_collate_var)").Check(testkit.Rows("utf8mb4_bin")) +} + +func (s *testIntegrationSuite) TestCoercibility(c *C) { + tk := testkit.NewTestKit(c, s.store) + + type testCase struct { + expr string + result int + } + testFunc := func(cases []testCase, suffix string) { + for _, tc := range cases { + tk.MustQuery(fmt.Sprintf("select coercibility(%v) %v", tc.expr, suffix)).Check(testkit.Rows(fmt.Sprintf("%v", tc.result))) + } + } + testFunc([]testCase{ + // constants + {"1", 5}, {"null", 6}, {"'abc'", 4}, + // sys-constants + {"version()", 3}, {"user()", 3}, {"database()", 3}, + {"current_role()", 3}, {"current_user()", 3}, + // scalar functions after constant folding + {"1+null", 5}, {"null+'abcde'", 5}, {"concat(null, 'abcde')", 4}, + // non-deterministic functions + {"rand()", 5}, {"now()", 5}, {"sysdate()", 5}, + }, "") + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (i int, r real, d datetime, t timestamp, c char(10), vc varchar(10), b binary(10), vb binary(10))") + tk.MustExec("insert into t values (null, null, null, null, null, null, null, null)") + testFunc([]testCase{ + {"i", 5}, {"r", 5}, {"d", 5}, {"t", 5}, + {"c", 2}, {"b", 2}, {"vb", 2}, {"vc", 2}, + {"i+r", 5}, {"i*r", 5}, {"cos(r)+sin(i)", 5}, {"d+2", 5}, + {"t*10", 5}, {"concat(c, vc)", 2}, {"replace(c, 'x', 'y')", 2}, + }, "from t") +} + +func (s *testIntegrationSerialSuite) TestCacheConstEval(c *C) { + tk := testkit.NewTestKit(c, s.store) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(col_double double)") + tk.MustExec("insert into t values (1)") + tk.Se.GetSessionVars().EnableVectorizedExpression = false + tk.MustExec("insert into mysql.expr_pushdown_blacklist values('cast', 'tikv,tiflash,tidb', 'for test')") + tk.MustExec("admin reload expr_pushdown_blacklist") + tk.MustExec("prepare stmt from 'SELECT * FROM (SELECT col_double AS c0 FROM t) t WHERE (ABS((REPEAT(?, ?) OR 5617780767323292672)) < LN(EXP(c0)) + (? ^ ?))'") + tk.MustExec("set @a1 = 'JuvkBX7ykVux20zQlkwDK2DFelgn7'") + tk.MustExec("set @a2 = 1") + tk.MustExec("set @a3 = -112990.35179796701") + tk.MustExec("set @a4 = 87997.92704840179") + // Main purpose here is checking no error is reported. 1 is the result when plan cache is disabled, it is + // incompatible with MySQL actually, update the result after fixing it. + tk.MustQuery("execute stmt using @a1, @a2, @a3, @a4").Check(testkit.Rows("1")) + tk.Se.GetSessionVars().EnableVectorizedExpression = true + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'cast' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'") + tk.MustExec("admin reload expr_pushdown_blacklist") +} + +func (s *testIntegrationSerialSuite) TestCollationBasic(c *C) { + tk := testkit.NewTestKit(c, s.store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk.MustExec("use test") + tk.MustExec("create table t_ci(a varchar(10) collate utf8mb4_general_ci, unique key(a))") + tk.MustExec("insert into t_ci values ('a')") + tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a'").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='A'").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) +} + +func (s *testIntegrationSerialSuite) TestWeightString(c *C) { + tk := testkit.NewTestKit(c, s.store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + type testCase struct { + input []string + result []string + resultAsChar1 []string + resultAsChar3 []string + resultAsBinary1 []string + resultAsBinary5 []string + resultExplicitCollateBin []string + } + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int, a varchar(20) collate utf8mb4_general_ci)") + cases := testCase{ + input: []string{"aAÁàãăâ", "a", "a ", "中", "中 "}, + result: []string{"\x00A\x00A\x00A\x00A\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsChar1: []string{"\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsChar3: []string{"\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsBinary1: []string{"a", "a", "a", "\xE4", "\xE4"}, + resultAsBinary5: []string{"aA\xc3\x81\xc3", "a\x00\x00\x00\x00", "a \x00\x00", "中\x00\x00", "中 \x00"}, + resultExplicitCollateBin: []string{"aAÁàãăâ", "a", "a", "中", "中"}, + } + values := make([]string, len(cases.input)) + for i, input := range cases.input { + values[i] = fmt.Sprintf("(%d, '%s')", i, input) + } + tk.MustExec("insert into t values " + strings.Join(values, ",")) + rows := tk.MustQuery("select weight_string(a) from t order by id").Rows() + for i, out := range cases.result { + c.Assert(rows[i][0].(string), Equals, out) + } + rows = tk.MustQuery("select weight_string(a as char(1)) from t order by id").Rows() + for i, out := range cases.resultAsChar1 { + c.Assert(rows[i][0].(string), Equals, out) + } + rows = tk.MustQuery("select weight_string(a as char(3)) from t order by id").Rows() + for i, out := range cases.resultAsChar3 { + c.Assert(rows[i][0].(string), Equals, out) + } + rows = tk.MustQuery("select weight_string(a as binary(1)) from t order by id").Rows() + for i, out := range cases.resultAsBinary1 { + c.Assert(rows[i][0].(string), Equals, out) + } + rows = tk.MustQuery("select weight_string(a as binary(5)) from t order by id").Rows() + for i, out := range cases.resultAsBinary5 { + c.Assert(rows[i][0].(string), Equals, out) + } + c.Assert(tk.MustQuery("select weight_string(NULL);").Rows()[0][0], Equals, "") + c.Assert(tk.MustQuery("select weight_string(7);").Rows()[0][0], Equals, "") + c.Assert(tk.MustQuery("select weight_string(cast(7 as decimal(5)));").Rows()[0][0], Equals, "") + c.Assert(tk.MustQuery("select weight_string(cast(20190821 as date));").Rows()[0][0], Equals, "2019-08-21") + c.Assert(tk.MustQuery("select weight_string(cast(20190821 as date) as binary(5));").Rows()[0][0], Equals, "2019-") + c.Assert(tk.MustQuery("select weight_string(7.0);").Rows()[0][0], Equals, "") + c.Assert(tk.MustQuery("select weight_string(7 AS BINARY(2));").Rows()[0][0], Equals, "7\x00") + // test explicit collation + c.Assert(tk.MustQuery("select weight_string('中 ' collate utf8mb4_general_ci);").Rows()[0][0], Equals, "\x4E\x2D") + c.Assert(tk.MustQuery("select weight_string('中 ' collate utf8mb4_bin);").Rows()[0][0], Equals, "中") + c.Assert(tk.MustQuery("select collation(a collate utf8mb4_general_ci) from t order by id").Rows()[0][0], Equals, "utf8mb4_general_ci") + c.Assert(tk.MustQuery("select collation('中 ' collate utf8mb4_general_ci);").Rows()[0][0], Equals, "utf8mb4_general_ci") + rows = tk.MustQuery("select weight_string(a collate utf8mb4_bin) from t order by id").Rows() + for i, out := range cases.resultExplicitCollateBin { + c.Assert(rows[i][0].(string), Equals, out) + } + tk.MustGetErrMsg("select weight_string(a collate utf8_general_ci) from t order by id", "[ddl:1253]COLLATION 'utf8_general_ci' is not valid for CHARACTER SET 'utf8mb4'") + tk.MustGetErrMsg("select weight_string('中' collate utf8_bin)", "[ddl:1253]COLLATION 'utf8_bin' is not valid for CHARACTER SET 'utf8mb4'") +} + +func (s *testIntegrationSerialSuite) TestCollationCreateIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("insert into t values ('b');") + tk.MustExec("insert into t values ('B');") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("create index idx on t(a);") + tk.MustQuery("select * from t order by a").Check(testkit.Rows("a", "A", "a", "A", "b", "B")) +} + +func (s *testIntegrationSerialSuite) TestCollateConstantPropagation(c *C) { + tk := testkit.NewTestKit(c, s.store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a char(10) collate utf8mb4_bin, b char(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_general_ci;").Check(nil) + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_general_ci;").Check(nil) + tk.MustExec("drop table t;") + tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('A', 'a');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4") + tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='A'").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_general_ci, b char collate utf8mb4_general_ci, c char collate utf8_bin);") + tk.MustExec("insert into t values ('b', 'B', 'B');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_bin, b char collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.b=t2.b and t2.b=t1.a collate utf8_general_ci;").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") + tk.MustExec("insert into t1 values ('A', 'a');") + tk.MustExec("insert into t2 values ('a', 'a')") + tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t(a char collate utf8mb4_bin, b char collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a', 'a');") + tk.MustQuery("select * from t t1, t t2 where t2.b = 'A' and lower(concat(t1.a , '' )) = t2.b;").Check(testkit.Rows("a a a a")) +} +func (s *testIntegrationSerialSuite) prepare4Join(c *C) *testkit.TestKit { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("USE test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t_bin") + tk.MustExec("CREATE TABLE `t` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL)") + tk.MustExec("CREATE TABLE `t_bin` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET binary)") + tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + return tk +} + +func (s *testIntegrationSerialSuite) prepare4Join2(c *C) *testkit.TestKit { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("USE test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (id int, v varchar(5) character set binary, key(v))") + tk.MustExec("create table t2 (v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(v))") + tk.MustExec("insert into t1 values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t2 values ('a'), ('À'), ('á'), ('à'), ('b'), ('c'), (' ')") + return tk +} + +func (s *testIntegrationSerialSuite) TestCollateHashJoin(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Join(c) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows()) +} + +func (s *testIntegrationSerialSuite) TestCollateHashJoin2(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Join2(c) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( + testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) +} + +func (s *testIntegrationSerialSuite) TestCollateMergeJoin(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Join(c) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows()) +} + +func (s *testIntegrationSerialSuite) TestCollateMergeJoin2(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Join2(c) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( + testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) +} + +func (s *testIntegrationSerialSuite) TestCollateIndexMergeJoin(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 t (a varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(a), key(b))") + tk.MustExec("insert into t values ('a', 'x'), ('x', 'À'), ('á', 'x'), ('à', 'à'), ('à', 'x')") + + tk.MustExec("set tidb_enable_index_merge=1") + tk.MustQuery("select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 'a' or b = 'a'").Sort().Check( + testkit.Rows("a x", "x À", "à x", "à à", "á x")) +} + +func (s *testIntegrationSerialSuite) prepare4Collation(c *C, hasIndex bool) *testkit.TestKit { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("USE test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t_bin") + idxSQL := ", key(v)" + if !hasIndex { + idxSQL = "" + } + tk.MustExec(fmt.Sprintf("create table t (id int, v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL %v)", idxSQL)) + tk.MustExec(fmt.Sprintf("create table t_bin (id int, v varchar(5) CHARACTER SET binary %v)", idxSQL)) + tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + return tk +} + +func (s *testIntegrationSerialSuite) TestCollateSelection(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, false) + tk.MustQuery("select v from t where v='a' order by id").Check(testkit.Rows("a", "À", "á", "à")) + tk.MustQuery("select v from t_bin where v='a' order by id").Check(testkit.Rows("a")) + tk.MustQuery("select v from t where v<'b' and id<=3").Check(testkit.Rows("a", "À", "á")) + tk.MustQuery("select v from t_bin where v<'b' and id<=3").Check(testkit.Rows("a")) +} + +func (s *testIntegrationSerialSuite) TestCollateSort(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, false) + tk.MustQuery("select id from t order by v, id").Check(testkit.Rows("7", "1", "2", "3", "4", "5", "6")) + tk.MustQuery("select id from t_bin order by v, id").Check(testkit.Rows("7", "1", "5", "6", "2", "4", "3")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustQuery("select * from t order by a collate utf8mb4_bin").Check(testkit.Rows("A", "A", "A", "a", "a", "a", "b", "b", "b")) +} + +func (s *testIntegrationSerialSuite) TestCollateHashAgg(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, false) + tk.HasPlan("select distinct(v) from t_bin", "HashAgg") + tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) + tk.HasPlan("select distinct(v) from t", "HashAgg") + tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) + tk.HasPlan("select v, count(*) from t_bin group by v", "HashAgg") + tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) + tk.HasPlan("select v, count(*) from t group by v", "HashAgg") + tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustQuery("select count(1) from t group by a collate utf8mb4_bin").Check(testkit.Rows("3", "3", "3")) +} + +func (s *testIntegrationSerialSuite) TestCollateStreamAgg(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, true) + tk.HasPlan("select distinct(v) from t_bin", "StreamAgg") + tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) + tk.HasPlan("select distinct(v) from t", "StreamAgg") + tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) + tk.HasPlan("select v, count(*) from t_bin group by v", "StreamAgg") + tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) + tk.HasPlan("select v, count(*) from t group by v", "StreamAgg") + tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) +} + +func (s *testIntegrationSerialSuite) TestCollateIndexReader(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, true) + tk.HasPlan("select v from t where v < 'b' order by v", "IndexReader") + tk.MustQuery("select v from t where v < 'b' order by v").Check(testkit.Rows(" ", "a", "À", "á", "à")) + tk.HasPlan("select v from t where v < 'b' and v > ' ' order by v", "IndexReader") + tk.MustQuery("select v from t where v < 'b' and v > ' ' order by v").Check(testkit.Rows("a", "À", "á", "à")) + tk.HasPlan("select v from t_bin where v < 'b' order by v", "IndexReader") + tk.MustQuery("select v from t_bin where v < 'b' order by v").Sort().Check(testkit.Rows(" ", "a")) + tk.HasPlan("select v from t_bin where v < 'b' and v > ' ' order by v", "IndexReader") + tk.MustQuery("select v from t_bin where v < 'b' and v > ' ' order by v").Sort().Check(testkit.Rows("a")) +} + +func (s *testIntegrationSerialSuite) TestCollateIndexLookup(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, true) + + tk.HasPlan("select id from t where v < 'b'", "IndexLookUp") + tk.MustQuery("select id from t where v < 'b'").Sort().Check(testkit.Rows("1", "2", "3", "4", "7")) + tk.HasPlan("select id from t where v < 'b' and v > ' '", "IndexLookUp") + tk.MustQuery("select id from t where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1", "2", "3", "4")) + tk.HasPlan("select id from t_bin where v < 'b'", "IndexLookUp") + tk.MustQuery("select id from t_bin where v < 'b'").Sort().Check(testkit.Rows("1", "7")) + tk.HasPlan("select id from t_bin where v < 'b' and v > ' '", "IndexLookUp") + tk.MustQuery("select id from t_bin where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1")) +} + +func (s *testIntegrationSerialSuite) TestCollateStringFunction(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := testkit.NewTestKit(c, s.store) + + tk.MustQuery("select field('a', 'b', 'a');").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'A');").Check(testkit.Rows("0")) + tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_bin);").Check(testkit.Rows("0")) + tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_bin);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + + tk.MustQuery("select FIND_IN_SET('a','b,a,c,d');").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d');").Check(testkit.Rows("0")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_bin);").Check(testkit.Rows("0")) + tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin);").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + + tk.MustExec("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin);") + tk.MustGetErrMsg("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci);", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'concat'") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char)") + tk.MustGetErrMsg("select * from t t1 join t t2 on t1.a collate utf8mb4_bin = t2.a collate utf8mb4_general_ci;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'eq'") +} + +func (s *testIntegrationSerialSuite) TestCollateLike(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") + tk.MustQuery("select 'a' like 'A'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'A' collate utf8mb4_general_ci").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'À'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like '%À'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like '%À '").Check(testkit.Rows("0")) + tk.MustQuery("select 'a' like 'À%'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'À_'").Check(testkit.Rows("0")) + tk.MustQuery("select 'a' like '%À%'").Check(testkit.Rows("1")) + tk.MustQuery("select 'aaa' like '%ÀAa%'").Check(testkit.Rows("1")) + tk.MustExec("set names utf8mb4 collate utf8mb4_bin") + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t_like;") + tk.MustExec("create table t_like(id int, b varchar(20) collate utf8mb4_general_ci);") + tk.MustExec("insert into t_like values (1, 'aaa'), (2, 'abc'), (3, 'aac');") + tk.MustQuery("select b like 'AaÀ' from t_like order by id;").Check(testkit.Rows("1", "0", "0")) + tk.MustQuery("select b like 'Aa_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) + tk.MustQuery("select b like '_A_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) + tk.MustQuery("select b from t_like where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) + tk.MustQuery("select b from t_like where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustQuery("select b from t_like where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustExec("alter table t_like add index idx_b(b);") + tk.MustQuery("select b from t_like use index(idx_b) where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) + tk.MustQuery("select b from t_like use index(idx_b) where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustQuery("select b from t_like use index(idx_b) where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) +} + +func (s *testIntegrationSerialSuite) TestCollateSubQuery(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := s.prepare4Collation(c, false) + tk.MustQuery("select id from t where v in (select v from t_bin) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t_bin where v in (select v from t) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t where v not in (select v from t_bin) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t_bin where v not in (select v from t) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t where exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t_bin where exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t where not exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t_bin where not exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows()) +} + +func (s *testIntegrationSerialSuite) TestCollateDDL(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database t;") + tk.MustExec("use t;") + tk.MustExec("drop database t;") +} + +func (s *testIntegrationSuite) TestIssue15986(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 int)") + tk.MustExec("INSERT INTO t0 VALUES (0)") + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE CHAR(204355900);").Check(testkit.Rows("0")) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE not CHAR(204355900);").Check(testkit.Rows()) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE '.0';").Check(testkit.Rows()) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE not '.0';").Check(testkit.Rows("0")) + // If the number does not exceed the range of float64 and its value is not 0, it will be converted to true. + tk.MustQuery("select * from t0 where '.000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "0000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows("0")) + tk.MustQuery("select * from t0 where not '.000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "0000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows()) + + // If the number is truncated beyond the range of float64, it will be converted to true when the truncated result is 0. + tk.MustQuery("select * from t0 where '.0000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows()) + tk.MustQuery("select * from t0 where not '.0000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows("0")) +} + +func (s *testIntegrationSuite) TestNegativeZeroForHashJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0, t1") + tk.MustExec("CREATE TABLE t0(c0 float);") + tk.MustExec("CREATE TABLE t1(c0 float);") + tk.MustExec("INSERT INTO t1(c0) VALUES (0);") + tk.MustExec("INSERT INTO t0(c0) VALUES (0);") + tk.MustQuery("SELECT t1.c0 FROM t1, t0 WHERE t0.c0=-t1.c0;").Check(testkit.Rows("0")) + tk.MustExec("drop TABLE t0;") + tk.MustExec("drop table t1;") +} + +func (s *testIntegrationSuite) TestIssue15743(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 int)") + tk.MustExec("INSERT INTO t0 VALUES (1)") + tk.MustQuery("SELECT * FROM t0 WHERE 1 AND 0.4").Check(testkit.Rows("1")) +} + +func (s *testIntegrationSuite) TestIssue15725(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(2)") + tk.MustQuery("select * from t where (not not a) = a").Check(testkit.Rows()) + tk.MustQuery("select * from t where (not not not not a) = a").Check(testkit.Rows()) +} + +func (s *testIntegrationSuite) TestIssue15790(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 INT);") + tk.MustExec("INSERT INTO t0(c0) VALUES (0);") + tk.MustQuery("SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION SELECT * FROM t0;").Check(testkit.Rows("0")) + tk.MustQuery("SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION all SELECT * FROM t0;").Check(testkit.Rows("0", "0")) + tk.MustExec("drop table t0;") +} + +func (s *testIntegrationSuite) TestIssue15990(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0;") + tk.MustExec("CREATE TABLE t0(c0 TEXT(10));") + tk.MustExec("INSERT INTO t0(c0) VALUES (1);") + tk.MustQuery("SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;").Check(testkit.Rows("1")) + tk.MustExec("CREATE INDEX i0 ON t0(c0(10));") + tk.MustQuery("SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;").Check(testkit.Rows("1")) + tk.MustExec("drop table t0;") +} + +func (s *testIntegrationSuite) TestIssue15992(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 INT, c1 INT AS (c0));") + tk.MustExec("CREATE INDEX i0 ON t0(c1);") + tk.MustQuery("SELECT t0.c0 FROM t0 UNION ALL SELECT 0 FROM t0;").Check(testkit.Rows()) + tk.MustExec("drop table t0;") +} + +func (s *testIntegrationSuite) TestIssue16419(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE t0(c0 INT);") + tk.MustExec("CREATE TABLE t1(c0 INT);") + tk.MustQuery("SELECT * FROM t1 NATURAL LEFT JOIN t0 WHERE NOT t1.c0;").Check(testkit.Rows()) + tk.MustExec("drop table t0, t1;") +} + +func (s *testIntegrationSuite) TestIssue16029(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0,t1;") + tk.MustExec("CREATE TABLE t0(c0 INT);") + tk.MustExec("CREATE TABLE t1(c0 INT);") + tk.MustExec("INSERT INTO t0 VALUES (NULL), (1);") + tk.MustExec("INSERT INTO t1 VALUES (0);") + tk.MustQuery("SELECT t0.c0 FROM t0 JOIN t1 ON (t0.c0 REGEXP 1) | t1.c0 WHERE BINARY STRCMP(t1.c0, t0.c0);").Check(testkit.Rows("1")) + tk.MustExec("drop table t0;") + tk.MustExec("drop table t1;") +} + +func (s *testIntegrationSuite) TestIssue16426(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int)") + tk.MustExec("insert into t values (42)") + tk.MustQuery("select a from t where a/10000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/100000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/1000000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/10000000").Check(testkit.Rows("42")) +} + +func (s *testIntegrationSuite) TestIssue16505(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("CREATE TABLE t(c varchar(100), index idx(c(100)));") + tk.MustExec("INSERT INTO t VALUES (NULL),('1'),('0'),(''),('aaabbb'),('0abc'),('123e456'),('0.0001deadsfeww');") + tk.MustQuery("select * from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustQuery("select /*+ USE_INDEX(t, idx) */ * from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustQuery("select /*+ IGNORE_INDEX(t, idx) */* from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustExec("drop table t;") +} +>>>>>>> 94011e6... expression: fix the issue that incorrect result for query that uses an AND operator on floats (#15927) diff --git a/types/datum.go b/types/datum.go index 9df170ddc3e48..df849cb0e5589 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1330,9 +1330,9 @@ func (d *Datum) ToBool(sc *stmtctx.StatementContext) (int64, error) { case KindUint64: isZero = d.GetUint64() == 0 case KindFloat32: - isZero = RoundFloat(d.GetFloat64()) == 0 + isZero = d.GetFloat64() == 0 case KindFloat64: - isZero = RoundFloat(d.GetFloat64()) == 0 + isZero = d.GetFloat64() == 0 case KindString, KindBytes: iVal, err1 := StrToInt(sc, d.GetString()) isZero, err = iVal == 0, err1 @@ -1341,8 +1341,7 @@ func (d *Datum) ToBool(sc *stmtctx.StatementContext) (int64, error) { case KindMysqlDuration: isZero = d.GetMysqlDuration().Duration == 0 case KindMysqlDecimal: - v, err1 := d.GetMysqlDecimal().ToFloat64() - isZero, err = RoundFloat(v) == 0, err1 + isZero = d.GetMysqlDecimal().IsZero() case KindMysqlEnum: isZero = d.GetMysqlEnum().ToNumber() == 0 case KindMysqlSet: diff --git a/types/datum_test.go b/types/datum_test.go index a1e8eb468983b..d25969856ac60 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -64,8 +64,15 @@ func (ts *testDatumSuite) TestToBool(c *C) { testDatumToBool(c, int(0), 0) testDatumToBool(c, int64(0), 0) testDatumToBool(c, uint64(0), 0) +<<<<<<< HEAD testDatumToBool(c, float32(0.1), 0) testDatumToBool(c, float64(0.1), 0) +======= + testDatumToBool(c, float32(0.1), 1) + testDatumToBool(c, float64(0.1), 1) + testDatumToBool(c, float64(0.5), 1) + testDatumToBool(c, float64(0.499), 1) +>>>>>>> 94011e6... expression: fix the issue that incorrect result for query that uses an AND operator on floats (#15927) testDatumToBool(c, "", 0) testDatumToBool(c, "0.1", 0) testDatumToBool(c, []byte{}, 0) @@ -86,7 +93,7 @@ func (ts *testDatumSuite) TestToBool(c *C) { ft.Decimal = 5 v, err := Convert(0.1415926, ft) c.Assert(err, IsNil) - testDatumToBool(c, v, 0) + testDatumToBool(c, v, 1) d := NewDatum(&invalidMockType{}) sc := new(stmtctx.StatementContext) sc.IgnoreTruncate = true