Skip to content

Commit

Permalink
Merge branch 'release-5.0' into release-5.0-9c75cfa4e2bb
Browse files Browse the repository at this point in the history
  • Loading branch information
AilinKid authored Aug 12, 2021
2 parents d1f71ca + 85b4864 commit df04318
Show file tree
Hide file tree
Showing 54 changed files with 2,782 additions and 290 deletions.
4 changes: 3 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -416,12 +416,13 @@ type Performance struct {
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"`
DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"distinct-agg-push-down"`
CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"`
MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"`
MemProfileInterval string `toml:"mem-profile-interval" json:"mem-profile-interval"`
IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"`
GOGC int `toml:"gogc" json:"gogc"`
EnforceMPP bool `toml:"enforce-mpp" json:"enforce-mpp"`
}

// PlanCache is the PlanCache section of the config.
Expand Down Expand Up @@ -618,6 +619,7 @@ var defaultConf = Config{
// TODO: set indexUsageSyncLease to 60s.
IndexUsageSyncLease: "0s",
GOGC: 100,
EnforceMPP: false,
},
ProxyProtocol: ProxyProtocol{
Networks: "",
Expand Down
9 changes: 9 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"os"
"os/user"
"path/filepath"
"reflect"
"runtime"
"testing"

Expand Down Expand Up @@ -421,6 +422,14 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1
// is recycled when the reference count drops to 0.
c.Assert(os.Remove(certFile), IsNil)
c.Assert(os.Remove(keyFile), IsNil)

// test for config `toml` and `json` tag names
c1 := Config{}
st := reflect.TypeOf(c1)
for i := 0; i < st.NumField(); i++ {
field := st.Field(i)
c.Assert(field.Tag.Get("toml"), Equals, field.Tag.Get("json"))
}
}

func (s *testConfigSuite) TestOOMActionValid(c *C) {
Expand Down
6 changes: 5 additions & 1 deletion executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,9 +155,13 @@ func buildApproxPercentile(sctx sessionctx.Context, aggFuncDesc *aggregation.Agg

base := basePercentile{percent: int(percent), baseAggFunc: baseAggFunc{args: aggFuncDesc.Args, ordinal: ordinal}}

evalType := aggFuncDesc.Args[0].GetType().EvalType()
if aggFuncDesc.Args[0].GetType().Tp == mysql.TypeBit {
evalType = types.ETString // same as other aggregate function
}
switch aggFuncDesc.Mode {
case aggregation.CompleteMode, aggregation.Partial1Mode, aggregation.FinalMode:
switch aggFuncDesc.Args[0].GetType().EvalType() {
switch evalType {
case types.ETInt:
return &percentileOriginal4Int{base}
case types.ETReal:
Expand Down
5 changes: 4 additions & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1662,7 +1662,10 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
case *ast.LoadDataStmt:
sc.DupKeyAsWarning = true
sc.BadNullAsWarning = true
sc.TruncateAsWarning = !vars.StrictSQLMode
// With IGNORE or LOCAL, data-interpretation errors become warnings and the load operation continues,
// even if the SQL mode is restrictive. For details: https://dev.mysql.com/doc/refman/8.0/en/load-data.html
// TODO: since TiDB only support the LOCAL by now, so the TruncateAsWarning are always true here.
sc.TruncateAsWarning = true
sc.InLoadDataStmt = true
// return warning instead of error when load data meet no partition for value
sc.IgnoreNoPartition = true
Expand Down
176 changes: 105 additions & 71 deletions executor/merge_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"bytes"
"fmt"
"math/rand"
"strconv"
"strings"

. "github.com/pingcap/check"
Expand Down Expand Up @@ -728,20 +729,25 @@ func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) {
func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) {
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 (a int, b int)")
tk.MustExec("create table t2 (a int, b int)")
runTest := func(t1, t2 []int) {
tk.MustExec("truncate table t1")
tk.MustExec("truncate table t2")
insert := func(tName string, ts []int) {
existTableMap := make(map[string]struct{})
runTest := func(ts1, ts2 []int) {
getTable := func(prefix string, ts []int) string {
tableName := prefix
for _, i := range ts {
tableName = tableName + "_" + strconv.Itoa(i)
}
if _, ok := existTableMap[tableName]; ok {
return tableName
}
tk.MustExec(fmt.Sprintf("drop table if exists %s", tableName))
tk.MustExec(fmt.Sprintf("create table %s (a int, b int)", tableName))
existTableMap[tableName] = struct{}{}
for i, n := range ts {
if n == 0 {
continue
}
var buf bytes.Buffer
buf.WriteString(fmt.Sprintf("insert into %v values ", tName))
buf.WriteString(fmt.Sprintf("insert into %v values ", tableName))
for j := 0; j < n; j++ {
if j > 0 {
buf.WriteString(", ")
Expand All @@ -750,33 +756,45 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) {
}
tk.MustExec(buf.String())
}
return tableName
}
insert("t1", t1)
insert("t2", t2)

tk.MustQuery("explain format = 'brief' select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Check(testkit.Rows(
`MergeJoin 4150.01 root inner join, left key:test.t1.a, right key:test.t2.a`,
`├─Sort(Build) 3320.01 root test.t2.a`,
`│ └─TableReader 3320.01 root data:Selection`,
`│ └─Selection 3320.01 cop[tikv] lt(test.t2.b, 5), not(isnull(test.t2.a))`,
`│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo`,
`└─Sort(Probe) 3330.00 root test.t1.a`,
` └─TableReader 3330.00 root data:Selection`,
` └─Selection 3330.00 cop[tikv] gt(test.t1.b, 5), not(isnull(test.t1.a))`,
` └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo`,
t1 := getTable("t", ts1)
t2 := getTable("t", ts2)
if t1 == t2 {
t2 = getTable("t2", ts2)
}

tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Check(testkit.Rows(
fmt.Sprintf(`MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2),
fmt.Sprintf(`├─Sort(Build) 3320.01 root test.%s.a`, t2),
fmt.Sprintf(`│ └─TableReader 3320.01 root data:Selection`),
fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2),
fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2),
fmt.Sprintf(`└─Sort(Probe) 3330.00 root test.%s.a`, t1),
fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`),
fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1),
fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1),
))
tk.MustQuery("explain format = 'brief' select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Check(testkit.Rows(
`HashJoin 4150.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]`,
`├─TableReader(Build) 3320.01 root data:Selection`,
`│ └─Selection 3320.01 cop[tikv] lt(test.t2.b, 5), not(isnull(test.t2.a))`,
`│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo`,
`└─TableReader(Probe) 3330.00 root data:Selection`,
` └─Selection 3330.00 cop[tikv] gt(test.t1.b, 5), not(isnull(test.t1.a))`,
` └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo`,
tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_HJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Check(testkit.Rows(
fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2),
fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`),
fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2),
fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2),
fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`),
fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1),
fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1),
))

r1 := tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Sort()
r2 := tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Sort()
r1 := tk.MustQuery(fmt.Sprintf("select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Sort()
r2 := tk.MustQuery(fmt.Sprintf("select /*+ TIDB_HJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Sort()
c.Assert(len(r1.Rows()), Equals, len(r2.Rows()))

i := 0
Expand Down Expand Up @@ -806,10 +824,7 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) {
{[]int{chunkSize - 1}, []int{chunkSize - 1}},
{[]int{chunkSize - 1}, []int{chunkSize + 1}},
{[]int{chunkSize}, []int{chunkSize}},
{[]int{chunkSize}, []int{chunkSize - 1}},
{[]int{chunkSize}, []int{chunkSize + 1}},
{[]int{chunkSize + 1}, []int{chunkSize}},
{[]int{chunkSize + 1}, []int{chunkSize - 1}},
{[]int{chunkSize + 1}, []int{chunkSize + 1}},
{[]int{1, 1, 1}, []int{chunkSize + 1, chunkSize*5 + 5, chunkSize - 5}},
{[]int{0, 0, chunkSize}, []int{chunkSize + 1, chunkSize*5 + 5, chunkSize - 5}},
Expand All @@ -819,27 +834,37 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) {
runTest(ca.t1, ca.t2)
runTest(ca.t2, ca.t1)
}
fmt.Println(existTableMap)
for tableName := range existTableMap {
tk.MustExec(fmt.Sprintf("drop table if exists %s", tableName))
}
}

// TestVectorizedShuffleMergeJoin is used to test vectorized shuffle merge join with some corner cases.
func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("set @@session.tidb_merge_join_concurrency = 4;")
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t1 (a int, b int)")
tk.MustExec("create table t2 (a int, b int)")
runTest := func(t1, t2 []int) {
tk.MustExec("truncate table t1")
tk.MustExec("truncate table t2")
insert := func(tName string, ts []int) {
tk.MustExec("use test")
existTableMap := make(map[string]struct{})
runTest := func(ts1, ts2 []int) {
getTable := func(prefix string, ts []int) string {
tableName := prefix
for _, i := range ts {
tableName = tableName + "_" + strconv.Itoa(i)
}
if _, ok := existTableMap[tableName]; ok {
return tableName
}
tk.MustExec(fmt.Sprintf("drop table if exists %s", tableName))
tk.MustExec(fmt.Sprintf("create table %s (a int, b int)", tableName))
existTableMap[tableName] = struct{}{}
for i, n := range ts {
if n == 0 {
continue
}
var buf bytes.Buffer
buf.WriteString(fmt.Sprintf("insert into %v values ", tName))
buf.WriteString(fmt.Sprintf("insert into %v values ", tableName))
for j := 0; j < n; j++ {
if j > 0 {
buf.WriteString(", ")
Expand All @@ -848,34 +873,46 @@ func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) {
}
tk.MustExec(buf.String())
}
return tableName
}
t1 := getTable("t", ts1)
t2 := getTable("t", ts2)
if t1 == t2 {
t2 = getTable("t2", ts2)
}
insert("t1", t1)
insert("t2", t2)

tk.MustQuery("explain format = 'brief' select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Check(testkit.Rows(
`Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`,
`└─MergeJoin 4150.01 root inner join, left key:test.t1.a, right key:test.t2.a`,
` ├─Sort(Build) 3320.01 root test.t2.a`,
` │ └─TableReader 3320.01 root data:Selection`,
` │ └─Selection 3320.01 cop[tikv] lt(test.t2.b, 5), not(isnull(test.t2.a))`,
` │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo`,
` └─Sort(Probe) 3330.00 root test.t1.a`,
` └─TableReader 3330.00 root data:Selection`,
` └─Selection 3330.00 cop[tikv] gt(test.t1.b, 5), not(isnull(test.t1.a))`,
` └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo`,

tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Check(testkit.Rows(
fmt.Sprintf(`Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`),
fmt.Sprintf(`└─MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2),
fmt.Sprintf(` ├─Sort(Build) 3320.01 root test.%s.a`, t2),
fmt.Sprintf(` │ └─TableReader 3320.01 root data:Selection`),
fmt.Sprintf(` │ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2),
fmt.Sprintf(` │ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2),
fmt.Sprintf(` └─Sort(Probe) 3330.00 root test.%s.a`, t1),
fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`),
fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1),
fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1),
))
tk.MustQuery("explain format = 'brief' select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Check(testkit.Rows(
`HashJoin 4150.01 root inner join, equal:[eq(test.t1.a, test.t2.a)]`,
`├─TableReader(Build) 3320.01 root data:Selection`,
`│ └─Selection 3320.01 cop[tikv] lt(test.t2.b, 5), not(isnull(test.t2.a))`,
`│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo`,
`└─TableReader(Probe) 3330.00 root data:Selection`,
` └─Selection 3330.00 cop[tikv] gt(test.t1.b, 5), not(isnull(test.t1.a))`,
` └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo`,
tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_HJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Check(testkit.Rows(
fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2),
fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`),
fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2),
fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2),
fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`),
fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1),
fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1),
))

r1 := tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Sort()
r2 := tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.a=t2.a and t1.b>5 and t2.b<5").Sort()
r1 := tk.MustQuery(fmt.Sprintf("select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Sort()
r2 := tk.MustQuery(fmt.Sprintf("select /*+ TIDB_HJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5",
t1, t2, t1, t2, t1, t2, t1, t2,
)).Sort()
c.Assert(len(r1.Rows()), Equals, len(r2.Rows()))

i := 0
Expand Down Expand Up @@ -905,10 +942,7 @@ func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) {
{[]int{chunkSize - 1}, []int{chunkSize - 1}},
{[]int{chunkSize - 1}, []int{chunkSize + 1}},
{[]int{chunkSize}, []int{chunkSize}},
{[]int{chunkSize}, []int{chunkSize - 1}},
{[]int{chunkSize}, []int{chunkSize + 1}},
{[]int{chunkSize + 1}, []int{chunkSize}},
{[]int{chunkSize + 1}, []int{chunkSize - 1}},
{[]int{chunkSize + 1}, []int{chunkSize + 1}},
{[]int{1, 1, 1}, []int{chunkSize + 1, chunkSize*5 + 5, chunkSize - 5}},
{[]int{0, 0, chunkSize}, []int{chunkSize + 1, chunkSize*5 + 5, chunkSize - 5}},
Expand Down
2 changes: 1 addition & 1 deletion executor/mpp_gather.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
)

func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader) bool {
if !ctx.GetSessionVars().AllowMPPExecution {
if !ctx.GetSessionVars().IsMPPAllowed() {
return false
}
_, ok := tr.GetTablePlan().(*plannercore.PhysicalExchangeSender)
Expand Down
10 changes: 10 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -547,6 +547,16 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustQuery(`show warnings`).Check(testkit.Rows())
tk.MustExec("set @@tidb_enable_clustered_index = 'int_only'")
tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1287 'INT_ONLY' is deprecated and will be removed in a future release. Please use 'ON' or 'OFF' instead"))

// test for tidb_enable_ordered_result_mode
tk.MustQuery(`select @@tidb_enable_ordered_result_mode`).Check(testkit.Rows("0"))
tk.MustExec(`set global tidb_enable_ordered_result_mode = 1`)
tk.MustQuery(`select @@global.tidb_enable_ordered_result_mode`).Check(testkit.Rows("1"))
tk.MustExec(`set global tidb_enable_ordered_result_mode = 0`)
tk.MustQuery(`select @@global.tidb_enable_ordered_result_mode`).Check(testkit.Rows("0"))
tk.MustExec(`set tidb_enable_ordered_result_mode=1`)
tk.MustQuery(`select @@global.tidb_enable_ordered_result_mode`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@tidb_enable_ordered_result_mode`).Check(testkit.Rows("1"))
}

func (s *testSuite5) TestTruncateIncorrectIntSessionVar(c *C) {
Expand Down
10 changes: 10 additions & 0 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,16 @@ func (s *testSuite5) TestShowWarningsForExprPushdown(c *C) {
tk.MustExec("explain select * from show_warnings_expr_pushdown where date_add(value, interval 1 day) = '2020-01-01'")
c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'date_add'(signature: AddDateDatetimeInt) can not be pushed to tikv"))
tk.MustExec("explain select max(date_add(value, interval 1 day)) from show_warnings_expr_pushdown group by a")
c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'date_add'(signature: AddDateDatetimeInt) can not be pushed to tikv", "Warning|1105|Aggregation can not be pushed to tikv because arguments of AggFunc `max` contains unsupported exprs"))
tk.MustExec("explain select max(a) from show_warnings_expr_pushdown group by date_add(value, interval 1 day)")
c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'date_add'(signature: AddDateDatetimeInt) can not be pushed to tikv", "Warning|1105|Aggregation can not be pushed to tikv because groupByItems contain unsupported exprs"))
tk.MustExec("set tidb_opt_distinct_agg_push_down=0")
tk.MustExec("explain select max(distinct a) from show_warnings_expr_pushdown group by value")
c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Aggregation can not be pushed to storage layer in non-mpp mode because it contains agg function with distinct"))
}

func (s *testSuite5) TestShowGrantsPrivilege(c *C) {
Expand Down
Loading

0 comments on commit df04318

Please sign in to comment.