Skip to content

Commit

Permalink
Merge branch 'release-5.1' into release-5.1-50ae2d8266b9
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Sep 15, 2021
2 parents 34b2ae4 + 88689b4 commit 509945f
Show file tree
Hide file tree
Showing 23 changed files with 132 additions and 443 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 @@ -319,6 +319,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 @@ -328,7 +332,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 memDelta, err
}
Expand All @@ -349,7 +353,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 @@ -401,7 +405,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 @@ -784,14 +788,18 @@ 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 err error
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 memDelta, err
}
Expand Down
22 changes: 17 additions & 5 deletions executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,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 @@ -220,18 +220,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 @@ -536,18 +542,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
3 changes: 2 additions & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
derr "github.com/pingcap/tidb/store/driver/error"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -1682,7 +1683,7 @@ func (e *AnalyzeFastExec) buildSampTask() (err error) {
// Search for the region which contains the targetKey.
loc, err := e.cache.LocateKey(bo, targetKey)
if err != nil {
return err
return derr.ToTiDBErr(err)
}
if bytes.Compare(endKey, loc.StartKey) < 0 {
break
Expand Down
13 changes: 13 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -931,6 +931,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
2 changes: 1 addition & 1 deletion expression/aggregation/base_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,7 +431,7 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) {
if col, ok := a.Args[i].(*expression.Column); ok {
col.RetType = types.NewFieldType(col.RetType.Tp)
}
// originTp is used when the the `Tp` of column is TypeFloat32 while
// originTp is used when the `Tp` of column is TypeFloat32 while
// the type of the aggregation function is TypeFloat64.
originTp := a.Args[i].GetType().Tp
*(a.Args[i].GetType()) = *(a.RetTp)
Expand Down
2 changes: 1 addition & 1 deletion expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,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
34 changes: 34 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6976,6 +6976,22 @@ func (s *testIntegrationSerialSuite) TestIssue16668(c *C) {
tk.MustQuery("select count(distinct(b)) from tx").Check(testkit.Rows("4"))
}

func (s *testIntegrationSerialSuite) TestIssue27091(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 tx")
tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `c` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin DEFAULT NULL)")
tk.MustExec("insert into tx values (1, 'a', 'a'), (2, 'A ', 'a '), (3, 'A', 'A'), (4, 'a ', 'A ')")
tk.MustQuery("select count(distinct b) from tx").Check(testkit.Rows("1"))
tk.MustQuery("select count(distinct c) from tx").Check(testkit.Rows("2"))
tk.MustQuery("select count(distinct b, c) from tx where a < 3").Check(testkit.Rows("1"))
tk.MustQuery("select approx_count_distinct(b) from tx").Check(testkit.Rows("1"))
tk.MustQuery("select approx_count_distinct(c) from tx").Check(testkit.Rows("2"))
tk.MustQuery("select approx_count_distinct(b, c) from tx where a < 3").Check(testkit.Rows("1"))
}

func (s *testIntegrationSerialSuite) TestCollateStringFunction(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
Expand Down Expand Up @@ -9765,3 +9781,21 @@ func (s *testIntegrationSuite2) TestIssue25526(c *C) {
rows := tk.MustQuery("select tbl_6.col_31 from tbl_6 where col_31 in (select col_102 from tbl_17 where tbl_17.col_102 = 9999 and tbl_17.col_105 = 0);")
rows.Check(testkit.Rows())
}

func (s *testIntegrationSuite) TestConstPropNullFunctions(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (a integer)")
tk.MustExec("insert into t1 values (0), (1), (2), (3)")
tk.MustExec("create table t2 (a integer, b integer)")
tk.MustExec("insert into t2 values (0,1), (1,1), (2,1), (3,1)")
tk.MustQuery("select t1.* from t1 left join t2 on t2.a = t1.a where t1.a = ifnull(t2.b, 0)").Check(testkit.Rows("1"))

tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (i1 integer, c1 char)")
tk.MustExec("insert into t1 values (2, 'a'), (1, 'b'), (3, 'c'), (0, null);")
tk.MustExec("create table t2 (i2 integer, c2 char, f2 float)")
tk.MustExec("insert into t2 values (0, 'c', null), (1, null, 0.1), (3, 'b', 0.01), (2, 'q', 0.12), (null, 'a', -0.1), (null, null, null)")
tk.MustQuery("select * from t2 where t2.i2=((select count(1) from t1 where t1.i1=t2.i2))").Check(testkit.Rows("1 <nil> 0.1"))
}
4 changes: 2 additions & 2 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5090,7 +5090,7 @@ func (b *PlanBuilder) buildProjectionForWindow(ctx context.Context, p LogicalPla
p = np
switch newArg.(type) {
case *expression.Column, *expression.Constant:
newArgList = append(newArgList, newArg)
newArgList = append(newArgList, newArg.Clone())
continue
}
proj.Exprs = append(proj.Exprs, newArg)
Expand Down Expand Up @@ -5122,7 +5122,7 @@ func (b *PlanBuilder) buildArgs4WindowFunc(ctx context.Context, p LogicalPlan, a
p = np
switch newArg.(type) {
case *expression.Column, *expression.Constant:
newArgList = append(newArgList, newArg)
newArgList = append(newArgList, newArg.Clone())
continue
}
col := &expression.Column{
Expand Down
1 change: 1 addition & 0 deletions planner/core/partition_pruner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ func (s *testPartitionPruneSuit) TestHashPartitionPruner(c *C) {
tk.MustExec("create table t7(a int, b int) partition by hash(a + b) partitions 10;")
tk.MustExec("create table t8(a int, b int) partition by hash(a) partitions 6;")
tk.MustExec("create table t9(a bit(1) default null, b int(11) default null) partition by hash(a) partitions 3;") //issue #22619
tk.MustExec("create table t10(a bigint unsigned) partition BY hash (a);")

var input []string
var output []struct {
Expand Down
8 changes: 7 additions & 1 deletion planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,13 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl
if r.HighExclude {
posHigh--
}
rangeScalar := float64(posHigh) - float64(posLow) // use float64 to avoid integer overflow

var rangeScalar float64
if mysql.HasUnsignedFlag(col.RetType.Flag) {
rangeScalar = float64(uint64(posHigh)) - float64(uint64(posLow)) // use float64 to avoid integer overflow
} else {
rangeScalar = float64(posHigh) - float64(posLow) // use float64 to avoid integer overflow
}

// if range is less than the number of partitions, there will be unused partitions we can prune out.
if rangeScalar < float64(numPartitions) && !highIsNull && !lowIsNull {
Expand Down
15 changes: 7 additions & 8 deletions planner/core/testdata/integration_serial_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -2347,14 +2347,13 @@
" └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#8",
" └─Selection 8000.00 batchCop[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))",
" └─HashJoin 10000.00 batchCop[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]",
" ├─Selection(Build) 6387.21 batchCop[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))",
" │ └─Projection 7984.01 batchCop[tiflash] Column#7, test.t.id, test.t.value",
" │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value",
" │ └─ExchangeReceiver 7984.01 batchCop[tiflash] ",
" │ └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id",
" │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24",
" │ └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))",
" │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo",
" ├─Projection(Build) 7984.01 batchCop[tiflash] Column#7, test.t.id, test.t.value",
" │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value",
" │ └─ExchangeReceiver 7984.01 batchCop[tiflash] ",
" │ └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id",
" │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24",
" │ └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))",
" │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo",
" └─ExchangeReceiver(Probe) 10000.00 batchCop[tiflash] ",
" └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id",
" └─TableFullScan 10000.00 batchCop[tiflash] table:B keep order:false, stats:pseudo"
Expand Down
3 changes: 2 additions & 1 deletion planner/core/testdata/partition_pruner_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,8 @@
"explain format = 'brief' select * from t8 where a between 5 and 12",
"explain format = 'brief' select * from t8 where (a <= 10 and a >= 8) or (a <= 13 and a >= 11) or (a <= 16 and a >= 14)",
"explain format = 'brief' select * from t8 where a < 12 and a > 9",
"explain format = 'brief' select * from t9"
"explain format = 'brief' select * from t9",
"explain format = 'brief' select * from t10 where a between 0 AND 15218001646226433652"
]
},
{
Expand Down
8 changes: 8 additions & 0 deletions planner/core/testdata/partition_pruner_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,14 @@
"TableReader 10000.00 root partition:p0,p1 data:TableFullScan",
"└─TableFullScan 10000.00 cop[tikv] table:t9 keep order:false, stats:pseudo"
]
},
{
"SQL": "explain format = 'brief' select * from t10 where a between 0 AND 15218001646226433652",
"Result": [
"TableReader 250.00 root partition:all data:Selection",
"└─Selection 250.00 cop[tikv] ge(test_partition.t10.a, 0), le(test_partition.t10.a, 15218001646226433652)",
" └─TableFullScan 10000.00 cop[tikv] table:t10 keep order:false, stats:pseudo"
]
}
]
},
Expand Down
3 changes: 2 additions & 1 deletion server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ import (
"github.com/pingcap/tidb/sessionctx/binloginfo"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
derr "github.com/pingcap/tidb/store/driver/error"
"github.com/pingcap/tidb/store/gcworker"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/store/tikv"
Expand Down Expand Up @@ -151,7 +152,7 @@ type mvccKV struct {
func (t *tikvHandlerTool) getRegionIDByKey(encodedKey []byte) (uint64, error) {
keyLocation, err := t.RegionCache.LocateKey(tikv.NewBackofferWithVars(context.Background(), 500, nil), encodedKey)
if err != nil {
return 0, err
return 0, derr.ToTiDBErr(err)
}
return keyLocation.Region.GetID(), nil
}
Expand Down
61 changes: 0 additions & 61 deletions server/http_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1371,67 +1371,6 @@ func (ts *HTTPHandlerTestSuite) TestCheckCN(c *C) {
c.Assert(err, NotNil)
}

func (ts *HTTPHandlerTestSuite) TestZipInfoForSQL(c *C) {
ts.startServer(c)
defer ts.stopServer(c)

db, err := sql.Open("mysql", ts.getDSN())
c.Assert(err, IsNil, Commentf("Error connecting"))
defer func() {
err := db.Close()
c.Assert(err, IsNil)
}()
dbt := &DBTest{c, db}

dbt.mustExec("use test")
dbt.mustExec("create table if not exists t (a int)")

urlValues := url.Values{
"sql": {"select * from t"},
"current_db": {"test"},
}
resp, err := ts.formStatus("/debug/sub-optimal-plan", urlValues)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusOK)
b, err := httputil.DumpResponse(resp, true)
c.Assert(err, IsNil)
c.Assert(len(b), Greater, 0)
c.Assert(resp.Body.Close(), IsNil)

resp, err = ts.formStatus("/debug/sub-optimal-plan?pprof_time=5&timeout=0", urlValues)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusOK)
b, err = httputil.DumpResponse(resp, true)
c.Assert(err, IsNil)
c.Assert(len(b), Greater, 0)
c.Assert(resp.Body.Close(), IsNil)

resp, err = ts.formStatus("/debug/sub-optimal-plan?pprof_time=5", urlValues)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusOK)
b, err = httputil.DumpResponse(resp, true)
c.Assert(err, IsNil)
c.Assert(len(b), Greater, 0)
c.Assert(resp.Body.Close(), IsNil)

resp, err = ts.formStatus("/debug/sub-optimal-plan?timeout=1", urlValues)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusOK)
b, err = httputil.DumpResponse(resp, true)
c.Assert(err, IsNil)
c.Assert(len(b), Greater, 0)
c.Assert(resp.Body.Close(), IsNil)

urlValues.Set("current_db", "non_exists_db")
resp, err = ts.formStatus("/debug/sub-optimal-plan", urlValues)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusInternalServerError)
b, err = io.ReadAll(resp.Body)
c.Assert(err, IsNil)
c.Assert(string(b), Equals, "use database non_exists_db failed, err: [schema:1049]Unknown database 'non_exists_db'\n")
c.Assert(resp.Body.Close(), IsNil)
}

func (ts *HTTPHandlerTestSuite) TestFailpointHandler(c *C) {
defer ts.stopServer(c)

Expand Down
Loading

0 comments on commit 509945f

Please sign in to comment.