Skip to content

Commit

Permalink
Merge branch 'issue_27422' of github.com:sylzd/tidb into issue_27422
Browse files Browse the repository at this point in the history
  • Loading branch information
sylzd committed Aug 25, 2021
2 parents be01322 + 71cae28 commit 25922f4
Show file tree
Hide file tree
Showing 26 changed files with 600 additions and 368 deletions.
8 changes: 0 additions & 8 deletions br/pkg/restore/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,14 +99,6 @@ func (rs *RegionSplitter) Split(
maxKey = rule.GetNewKeyPrefix()
}
}
for _, rule := range rewriteRules.Data {
if bytes.Compare(minKey, rule.GetNewKeyPrefix()) > 0 {
minKey = rule.GetNewKeyPrefix()
}
if bytes.Compare(maxKey, rule.GetNewKeyPrefix()) < 0 {
maxKey = rule.GetNewKeyPrefix()
}
}
interval := SplitRetryInterval
scatterRegions := make([]*RegionInfo, 0)
SplitRegions:
Expand Down
7 changes: 6 additions & 1 deletion br/pkg/task/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -345,7 +345,12 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig
pdAddress := strings.Join(cfg.PD, ",")
log.Warn("Nothing to backup, maybe connected to cluster for restoring",
zap.String("PD address", pdAddress))
return metawriter.FlushBackupMeta(ctx)

err = metawriter.FlushBackupMeta(ctx)
if err == nil {
summary.SetSuccessStatus(true)
}
return err
}

if isIncrementalBackup {
Expand Down
8 changes: 0 additions & 8 deletions docs/MAINTAINERS.md

This file was deleted.

22 changes: 17 additions & 5 deletions executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,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 @@ -221,18 +221,24 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI
memDelta += int64(p.buffer.Cap())
}
}()

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 memDelta, 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 @@ -537,18 +543,24 @@ func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context,
v, isNull := "", false
memDelta -= int64(cap(p.encodeBytesBuffer))
defer func() { memDelta += int64(cap(p.encodeBytesBuffer)) }()

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 memDelta, 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 @@ -949,6 +949,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
33 changes: 33 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9019,6 +9019,27 @@ func (s *testSuite) TestIssue25506(c *C) {
tk.MustQuery("(select col_15 from tbl_23) union all (select col_15 from tbl_3 for update) order by col_15").Check(testkit.Rows("\x00\x00\x0F", "\x00\x00\xFF", "\x00\xFF\xFF"))
}

func (s *testSuite) TestIssue26348(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 varchar(8) DEFAULT NULL,
b varchar(8) DEFAULT NULL,
c decimal(20,2) DEFAULT NULL,
d decimal(15,8) DEFAULT NULL
);`)
tk.MustExec(`insert into t values(20210606, 20210606, 50000.00, 5.04600000);`)
tk.MustQuery(`select a * c *(d/36000) from t;`).Check(testkit.Rows("141642663.71666598"))
tk.MustQuery(`select cast(a as double) * cast(c as double) *cast(d/36000 as double) from t;`).Check(testkit.Rows("141642663.71666598"))
tk.MustQuery("select 20210606*50000.00*(5.04600000/36000)").Check(testkit.Rows("141642663.71666599297980"))

// differs from MySQL cause constant-fold .
tk.MustQuery("select \"20210606\"*50000.00*(5.04600000/36000)").Check(testkit.Rows("141642663.71666598"))
tk.MustQuery("select cast(\"20210606\" as double)*50000.00*(5.04600000/36000)").Check(testkit.Rows("141642663.71666598"))
}

func (s *testSuite) TestIssue26532(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down Expand Up @@ -9055,3 +9076,15 @@ func (s *testSuite) TestIssue23602(c *C) {
"[└─TableRangeScan 2.00 cop[tikv] table:t range:[1,2], keep order:false"))
tk.MustQuery("SELECT a FROM t WHERE a BETWEEN 0xFFFFFFFFFFFFFFF5 AND X'FFFFFFFFFFFFFFFA'").Check(testkit.Rows("18446744073709551605", "18446744073709551610"))
}

func (s *testSuite) TestCTEWithIndexLookupJoinDeadLock(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table t (a int(11) default null,b int(11) default null,key b (b),key ba (b))")
tk.MustExec("create table t1 (a int(11) default null,b int(11) default null,key idx_ab (a,b),key idx_a (a),key idx_b (b))")
tk.MustExec("create table t2 (a int(11) default null,b int(11) default null,key idx_ab (a,b),key idx_a (a),key idx_b (b))")
// It's easy to reproduce this problem in 30 times execution of IndexLookUpJoin.
for i := 0; i < 30; i++ {
tk.MustExec("with cte as (with cte1 as (select * from t2 use index(idx_ab) where a > 1 and b > 1) select * from cte1) select /*+use_index(t1 idx_ab)*/ * from cte join t1 on t1.a=cte.a;")
}
}
2 changes: 1 addition & 1 deletion expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,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
15 changes: 15 additions & 0 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,7 @@ func (c *castAsRealFunctionClass) getFunction(ctx sessionctx.Context, args []Exp
sig.setPbCode(tipb.ScalarFuncSig_CastRealAsReal)
case types.ETDecimal:
sig = &builtinCastDecimalAsRealSig{bf}
PropagateType(types.ETReal, sig.getArgs()...)
sig.setPbCode(tipb.ScalarFuncSig_CastDecimalAsReal)
case types.ETDatetime, types.ETTimestamp:
sig = &builtinCastTimeAsRealSig{bf}
Expand Down Expand Up @@ -1012,6 +1013,20 @@ type builtinCastDecimalAsRealSig struct {
baseBuiltinCastFunc
}

func setDataTypeDouble(srcDecimal int) (flen, decimal int) {
decimal = mysql.NotFixedDec
flen = floatLength(srcDecimal, decimal)
return
}

func floatLength(srcDecimal int, decimalPar int) int {
const dblDIG = 15
if srcDecimal != mysql.NotFixedDec {
return dblDIG + 2 + decimalPar
}
return dblDIG + 8
}

func (b *builtinCastDecimalAsRealSig) Clone() builtinFunc {
newSig := &builtinCastDecimalAsRealSig{}
newSig.cloneFrom(&b.baseBuiltinCastFunc)
Expand Down
8 changes: 8 additions & 0 deletions expression/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,14 @@ func (c *Constant) EvalDecimal(ctx sessionctx.Context, row chunk.Row) (*types.My
return nil, true, nil
}
res, err := dt.ToDecimal(ctx.GetSessionVars().StmtCtx)
if err != nil {
return nil, false, err
}
// The decimal may be modified during plan building.
_, frac := res.PrecisionAndFrac()
if frac < c.GetType().Decimal {
err = res.Round(res, c.GetType().Decimal, types.ModeHalfEven)
}
return res, false, err
}

Expand Down
48 changes: 48 additions & 0 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -1290,3 +1290,51 @@ func wrapWithIsTrue(ctx sessionctx.Context, keepNull bool, arg Expression, wrapF
}
return FoldConstant(sf), nil
}

// PropagateType propagates the type information to the `expr`.
// Note: For now, we only propagate type for the function CastDecimalAsDouble.
//
// e.g.
// > create table t(a decimal(9, 8));
// > insert into t values(5.04600000)
// > select a/36000 from t;
// Type: NEWDECIMAL
// Length: 15
// Decimals: 12
// +------------------+
// | 5.04600000/36000 |
// +------------------+
// | 0.000140166667 |
// +------------------+
//
// > select cast(a/36000 as double) as result from t;
// Type: DOUBLE
// Length: 23
// Decimals: 31
// +----------------------+
// | result |
// +----------------------+
// | 0.000140166666666666 |
// +----------------------+
// The expected `decimal` and `length` of the outer cast_as_double need to be
// propagated to the inner div.
func PropagateType(evalType types.EvalType, args ...Expression) {
switch evalType {
case types.ETReal:
expr := args[0]
oldFlen, oldDecimal := expr.GetType().Flen, expr.GetType().Decimal
newFlen, newDecimal := setDataTypeDouble(expr.GetType().Decimal)
// For float(M,D), double(M,D) or decimal(M,D), M must be >= D.
if newFlen < newDecimal {
newFlen = oldFlen - oldDecimal + newDecimal
}
if oldFlen != newFlen || oldDecimal != newDecimal {
if col, ok := args[0].(*Column); ok {
newCol := col.Clone()
newCol.(*Column).RetType = col.RetType.Clone()
args[0] = newCol
}
args[0].GetType().Flen, args[0].GetType().Decimal = newFlen, newDecimal
}
}
}
2 changes: 1 addition & 1 deletion expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8959,7 +8959,7 @@ func (s *testIntegrationSerialSuite) TestLikeWithCollation(c *C) {
defer collate.SetNewCollationEnabledForTest(false)

tk.MustQuery(`select 'a' like 'A' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("1"))
tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci;`, "[expression:1270]Illegal mix of collations (utf8mb4_bin,EXPLICIT), (utf8mb4_unicode_ci,EXPLICIT), (utf8mb4_bin,NUMERIC) for operation 'like'")
tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci;`, "[expression:1270]Illegal mix of collations (utf8mb4_bin,EXPLICIT), (utf8mb4_unicode_ci,EXPLICIT), (binary,NUMERIC) for operation 'like'")
tk.MustQuery(`select '😛' collate utf8mb4_general_ci like '😋';`).Check(testkit.Rows("1"))
tk.MustQuery(`select '😛' collate utf8mb4_general_ci = '😋';`).Check(testkit.Rows("1"))
tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci like '😋';`).Check(testkit.Rows("0"))
Expand Down
20 changes: 14 additions & 6 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1636,17 +1636,25 @@ func (er *expressionRewriter) betweenToExpression(v *ast.BetweenExpr) {

expr, lexp, rexp := er.wrapExpWithCast()

var op string
er.err = expression.CheckIllegalMixCollation("between", []expression.Expression{expr, lexp, rexp}, types.ETInt)
if er.err != nil {
return
}

dstCharset, dstCollation := expression.DeriveCollationFromExprs(er.sctx, expr, lexp, rexp)

var l, r expression.Expression
l, er.err = er.newFunction(ast.GE, &v.Type, expr, lexp)
if er.err == nil {
r, er.err = er.newFunction(ast.LE, &v.Type, expr, rexp)
l, er.err = expression.NewFunctionBase(er.sctx, ast.GE, &v.Type, expr, lexp)
if er.err != nil {
return
}
op = ast.LogicAnd
r, er.err = expression.NewFunctionBase(er.sctx, ast.LE, &v.Type, expr, rexp)
if er.err != nil {
return
}
function, err := er.newFunction(op, &v.Type, l, r)
l.SetCharsetAndCollation(dstCharset, dstCollation)
r.SetCharsetAndCollation(dstCharset, dstCollation)
function, err := er.newFunction(ast.LogicAnd, &v.Type, l, r)
if err != nil {
er.err = err
return
Expand Down
28 changes: 28 additions & 0 deletions planner/core/expression_rewriter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,14 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tidb/util/testutil"
)

var _ = Suite(&testExpressionRewriterSuite{})
var _ = SerialSuites(&testExpressionRewriterSuiteSerial{})

type testExpressionRewriterSuite struct {
testData testutil.TestData
Expand All @@ -39,6 +41,9 @@ func (s *testExpressionRewriterSuite) TearDownSuite(c *C) {
c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil)
}

type testExpressionRewriterSuiteSerial struct {
}

func (s *testExpressionRewriterSuite) TestIfNullEliminateColName(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
Expand Down Expand Up @@ -437,6 +442,29 @@ func (s *testExpressionRewriterSuite) TestIssue24705(c *C) {
c.Assert(err.Error(), Equals, "[expression:1267]Illegal mix of collations (utf8_general_ci,IMPLICIT) and (utf8_unicode_ci,IMPLICIT) for operation '<'")
}

func (s *testExpressionRewriterSuiteSerial) TestBetweenExprCollation(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)

defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()

tk.MustExec("use test")
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(a char(10) charset latin1 collate latin1_bin, c char(10) collate utf8mb4_general_ci);")
tk.MustExec("insert into t1 values ('a', 'B');")
tk.MustExec("insert into t1 values ('c', 'D');")
tk.MustQuery("select * from t1 where a between 'B' and c;").Check(testkit.Rows("c D"))

tk.MustGetErrMsg("select * from t1 where a between 'B' collate utf8mb4_general_ci and c collate utf8mb4_unicode_ci;", "[expression:1270]Illegal mix of collations (latin1_bin,IMPLICIT), (utf8mb4_general_ci,EXPLICIT), (utf8mb4_unicode_ci,EXPLICIT) for operation 'between'")
}

func (s *testExpressionRewriterSuite) TestMultiColInExpression(c *C) {
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
Expand Down
31 changes: 31 additions & 0 deletions planner/core/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2021 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package core

import (
"testing"

"github.com/pingcap/tidb/util/testbridge"
"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
testbridge.WorkaroundGoCheckFlags()
opts := []goleak.Option{
goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"),
goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"),
}
goleak.VerifyTestMain(m, opts...)
}
Loading

0 comments on commit 25922f4

Please sign in to comment.