diff --git a/executor/test/tiflashtest/BUILD.bazel b/executor/test/tiflashtest/BUILD.bazel new file mode 100644 index 0000000000000..2f6658a8fa74f --- /dev/null +++ b/executor/test/tiflashtest/BUILD.bazel @@ -0,0 +1,36 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "tiflashtest_test", + timeout = "moderate", + srcs = [ + "main_test.go", + "tiflash_test.go", + ], + flaky = True, + race = "on", + shard_count = 39, + deps = [ + "//config", + "//domain", + "//kv", + "//meta/autoid", + "//parser/terror", + "//planner/core", + "//store/mockstore", + "//store/mockstore/unistore", + "//testkit", + "//testkit/external", + "//util/dbterror/exeerrors", + "//util/memory", + "//util/tiflashcompute", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/metapb", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//testutils", + "@com_github_tikv_client_go_v2//tikv", + "@io_opencensus_go//stats/view", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/executor/test/tiflashtest/tiflash_test.go b/executor/test/tiflashtest/tiflash_test.go new file mode 100644 index 0000000000000..bfc8d1caf8ff2 --- /dev/null +++ b/executor/test/tiflashtest/tiflash_test.go @@ -0,0 +1,1838 @@ +// Copyright 2020 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 tiflashtest + +import ( + "bytes" + "fmt" + "math/rand" + "strings" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/unistore" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" + "github.com/pingcap/tidb/util/dbterror/exeerrors" + "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/tiflashcompute" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/testutils" +) + +// withMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). +func withMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { + return mockstore.WithMultipleOptions( + mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockCluster := c.(*unistore.Cluster) + _, _, region1 := mockstore.BootstrapWithSingleStore(c) + tiflashIdx := 0 + for tiflashIdx < nodes { + store2 := c.AllocID() + peer2 := c.AllocID() + addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) + mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) + mockCluster.AddPeer(region1, store2, peer2) + tiflashIdx++ + } + }), + mockstore.WithStoreType(mockstore.EmbedUnistore), + ) +} + +func TestNonsupportCharsetTable(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b char(10) charset gbk collate gbk_bin)") + err := tk.ExecToErr("alter table t set tiflash replica 1") + require.Error(t, err) + require.Equal(t, "[ddl:8200]Unsupported ALTER TiFlash settings for tables not supported by TiFlash: table contains gbk charset", err.Error()) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) charset utf8)") + tk.MustExec("alter table t set tiflash replica 1") +} + +func TestReadPartitionTable(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null) partition by hash(a) partitions 2") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("3")) + tk.MustQuery("select * from t order by a").Check(testkit.Rows("1 0", "2 0", "3 0")) + + // test union scan + tk.MustExec("begin") + tk.MustExec("insert into t values(4,0)") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("4")) + tk.MustExec("insert into t values(5,0)") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("5")) + tk.MustExec("insert into t values(6,0)") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) + // test dynamic prune + union scan + tk.MustExec("set tidb_partition_prune_mode=dynamic") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) + // test dynamic prune + batch cop + union scan + tk.MustExec("set tidb_allow_batch_cop=2") + tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("6")) + tk.MustExec("commit") +} + +func TestAggPushDownApplyAll(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists foo") + tk.MustExec("drop table if exists bar") + tk.MustExec("create table foo(a int, b int)") + tk.MustExec("create table bar(a double not null, b decimal(65,0) not null)") + tk.MustExec("alter table foo set tiflash replica 1") + tk.MustExec("alter table bar set tiflash replica 1") + tk.MustExec("insert into foo values(0, NULL)") + tk.MustExec("insert into bar values(0, 0)") + + tk.MustExec("set @@session.tidb_allow_mpp=1") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + + tk.MustQuery("select * from foo where a=all(select a from bar where bar.b=foo.b)").Check(testkit.Rows("0 ")) +} + +func TestReadUnsigedPK(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t(a bigint unsigned not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("insert into t values(18446744073709551606,0)") + tk.MustExec("insert into t values(9223372036854775798,0)") + + tk.MustExec("create table t1(a bigint unsigned not null primary key, b int not null)") + tk.MustExec("alter table t1 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t1 values(1,0)") + tk.MustExec("insert into t1 values(2,0)") + tk.MustExec("insert into t1 values(3,0)") + tk.MustExec("insert into t1 values(18446744073709551606,0)") + tk.MustExec("insert into t1 values(9223372036854775798,0)") + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("5")) + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a and ((t1.a < 9223372036854775800 and t1.a > 2) or (t1.a <= 1 and t1.a > -1))").Check(testkit.Rows("3")) +} + +// to fix https://github.com/pingcap/tidb/issues/27952 +func TestJoinRace(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,1)") + tk.MustExec("insert into t values(2,1)") + tk.MustExec("insert into t values(3,1)") + tk.MustExec("insert into t values(1,2)") + tk.MustExec("insert into t values(2,2)") + tk.MustExec("insert into t values(3,2)") + tk.MustExec("insert into t values(1,2)") + tk.MustExec("insert into t values(2,2)") + tk.MustExec("insert into t values(3,2)") + tk.MustExec("insert into t values(1,3)") + tk.MustExec("insert into t values(2,3)") + tk.MustExec("insert into t values(3,4)") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + tk.MustExec("set @@tidb_opt_broadcast_cartesian_join=0") + tk.MustQuery("select count(*) from (select count(a) x from t group by b) t1 join (select count(a) x from t group by b) t2 on t1.x > t2.x").Check(testkit.Rows("6")) +} + +func TestMppExecution(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 2") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + + tk.MustExec("create table t1(a int primary key, b int not null)") + tk.MustExec("alter table t1 set tiflash replica 2") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t1 values(1,0)") + tk.MustExec("insert into t1 values(2,0)") + tk.MustExec("insert into t1 values(3,0)") + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("set @@session.tidb_opt_enable_late_materialization=OFF") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + for i := 0; i < 20; i++ { + // test if it is stable. + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("3")) + } + // test multi-way join + tk.MustExec("create table t2(a int primary key, b int not null)") + tk.MustExec("alter table t2 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into t2 values(1,0)") + tk.MustExec("insert into t2 values(2,0)") + tk.MustExec("insert into t2 values(3,0)") + tk.MustQuery("select count(*) from t1 , t, t2 where t1.a = t.a and t2.a = t.a").Check(testkit.Rows("3")) + + // test avg + tk.MustQuery("select avg(t1.a) from t1 , t where t1.a = t.a").Check(testkit.Rows("2.0000")) + // test proj and selection + tk.MustQuery("select count(*) from (select a * 2 as a from t1) t1 , (select b + 4 as a from t)t where t1.a = t.a").Check(testkit.Rows("3")) + + // test shuffle hash join. + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size=1") + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from t1 , t, t2 where t1.a = t.a and t2.a = t.a").Check(testkit.Rows("3")) + + // test agg by expression + tk.MustExec("insert into t1 values(4,0)") + tk.MustQuery("select count(*) k, t2.b from t1 left join t2 on t1.a = t2.a group by t2.b order by k").Check(testkit.Rows("1 ", "3 0")) + tk.MustQuery("select count(*) k, t2.b+1 from t1 left join t2 on t1.a = t2.a group by t2.b+1 order by k").Check(testkit.Rows("1 ", "3 1")) + tk.MustQuery("select count(*) k, t2.b * t2.a from t2 group by t2.b * t2.a").Check(testkit.Rows("3 0")) + tk.MustQuery("select count(*) k, t2.a/2 m from t2 group by t2.a / 2 order by m").Check(testkit.Rows("1 0.5000", "1 1.0000", "1 1.5000")) + tk.MustQuery("select count(*) k, t2.a div 2 from t2 group by t2.a div 2 order by k").Check(testkit.Rows("1 0", "2 1")) + // test task id for same start ts. + tk.MustExec("begin") + tk.MustQuery("select count(*) from ( select * from t2 group by a, b) A group by A.b").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from t1 where t1.a+100 > ( select count(*) from t2 where t1.a=t2.a and t1.b=t2.b) group by t1.b").Check(testkit.Rows("4")) + taskID := plannercore.AllocMPPTaskID(tk.Session()) + require.Equal(t, int64(1), taskID) + tk.MustExec("commit") + + tk.MustQuery("select avg(t.a) from t join t t1 on t.a = t1.a").Check(testkit.Rows("2.0000")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c1 decimal(8, 5) not null, c2 decimal(9, 5), c3 decimal(9, 4) , c4 decimal(8, 4) not null)") + tk.MustExec("alter table t set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1.00000,1.00000,1.0000,1.0000)") + tk.MustExec("insert into t values(1.00010,1.00010,1.0001,1.0001)") + tk.MustExec("insert into t values(1.00001,1.00001,1.0000,1.0002)") + tk.MustQuery("select t1.c1 from t t1 join t t2 on t1.c1 = t2.c1 order by t1.c1").Check(testkit.Rows("1.00000", "1.00001", "1.00010")) + tk.MustQuery("select t1.c1 from t t1 join t t2 on t1.c1 = t2.c3 order by t1.c1").Check(testkit.Rows("1.00000", "1.00000", "1.00010")) + tk.MustQuery("select t1.c4 from t t1 join t t2 on t1.c4 = t2.c3 order by t1.c4").Check(testkit.Rows("1.0000", "1.0000", "1.0001")) + // let this query choose hash join + tk.MustQuery("select /*+ nth_plan(2) */ t1.c1 from t t1 join t t2 on t1.c1 = t2.c3 order by t1.c1").Check(testkit.Rows("1.00000", "1.00000", "1.00010")) +} + +func TestInjectExtraProj(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint(20))") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + + tk.MustQuery("select avg(a) from t").Check(testkit.Rows("9223372036854775807.0000")) + tk.MustQuery("select avg(a), a from t group by a").Check(testkit.Rows("9223372036854775807.0000 9223372036854775807")) +} + +func TestTiFlashPartitionTableShuffledHashJoin(t *testing.T) { + t.Skip("too slow") + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`create database tiflash_partition_SHJ`) + tk.MustExec("use tiflash_partition_SHJ") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int)`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := external.GetTableByName(t, tk, "tiflash_partition_SHJ", tbl) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + + tk.MustExec("SET tidb_enforce_mpp=1") + tk.MustExec("SET tidb_broadcast_join_threshold_count=0") + tk.MustExec("SET tidb_broadcast_join_threshold_size=0") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + l2, r2 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v and t2.b>=%v and t2.b<=%v", l1, r1, l2, r2) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select count(*) from %v t1 join %v t2 on t1.a=t2.a where %v", tbl, tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} + +func TestTiFlashPartitionTableReader(t *testing.T) { + t.Skip("too slow") + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`create database tiflash_partition_tablereader`) + tk.MustExec("use tiflash_partition_tablereader") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int)`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := external.GetTableByName(t, tk, "tiflash_partition_tablereader", tbl) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + } + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + vals := make([]string, 0, 500) + for i := 0; i < 500; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + } + + tk.MustExec("SET tidb_enforce_mpp=1") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + for i := 0; i < 10; i++ { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + cond := fmt.Sprintf("a>=%v and a<=%v", l, r) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{"thash", "trange", "tlist", "tnormal"} { + q := fmt.Sprintf("select * from %v where %v", tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(res) + } + } + } + } +} + +func TestPartitionTable(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=1") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t(a int not null primary key, b int not null) partition by hash(a+1) partitions 4") + // Looks like setting replica number of a region is not supported in mock store, a region always has n replicas(where n + // is the number of stores), in this test, there are 2 TiFlash store, so the TiFlash replica is always 2, change the + // TiFlash replica to 2 to make it consist with mock store. + tk.MustExec("alter table t set tiflash replica 2") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("insert into t values(4,0)") + failpoint.Enable("github.com/pingcap/tidb/executor/checkUseMPP", `return(true)`) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("set @@session.tidb_opt_enable_late_materialization=OFF") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(2)`) + tk.MustQuery("select count(*) from t").Check(testkit.Rows("4")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + tk.MustExec("set @@session.tidb_partition_prune_mode='static-only'") + // when we lift the restriction of partition table can not take MPP path, here should `return(true)` + failpoint.Enable("github.com/pingcap/tidb/executor/checkUseMPP", `return(true)`) + tk.MustQuery("select count(*) from t").Check(testkit.Rows("4")) + tk.MustExec("set @@session.tidb_partition_prune_mode='dynamic-only'") + failpoint.Enable("github.com/pingcap/tidb/executor/checkUseMPP", `return(true)`) + + tk.MustExec("create table t1(a int not null primary key, b int not null) partition by hash(a) partitions 4") + tk.MustExec("alter table t1 set tiflash replica 2") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t1 values(1,4)") + tk.MustExec("insert into t1 values(2,3)") + tk.MustExec("insert into t1 values(3,2)") + tk.MustExec("insert into t1 values(4,1)") + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // test if it is really work. + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(4)`) + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("4")) + // test partition prune + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a and t1.a < 2 and t.a < 2").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from t1 , t where t1.a = t.a and t1.a < -1 and t.a < 2").Check(testkit.Rows("0")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + // test multi-way join + tk.MustExec("create table t2(a int not null primary key, b int not null)") + tk.MustExec("alter table t2 set tiflash replica 2") + tb = external.GetTableByName(t, tk, "test", "t2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into t2 values(1,0)") + tk.MustExec("insert into t2 values(2,0)") + tk.MustExec("insert into t2 values(3,0)") + tk.MustExec("insert into t2 values(4,0)") + // test with no partition table + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(5)`) + tk.MustQuery("select count(*) from t1 , t, t2 where t1.a = t.a and t2.a = t.a").Check(testkit.Rows("4")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + + tk.MustExec(`create table t3(a int not null, b int not null) PARTITION BY RANGE (b) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (3), + PARTITION p2 VALUES LESS THAN (5), + PARTITION p3 VALUES LESS THAN (7) + );`) + tk.MustExec("alter table t3 set tiflash replica 2") + tb = external.GetTableByName(t, tk, "test", "t3") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into t3 values(1,0)") + tk.MustExec("insert into t3 values(2,2)") + tk.MustExec("insert into t3 values(3,4)") + tk.MustExec("insert into t3 values(4,6)") + + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(4)`) + tk.MustQuery("select count(*) from t, t3 where t3.a = t.a and t3.b <= 4").Check(testkit.Rows("3")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(3)`) + tk.MustQuery("select count(*) from t, t3 where t3.a = t.a and t3.b > 10").Check(testkit.Rows("0")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + failpoint.Disable("github.com/pingcap/tidb/executor/checkUseMPP") +} + +func TestMppEnum(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b enum('aca','bca','zca'))") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,'aca')") + tk.MustExec("insert into t values(2,'bca')") + tk.MustExec("insert into t values(3,'zca')") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + tk.MustQuery("select t1.b from t t1 join t t2 on t1.a = t2.a order by t1.b").Check(testkit.Rows("aca", "bca", "zca")) +} + +func TestTiFlashPlanCacheable(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("alter table test.t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv, tiflash'") + tk.MustExec("insert into t values(1);") + tk.MustExec("prepare stmt from 'select /*+ read_from_storage(tiflash[t]) */ * from t;';") + tk.MustQuery("execute stmt;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt;").Check(testkit.Rows("1")) + // The TiFlash plan can not be cached. + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + tk.MustExec("prepare stmt from 'select /*+ read_from_storage(tikv[t]) */ * from t;';") + tk.MustQuery("execute stmt;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt;").Check(testkit.Rows("1")) + // The TiKV plan can be cached. + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + // test the mpp plan + tk.MustExec("set @@session.tidb_allow_mpp = 1;") + tk.MustExec("set @@session.tidb_enforce_mpp = 1;") + tk.MustExec("prepare stmt from 'select count(t1.a) from t t1 join t t2 on t1.a = t2.a where t1.a > ?;';") + tk.MustExec("set @a = 0;") + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) + + tk.MustExec("set @a = 1;") + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("0")) + // The TiFlash plan can not be cached. + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestDispatchTaskRetry(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("insert into t values(4,0)") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/mppDispatchTimeout", "3*return(true)")) + tk.MustQuery("select count(*) from t group by b").Check(testkit.Rows("4")) + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/mppDispatchTimeout")) +} + +func TestMppVersionError(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("insert into t values(1,0),(2,0),(3,0),(4,0)") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + { + item := fmt.Sprintf("return(%d)", kv.GetNewestMppVersion()+1) + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/MppVersionError", item)) + } + { + err := tk.QueryToErr("select count(*) from t group by b") + require.Error(t, err) + } + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/MppVersionError")) + { + item := fmt.Sprintf("return(%d)", kv.GetNewestMppVersion()) + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/MppVersionError", item)) + } + { + tk.MustQuery("select count(*) from t group by b").Check(testkit.Rows("4")) + } + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/MppVersionError")) +} + +func TestCancelMppTasks(t *testing.T) { + var hang = "github.com/pingcap/tidb/store/mockstore/unistore/mppRecvHang" + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("insert into t values(4,0)") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + atomic.StoreUint32(&tk.Session().GetSessionVars().Killed, 0) + require.Nil(t, failpoint.Enable(hang, `return(true)`)) + wg := &sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + err := tk.QueryToErr("select count(*) from t as t1 , t where t1.a = t.a") + require.Error(t, err) + require.Equal(t, int(exeerrors.ErrQueryInterrupted.Code()), int(terror.ToSQLError(errors.Cause(err).(*terror.Error)).Code)) + }() + time.Sleep(1 * time.Second) + atomic.StoreUint32(&tk.Session().GetSessionVars().Killed, 1) + wg.Wait() + require.Nil(t, failpoint.Disable(hang)) +} + +// all goroutines exit if one goroutine hangs but another return errors +func TestMppGoroutinesExitFromErrors(t *testing.T) { + // mock non-root tasks return error + var mppNonRootTaskError = "github.com/pingcap/tidb/executor/internal/mpp/mppNonRootTaskError" + // mock root tasks hang + var hang = "github.com/pingcap/tidb/store/mockstore/unistore/mppRecvHang" + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,0)") + tk.MustExec("insert into t values(2,0)") + tk.MustExec("insert into t values(3,0)") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a int not null primary key, b int not null)") + tk.MustExec("alter table t1 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t1 values(1,0)") + tk.MustExec("insert into t1 values(2,0)") + tk.MustExec("insert into t1 values(3,0)") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + require.Nil(t, failpoint.Enable(mppNonRootTaskError, `return(true)`)) + require.Nil(t, failpoint.Enable(hang, `return(true)`)) + + // generate 2 root tasks, one will hang and another will return errors + err = tk.QueryToErr("select count(*) from t as t1 , t where t1.a = t.a") + require.Error(t, err) + require.Nil(t, failpoint.Disable(mppNonRootTaskError)) + require.Nil(t, failpoint.Disable(hang)) +} + +func TestMppUnionAll(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists x1") + tk.MustExec("create table x1(a int , b int);") + tk.MustExec("alter table x1 set tiflash replica 2") + tk.MustExec("drop table if exists x2") + tk.MustExec("create table x2(a int , b int);") + tk.MustExec("alter table x2 set tiflash replica 2") + tb := external.GetTableByName(t, tk, "test", "x1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tb = external.GetTableByName(t, tk, "test", "x2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into x1 values (1, 1), (2, 2), (3, 3), (4, 4)") + tk.MustExec("insert into x2 values (5, 1), (2, 2), (3, 3), (4, 4)") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + // test join + union (join + select) + tk.MustQuery("select x1.a, x.a from x1 left join (select x2.b a, x1.b from x1 join x2 on x1.a = x2.b union all select * from x1 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "2 2", "2 2", "3 3", "3 3", "4 4", "4 4")) + tk.MustQuery("select x1.a, x.a from x1 left join (select count(*) a, sum(b) b from x1 group by a union all select * from x2 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "1 1", "1 1", "2 2", "3 3", "4 4")) + + tk.MustExec("drop table if exists x3") + tk.MustExec("create table x3(a int , b int);") + tk.MustExec("alter table x3 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "x3") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into x3 values (2, 2), (2, 3), (2, 4)") + // test nested union all + tk.MustQuery("select count(*) from (select a, b from x1 union all select a, b from x3 union all (select x1.a, x3.b from (select * from x3 union all select * from x2) x3 left join x1 on x3.a = x1.b))").Check(testkit.Rows("14")) + // test union all join union all + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count=100000") + failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks", `return(6)`) + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/checkTotalMPPTasks") + + tk.MustExec("drop table if exists x4") + tk.MustExec("create table x4(a int not null, b int not null);") + tk.MustExec("alter table x4 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "x4") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("set @@tidb_enforce_mpp=1") + tk.MustExec("insert into x4 values (2, 2), (2, 3)") + tk.MustQuery("(select * from x1 union all select * from x4) order by a, b").Check(testkit.Rows("1 1", "2 2", "2 2", "2 3", "3 3", "4 4")) +} + +func TestUnionWithEmptyDualTable(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t (a int not null, b int, c varchar(20))") + tk.MustExec("create table t1 (a int, b int not null, c double)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("alter table t1 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1,2,3)") + tk.MustExec("insert into t1 values(1,2,3)") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + tk.MustQuery("select count(*) from (select a , b from t union all select a , c from t1 where false) tt").Check(testkit.Rows("1")) +} + +func TestAvgOverflow(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + // avg int + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a decimal(1,0))") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(9)") + for i := 0; i < 16; i++ { + tk.MustExec("insert into t select * from t") + } + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + tk.MustQuery("select avg(a) from t group by a").Check(testkit.Rows("9.0000")) + tk.MustExec("drop table if exists t") + + // avg decimal + tk.MustExec("drop table if exists td;") + tk.MustExec("create table td (col_bigint bigint(20), col_smallint smallint(6));") + tk.MustExec("alter table td set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "td") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into td values (null, 22876);") + tk.MustExec("insert into td values (9220557287087669248, 32767);") + tk.MustExec("insert into td values (28030, 32767);") + tk.MustExec("insert into td values (-3309864251140603904,32767);") + tk.MustExec("insert into td values (4,0);") + tk.MustExec("insert into td values (null,0);") + tk.MustExec("insert into td values (4,-23828);") + tk.MustExec("insert into td values (54720,32767);") + tk.MustExec("insert into td values (0,29815);") + tk.MustExec("insert into td values (10017,-32661);") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_enforce_mpp=ON") + tk.MustExec("set @@session.tidb_opt_enable_late_materialization=OFF") + tk.MustQuery(" SELECT AVG( col_bigint / col_smallint) AS field1 FROM td;").Sort().Check(testkit.Rows("25769363061037.62077260")) + tk.MustQuery(" SELECT AVG(col_bigint) OVER (PARTITION BY col_smallint) as field2 FROM td where col_smallint = -23828;").Sort().Check(testkit.Rows("4.0000")) + tk.MustExec("drop table if exists td;") +} + +func TestMppApply(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists x1") + tk.MustExec("create table x1(a int primary key, b int);") + tk.MustExec("alter table x1 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "x1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into x1 values(1, 1),(2, 10),(0,11);") + + tk.MustExec("create table x2(a int primary key, b int);") + tk.MustExec("alter table x2 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "x2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into x2 values(1,2),(0,1),(2,-3);") + tk.MustExec("analyze table x1, x2;") + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + // table full scan with correlated filter + tk.MustQuery("select /*+ agg_to_cop(), hash_agg()*/ count(*) from x1 where a >= any (select a from x2 where x1.a = x2.a) order by 1;").Check(testkit.Rows("3")) + // table range scan with correlated access conditions + tk.MustQuery("select /*+ agg_to_cop(), hash_agg()*/ count(*) from x1 where b > any (select x2.a from x2 where x1.a = x2.a);").Check(testkit.Rows("2")) +} + +func TestTiFlashVirtualColumn(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1,t2,t3") + tk.MustExec("create table t1 (a bit(4), b bit(4), c bit(4) generated always as (a) virtual)") + tk.MustExec("alter table t1 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t1(a,b) values(b'01',b'01'),(b'10',b'10'),(b'11',b'11')") + + tk.MustExec("create table t2 (a int, b int, c int generated always as (a) virtual)") + tk.MustExec("alter table t2 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t2(a,b) values(1,1),(2,2),(3,3)") + + tk.MustExec("create table t3 (a bit(4), b bit(4), c bit(4) generated always as (b'01'+b'10') virtual)") + tk.MustExec("alter table t3 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t3") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t3(a,b) values(b'01',b'01'),(b'10',b'10'),(b'11',b'11')") + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + tk.MustQuery("select /*+ hash_agg() */ count(*) from t1 where c > b'01'").Check(testkit.Rows("2")) + tk.MustQuery("select /*+ hash_agg() */ count(*) from t2 where c > 1").Check(testkit.Rows("2")) + tk.MustQuery("select /*+ hash_agg() */ count(*) from t3 where c > b'01'").Check(testkit.Rows("3")) +} + +func TestTiFlashPartitionTableShuffledHashAggregation(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database tiflash_partition_AGG") + tk.MustExec("use tiflash_partition_AGG") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int) partition by hash(a) partitions 4`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := external.GetTableByName(t, tk, "tiflash_partition_AGG", tbl) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v", l1, r1) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select /*+ HASH_AGG() */ count(*) from %v t1 where %v", tbl, cond) + require.True(t, tk.HasPlan(q, "HashAgg")) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} + +func TestTiFlashPartitionTableBroadcastJoin(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database tiflash_partition_BCJ") + tk.MustExec("use tiflash_partition_BCJ") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int) partition by hash(a) partitions 4`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := external.GetTableByName(t, tk, "tiflash_partition_BCJ", tbl) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + l2, r2 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v and t2.b>=%v and t2.b<=%v", l1, r1, l2, r2) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select count(*) from %v t1 join %v t2 on t1.a=t2.a where %v", tbl, tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} + +func TestTiflashSupportStaleRead(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint(20))") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + time.Sleep(2 * time.Second) + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + tk.MustExec("insert into t values (9223372036854775807)") + rows := tk.MustQuery("explain select avg(a) from t").Rows() + resBuff := bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res := resBuff.String() + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") + tk.MustExec("set transaction read only as of timestamp now(1)") + rows = tk.MustQuery("explain select avg(a) from t").Rows() + resBuff = bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res = resBuff.String() + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") +} + +func TestForbidTiFlashIfExtraPhysTableIDIsNeeded(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null primary key, b int not null) partition by hash(a) partitions 2") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set tidb_partition_prune_mode=dynamic") + tk.MustExec("set tidb_enforce_mpp=1") + tk.MustExec("set tidb_cost_model_version=2") + + rows := tk.MustQuery("explain select count(*) from t").Rows() + resBuff := bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res := resBuff.String() + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") + + rows = tk.MustQuery("explain select count(*) from t for update").Rows() + resBuff = bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res = resBuff.String() + require.NotContains(t, res, "tiflash") + require.Contains(t, res, "tikv") + + tk.MustExec("begin") + rows = tk.MustQuery("explain select count(*) from t").Rows() + resBuff = bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res = resBuff.String() + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") + tk.MustExec("insert into t values(1,2)") + rows = tk.MustQuery("explain select count(*) from t").Rows() + resBuff = bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res = resBuff.String() + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") + tk.MustExec("rollback") +} + +func TestTiflashPartitionTableScan(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(\n a int,\n primary key(a)\n) partition by range(a) (\n partition p1 values less than (10),\n partition p2 values less than (20),\n partition p3 values less than (30),\n partition p4 values less than (40),\n partition p5 values less than (50)\n);") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into t values(1),(11),(21),(31),(41);") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\";") + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + // MPP + tk.MustExec("set @@session.tidb_allow_mpp=ON;") + tk.MustQuery("select count(*) from t where a < 12;").Check(testkit.Rows("2")) + + // BatchCop + tk.MustExec("set @@session.tidb_allow_mpp=OFF;") + tk.MustExec("set @@tidb_allow_batch_cop = 2;") + tk.MustQuery("select count(*) from t where a < 12;").Check(testkit.Rows("2")) + + // test retry batch cop + // MPP + wg := sync.WaitGroup{} + wg.Add(1) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", `return(true)`)) + go func() { + time.Sleep(100 * time.Millisecond) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy")) + wg.Done() + }() + tk.MustExec("set @@session.tidb_allow_mpp=ON;") + tk.MustQuery("select count(*) from t where a < 12;").Check(testkit.Rows("2")) + wg.Wait() + + // BatchCop + wg.Add(1) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", `return(true)`)) + go func() { + time.Sleep(100 * time.Millisecond) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy")) + wg.Done() + }() + tk.MustExec("set @@session.tidb_allow_mpp=OFF;") + tk.MustExec("set @@tidb_allow_batch_cop = 2;") + tk.MustQuery("select count(*) from t where a < 12;").Check(testkit.Rows("2")) + wg.Wait() +} + +func TestAggPushDownCountStar(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists c") + tk.MustExec("drop table if exists o") + tk.MustExec("create table c(c_id bigint primary key)") + tk.MustExec("create table o(o_id bigint primary key, c_id bigint not null)") + tk.MustExec("alter table c set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "c") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("alter table o set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "o") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("insert into c values(1),(2),(3),(4),(5)") + tk.MustExec("insert into o values(1,1),(2,1),(3,2),(4,2),(5,2)") + + tk.MustExec("set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_opt_agg_push_down=1") + + tk.MustQuery("select count(*) from c, o where c.c_id=o.c_id").Check(testkit.Rows("5")) +} + +func TestAggPushDownUnionAndMPP(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("set @@tidb_allow_mpp=1;") + tk.MustExec("set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_opt_agg_push_down=1") + + tk.MustExec("create table c(c_id int)") + tk.MustExec("create table o(o_id int, c_id int)") + tk.MustExec("insert into c values(1),(1),(1),(1)") + tk.MustExec("insert into o values(1,1),(1,1),(1,2)") + tk.MustExec("alter table c set tiflash replica 1") + tk.MustExec("alter table o set tiflash replica 1") + + tk.MustQuery("select a, count(*) from (select a, b from t " + + "union all " + + "select a, b from t" + + ") t group by a order by a limit 10;").Check(testkit.Rows("1 10")) + + tk.MustQuery("select o.o_id, count(*) from c, o where c.c_id=o.o_id group by o.o_id").Check(testkit.Rows("1 12")) +} + +func TestGroupStreamAggOnTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists foo") + tk.MustExec("create table foo(a int, b int, c int, d int, primary key(a,b,c,d))") + tk.MustExec("alter table foo set tiflash replica 1") + tk.MustExec("insert into foo values(1,2,3,1),(1,2,3,6),(1,2,3,5)," + + "(1,2,3,2),(1,2,3,4),(1,2,3,7),(1,2,3,3),(1,2,3,0)") + tb := external.GetTableByName(t, tk, "test", "foo") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@tidb_allow_mpp=0") + sql := "select a,b,c,count(*) from foo group by a,b,c order by a,b,c" + tk.MustQuery(sql).Check(testkit.Rows("1 2 3 8")) + rows := tk.MustQuery("explain " + sql).Rows() + + for _, row := range rows { + resBuff := bytes.NewBufferString("") + fmt.Fprintf(resBuff, "%s\n", row) + res := resBuff.String() + // StreamAgg with group keys on TiFlash is not supported + if strings.Contains(res, "tiflash") { + require.NotContains(t, res, "StreamAgg") + } + } +} + +// TestIssue41014 test issue that can't find proper physical plan +func TestIssue41014(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("CREATE TABLE `tai1` (\n `aid` int(11) DEFAULT NULL,\n `rid` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + tk.MustExec("CREATE TABLE `tai2` (\n `rid` int(11) DEFAULT NULL,\n `prilan` varchar(20) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + tk.MustExec("alter table tai1 set tiflash replica 1") + tk.MustExec("alter table tai2 set tiflash replica 1") + tk.MustExec("alter table tai2 add index idx((lower(prilan)));") + tk.MustExec("set @@tidb_opt_distinct_agg_push_down = 1;") + + tk.MustQuery("explain select count(distinct tai1.aid) as cb from tai1 inner join tai2 on tai1.rid = tai2.rid where lower(prilan) LIKE LOWER('%python%');").Check( + testkit.Rows("HashAgg_11 1.00 root funcs:count(distinct test.tai1.aid)->Column#8", + "└─HashJoin_15 9990.00 root inner join, equal:[eq(test.tai2.rid, test.tai1.rid)]", + " ├─Selection_20(Build) 8000.00 root like(lower(test.tai2.prilan), \"%python%\", 92)", + " │ └─Projection_19 10000.00 root test.tai2.rid, lower(test.tai2.prilan)", + " │ └─TableReader_18 9990.00 root data:Selection_17", + " │ └─Selection_17 9990.00 cop[tikv] not(isnull(test.tai2.rid))", + " │ └─TableFullScan_16 10000.00 cop[tikv] table:tai2 keep order:false, stats:pseudo", + " └─TableReader_23(Probe) 9990.00 root data:Selection_22", + " └─Selection_22 9990.00 cop[tikv] not(isnull(test.tai1.rid))", + " └─TableFullScan_21 10000.00 cop[tikv] table:tai1 keep order:false, stats:pseudo")) + tk.MustQuery("select count(distinct tai1.aid) as cb from tai1 inner join tai2 on tai1.rid = tai2.rid where lower(prilan) LIKE LOWER('%python%');").Check( + testkit.Rows("0")) +} + +func TestTiflashEmptyDynamicPruneResult(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `IDT_RP24833` ( `COL1` bigint(16) DEFAULT '15' COMMENT 'NUMERIC UNIQUE INDEX',\n `COL2` varchar(20) DEFAULT NULL,\n `COL4` datetime DEFAULT NULL,\n `COL3` bigint(20) DEFAULT NULL,\n `COL5` float DEFAULT NULL,\n KEY `UK_COL1` (`COL1`) /*!80000 INVISIBLE */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\nPARTITION BY RANGE ((`COL1`-57))\n(PARTITION `P0` VALUES LESS THAN (-3503857335115112215),\n PARTITION `P1` VALUES LESS THAN (-2987877108151063747),\n PARTITION `P2` VALUES LESS THAN (-1981049919102122710),\n PARTITION `P3` VALUES LESS THAN (-1635802972727465681),\n PARTITION `P4` VALUES LESS THAN (1186020639986357714),\n PARTITION `P5` VALUES LESS THAN (1220018677454711359),\n PARTITION `PMX` VALUES LESS THAN (MAXVALUE));") + tk.MustExec("alter table IDT_RP24833 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "IDT_RP24833") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("insert into IDT_RP24833 values(-8448770111093677011, \"郇鋺篤堯擈斥鍮啸赠璭饱磟朅闑傒聎疫ᛄ怖霃\", \"8781-05-02 04:23:03\", -27252736532807028, -1.34554e38);") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\";") + tk.MustExec("set @@session.tidb_allow_mpp=ON;") + tk.MustExec("set @@session.tidb_enforce_mpp = on;") + tk.MustQuery("select /*+ read_from_storage(tiflash[t1]) */ * from IDT_RP24833 partition(p3, p4) t1 where t1. col1 between -8448770111093677011 and -8448770111093677011;").Check(testkit.Rows()) + tk.MustQuery("select /*+ read_from_storage(tiflash[t2]) */ * from IDT_RP24833 partition(p2) t2 where t2. col1 <= -8448770111093677011;").Check(testkit.Rows()) + tk.MustQuery("select /*+ read_from_storage(tiflash[t1, t2]) */ * from IDT_RP24833 partition(p3, p4) t1 join IDT_RP24833 partition(p2) t2 on t1.col1 = t2.col1 where t1. col1 between -8448770111093677011 and -8448770111093677011 and t2. col1 <= -8448770111093677011;").Check(testkit.Rows()) +} + +func TestDisaggregatedTiFlash(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + conf.UseAutoScaler = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + conf.UseAutoScaler = false + }) + err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.TestASStr, "tmpAddr", "tmpClusterID", false) + require.NoError(t, err) + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tk.ExecToErr("select * from t;") + // Expect error, because TestAutoScaler return empty topo. + require.Contains(t, err.Error(), "Cannot find proper topo to dispatch MPPTask: topo from AutoScaler is empty") + + err = tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.AWSASStr, "tmpAddr", "tmpClusterID", false) + require.NoError(t, err) + err = tk.ExecToErr("select * from t;") + // Expect error, because AWSAutoScaler is not setup, so http request will fail. + require.Contains(t, err.Error(), "[util:1815]Internal : get tiflash_compute topology failed") +} + +// todo: remove this after AutoScaler is stable. +func TestDisaggregatedTiFlashNonAutoScaler(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + conf.UseAutoScaler = false + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + conf.UseAutoScaler = true + }) + + // Setting globalTopoFetcher to nil to can make sure cannot fetch topo from AutoScaler. + err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.InvalidASStr, "tmpAddr", "tmpClusterID", false) + require.Contains(t, err.Error(), "unexpected topo fetch type. expect: mock or aws or gcp, got invalid") + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tk.ExecToErr("select * from t;") + // This error message means we use PD instead of AutoScaler. + require.Contains(t, err.Error(), "tiflash_compute node is unavailable") +} + +func TestDisaggregatedTiFlashQuery(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl_1") + tk.MustExec(`create table tbl_1 ( col_1 bigint not null default -1443635317331776148, + col_2 text ( 176 ) collate utf8mb4_bin not null, + col_3 decimal ( 8, 3 ), + col_4 varchar ( 128 ) collate utf8mb4_bin not null, + col_5 varchar ( 377 ) collate utf8mb4_bin, + col_6 double, + col_7 varchar ( 459 ) collate utf8mb4_bin, + col_8 tinyint default -88 ) charset utf8mb4 collate utf8mb4_bin ;`) + tk.MustExec("alter table tbl_1 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "tbl_1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + tk.MustExec("explain select max( tbl_1.col_1 ) as r0 , sum( tbl_1.col_1 ) as r1 , sum( tbl_1.col_8 ) as r2 from tbl_1 where tbl_1.col_8 != 68 or tbl_1.col_3 between null and 939 order by r0,r1,r2;") + + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("create table t1(c1 int, c2 int) partition by hash(c1) partitions 3") + tk.MustExec("insert into t1 values(1, 1), (2, 2), (3, 3)") + tk.MustExec("alter table t1 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustQuery("explain select * from t1 where c1 < 2").Check(testkit.Rows( + "PartitionUnion_10 9970.00 root ", + "├─TableReader_15 3323.33 root MppVersion: 2, data:ExchangeSender_14", + "│ └─ExchangeSender_14 3323.33 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection_13 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + "│ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", + "├─TableReader_19 3323.33 root MppVersion: 2, data:ExchangeSender_18", + "│ └─ExchangeSender_18 3323.33 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection_17 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + "│ └─TableFullScan_16 10000.00 mpp[tiflash] table:t1, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", + "└─TableReader_23 3323.33 root MppVersion: 2, data:ExchangeSender_22", + " └─ExchangeSender_22 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_21 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + " └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo")) +} + +func TestMPPMemoryTracker(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set tidb_mem_quota_query = 1 << 30") + tk.MustExec("set global tidb_mem_oom_action = 'CANCEL'") + tk.MustExec("use test") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values (1);") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set tidb_enforce_mpp = on;") + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/internal/mpp/testMPPOOMPanic", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/internal/mpp/testMPPOOMPanic")) + }() + err = tk.QueryToErr("select * from t") + require.NotNil(t, err) + require.True(t, strings.Contains(err.Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery)) +} + +func TestTiFlashComputeDispatchPolicy(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + + var err error + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // Default policy is 'consistent_hash' + tk.MustQuery("select @@tiflash_compute_dispatch_policy").Check(testkit.Rows("consistent_hash")) + + // tiflash_compute_dispatch_policy is global variable. + tk.MustExec("set @@session.tiflash_compute_dispatch_policy = 'consistent_hash';") + tk.MustQuery("select @@tiflash_compute_dispatch_policy").Check(testkit.Rows("consistent_hash")) + tk.MustExec("set @@session.tiflash_compute_dispatch_policy = 'round_robin';") + tk.MustQuery("select @@tiflash_compute_dispatch_policy").Check(testkit.Rows("round_robin")) + err = tk.ExecToErr("set @@session.tiflash_compute_dispatch_policy = 'error_dispatch_policy';") + require.Error(t, err) + require.Equal(t, "unexpected tiflash_compute dispatch policy, expect [consistent_hash round_robin], got error_dispatch_policy", err.Error()) + + // Invalid values. + err = tk.ExecToErr("set global tiflash_compute_dispatch_policy = 'error_dispatch_policy';") + require.Error(t, err) + require.Equal(t, "unexpected tiflash_compute dispatch policy, expect [consistent_hash round_robin], got error_dispatch_policy", err.Error()) + err = tk.ExecToErr("set global tiflash_compute_dispatch_policy = '';") + require.Error(t, err) + require.Equal(t, "unexpected tiflash_compute dispatch policy, expect [consistent_hash round_robin], got ", err.Error()) + err = tk.ExecToErr("set global tiflash_compute_dispatch_policy = 100;") + require.Error(t, err) + require.Equal(t, "unexpected tiflash_compute dispatch policy, expect [consistent_hash round_robin], got 100", err.Error()) + + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.TestASStr, "tmpAddr", "tmpClusterID", false) + require.NoError(t, err) + + useASs := []bool{true, false} + // Valid values. + defer failpoint.Disable("github.com/pingcap/tidb/store/copr/testWhichDispatchPolicy") + for _, useAS := range useASs { + config.UpdateGlobal(func(conf *config.Config) { + conf.UseAutoScaler = useAS + }) + validPolicies := tiflashcompute.GetValidDispatchPolicy() + for _, p := range validPolicies { + tk.MustExec(fmt.Sprintf("set global tiflash_compute_dispatch_policy = '%s';", p)) + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustQuery("select @@tiflash_compute_dispatch_policy").Check(testkit.Rows(p)) + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/testWhichDispatchPolicy", fmt.Sprintf(`return("%s")`, p))) + err = tk1.ExecToErr("select * from t;") + if useAS { + // Expect error, because TestAutoScaler return empty topo. + require.Contains(t, err.Error(), "Cannot find proper topo to dispatch MPPTask: topo from AutoScaler is empty") + } else { + // This error message means we use PD instead of AutoScaler. + require.Contains(t, err.Error(), "tiflash_compute node is unavailable") + } + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/testWhichDispatchPolicy")) + } + } +} + +func TestDisaggregatedTiFlashGeneratedColumn(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 varchar(100), c2 varchar(100) AS (lower(c1)));") + tk.MustExec("insert into t1(c1) values('ABC');") + tk.MustExec("alter table t1 set tiflash replica 1;") + tb := external.GetTableByName(t, tk, "test", "t1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("drop table if exists t2;") + tk.MustExec("create table t2(c1 int, c2 varchar(100));") + tk.MustExec("insert into t2 values(1, 'xhy'), (2, 'abc');") + tk.MustExec("alter table t2 set tiflash replica 1;") + tb = external.GetTableByName(t, tk, "test", "t2") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("alter table t2 add index idx2((lower(c2)));") + + nthPlan := 100 + test1 := func(forceTiFlash bool) { + if forceTiFlash { + tk.MustExec("set tidb_isolation_read_engines = 'tiflash'") + } else { + tk.MustExec("set tidb_isolation_read_engines = 'tikv,tiflash'") + } + sqls := []string{ + "explain select /*+ nth_plan(%d) */ * from t2 where lower(c2) = 'abc';", + "explain select /*+ nth_plan(%d) */ count(*) from t2 where lower(c2) = 'abc';", + "explain select /*+ nth_plan(%d) */ count(c1) from t2 where lower(c2) = 'abc';", + } + for _, sql := range sqls { + var genTiFlashPlan bool + var selectionPushdownTiFlash bool + var aggPushdownTiFlash bool + + for i := 0; i < nthPlan; i++ { + s := fmt.Sprintf(sql, i) + rows := tk.MustQuery(s).Rows() + for _, row := range rows { + line := fmt.Sprintf("%v", row) + if strings.Contains(line, "tiflash") { + genTiFlashPlan = true + } + if strings.Contains(line, "Selection") && strings.Contains(line, "tiflash") { + selectionPushdownTiFlash = true + } + if strings.Contains(line, "Agg") && strings.Contains(line, "tiflash") { + aggPushdownTiFlash = true + } + } + } + if forceTiFlash { + // Can generate tiflash plan, also Agg/Selection can push down to tiflash. + require.True(t, genTiFlashPlan) + require.True(t, selectionPushdownTiFlash) + if strings.Contains(sql, "count") { + require.True(t, aggPushdownTiFlash) + } + } else { + // Can generate tiflash plan, but Agg/Selection cannot push down to tiflash. + require.True(t, genTiFlashPlan) + require.False(t, selectionPushdownTiFlash) + if strings.Contains(sql, "count") { + require.False(t, aggPushdownTiFlash) + } + } + } + } + + test2 := func() { + // Can generate tiflash plan when select generated column. + // But Agg cannot push down to tiflash. + sqls := []string{ + "explain select /*+ nth_plan(%d) */ * from t1;", + "explain select /*+ nth_plan(%d) */ c2 from t1;", + "explain select /*+ nth_plan(%d) */ count(c2) from t1;", + } + for _, sql := range sqls { + var genTiFlashPlan bool + var aggPushdownTiFlash bool + for i := 0; i < nthPlan; i++ { + s := fmt.Sprintf(sql, i) + rows := tk.MustQuery(s).Rows() + for _, row := range rows { + line := fmt.Sprintf("%v", row) + if strings.Contains(line, "tiflash") { + genTiFlashPlan = true + } + if strings.Contains(line, "tiflash") && strings.Contains(line, "Agg") { + aggPushdownTiFlash = true + } + } + } + require.True(t, genTiFlashPlan) + if strings.Contains(sql, "count") { + require.False(t, aggPushdownTiFlash) + } + } + } + + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + test1(true) + test1(false) + test2() + + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + test1(true) + test1(false) + test2() +} + +func TestMppStoreCntWithErrors(t *testing.T) { + // mock non-root tasks return error + var mppStoreCountPDError = "github.com/pingcap/tidb/store/copr/mppStoreCountPDError" + var mppStoreCountSetMPPCnt = "github.com/pingcap/tidb/store/copr/mppStoreCountSetMPPCnt" + var mppStoreCountSetLastUpdateTime = "github.com/pingcap/tidb/store/copr/mppStoreCountSetLastUpdateTime" + var mppStoreCountSetLastUpdateTimeP2 = "github.com/pingcap/tidb/store/copr/mppStoreCountSetLastUpdateTimeP2" + + store := testkit.CreateMockStore(t, withMockTiFlash(3)) + { + mppCnt, err := store.GetMPPClient().GetMPPStoreCount() + require.Nil(t, err) + require.Equal(t, mppCnt, 3) + } + require.Nil(t, failpoint.Enable(mppStoreCountSetMPPCnt, `return(1000)`)) + { + mppCnt, err := store.GetMPPClient().GetMPPStoreCount() + require.Nil(t, err) + // meet cache + require.Equal(t, mppCnt, 3) + } + require.Nil(t, failpoint.Enable(mppStoreCountSetLastUpdateTime, `return("0")`)) + { + mppCnt, err := store.GetMPPClient().GetMPPStoreCount() + require.Nil(t, err) + // update cache + require.Equal(t, mppCnt, 1000) + } + require.Nil(t, failpoint.Enable(mppStoreCountPDError, `return(true)`)) + { + _, err := store.GetMPPClient().GetMPPStoreCount() + require.Error(t, err) + } + require.Nil(t, failpoint.Disable(mppStoreCountPDError)) + require.Nil(t, failpoint.Enable(mppStoreCountSetMPPCnt, `return(2222)`)) + // set last update time to the latest + require.Nil(t, failpoint.Enable(mppStoreCountSetLastUpdateTime, fmt.Sprintf(`return("%d")`, time.Now().UnixMicro()))) + { + mppCnt, err := store.GetMPPClient().GetMPPStoreCount() + require.Nil(t, err) + // still update cache + require.Equal(t, mppCnt, 2222) + } + require.Nil(t, failpoint.Enable(mppStoreCountSetLastUpdateTime, `return("1")`)) + // fail to get lock and old cache + require.Nil(t, failpoint.Enable(mppStoreCountSetLastUpdateTimeP2, `return("2")`)) + require.Nil(t, failpoint.Enable(mppStoreCountPDError, `return(true)`)) + { + mppCnt, err := store.GetMPPClient().GetMPPStoreCount() + require.Nil(t, err) + require.Equal(t, mppCnt, 2222) + } + require.Nil(t, failpoint.Disable(mppStoreCountSetMPPCnt)) + require.Nil(t, failpoint.Disable(mppStoreCountSetLastUpdateTime)) + require.Nil(t, failpoint.Disable(mppStoreCountSetLastUpdateTimeP2)) + require.Nil(t, failpoint.Disable(mppStoreCountPDError)) +} + +func TestUnionScan(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_allow_mpp=1") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + tk.MustExec("set @@session.tidb_allow_tiflash_cop=off") + + for x := 0; x < 2; x++ { + tk.MustExec("drop table if exists t") + if x == 0 { + // Test cache table. + tk.MustExec("create table t(a int not null primary key, b int not null)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("alter table t cache") + } else { + // Test dirty transaction. + tk.MustExec("create table t(a int not null primary key, b int not null) partition by hash(a) partitions 2") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + } + + insertStr := "insert into t values(0, 0)" + for i := 1; i < 10; i++ { + insertStr += fmt.Sprintf(",(%d, %d)", i, i) + } + tk.MustExec(insertStr) + + if x != 0 { + // Test dirty transaction. + tk.MustExec("begin") + } + + // Test Basic. + sql := "select /*+ READ_FROM_STORAGE(tiflash[t]) */ count(1) from t" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("10")) + + // Test Delete. + tk.MustExec("delete from t where a = 0") + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ count(1) from t" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("9")) + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ a, b from t order by 1" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9")) + + // Test Insert. + tk.MustExec("insert into t values(100, 100)") + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ count(1) from t" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("10")) + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ a, b from t order by 1, 2" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9", "100 100")) + + // Test Update + tk.MustExec("update t set b = 200 where a = 100") + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ count(1) from t" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("10")) + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ a, b from t order by 1, 2" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9", "100 200")) + + if x != 0 { + // Test dirty transaction. + tk.MustExec("commit") + } + + sql = "select /*+ READ_FROM_STORAGE(tiflash[t]) */ count(1) from t" + checkMPPInExplain(t, tk, "explain "+sql) + tk.MustQuery(sql).Check(testkit.Rows("10")) + + if x == 0 { + tk.MustExec("alter table t nocache") + } + } +} + +func checkMPPInExplain(t *testing.T, tk *testkit.TestKit, sql string) { + rows := tk.MustQuery(sql).Rows() + resBuff := bytes.NewBufferString("") + for _, row := range rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + res := resBuff.String() + require.Contains(t, res, "mpp[tiflash]") +} diff --git a/planner/core/casetest/enforcempp/enforce_mpp_test.go b/planner/core/casetest/enforcempp/enforce_mpp_test.go new file mode 100644 index 0000000000000..b0e20e36e01a9 --- /dev/null +++ b/planner/core/casetest/enforcempp/enforce_mpp_test.go @@ -0,0 +1,723 @@ +// Copyright 2023 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 enforcempp + +import ( + "strings" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner/core/internal" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/collate" + "github.com/stretchr/testify/require" +) + +func TestEnforceMPP(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("create index idx on t(a)") + tk.MustExec("CREATE TABLE `s` (\n `a` int(11) DEFAULT NULL,\n `b` int(11) DEFAULT NULL,\n `c` int(11) DEFAULT NULL,\n `d` int(11) DEFAULT NULL,\n UNIQUE KEY `a` (`a`),\n KEY `ii` (`a`,`b`)\n)") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + // since allow-mpp is adjusted to false, there will be no physical plan if TiFlash cop is banned. + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + if tblInfo.Name.L == "s" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + filterWarnings := func(originalWarnings []stmtctx.SQLWarn) []stmtctx.SQLWarn { + warnings := make([]stmtctx.SQLWarn, 0, 4) + for _, warning := range originalWarnings { + // filter out warning about skyline pruning + if !strings.Contains(warning.Err.Error(), "remain after pruning paths for") { + warnings = append(warnings, warning) + } + } + return warnings + } + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))) + } +} + +// general cases. +func TestEnforceMPPWarning1(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int as (a+1), c enum('xx', 'yy'), d bit(1))") + tk.MustExec("create index idx on t(a)") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + if strings.HasPrefix(tt, "cmd: create-replica") { + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: false, + } + } + } + continue + } + if strings.HasPrefix(tt, "cmd: enable-replica") { + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// partition table. +func TestEnforceMPPWarning2(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t (a int, b char(20)) PARTITION BY HASH(a)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// new collation. +func TestEnforceMPPWarning3(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t (a int, b char(20))") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + if strings.HasPrefix(tt, "cmd: enable-new-collation") { + collate.SetNewCollationEnabledForTest(true) + continue + } + if strings.HasPrefix(tt, "cmd: disable-new-collation") { + collate.SetNewCollationEnabledForTest(false) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } + collate.SetNewCollationEnabledForTest(true) +} + +// Test enforce mpp warning for joins +func TestEnforceMPPWarning4(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t(a int primary key)") + tk.MustExec("drop table if exists s") + tk.MustExec("CREATE TABLE s(a int primary key)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" || tblInfo.Name.L == "s" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test agg push down for MPP mode +func TestMPP2PhaseAggPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists c") + tk.MustExec("drop table if exists o") + tk.MustExec("create table c(c_id bigint)") + tk.MustExec("create table o(o_id bigint, c_id bigint not null)") + + tk.MustExec("create table t (a int, b int)") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (1, 1);") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "c" || tblInfo.Name.L == "o" || tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test skewed group distinct aggregate rewrite for MPP mode +func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b bigint not null, c bigint, d date, e varchar(20))") + // since allow-mpp is adjusted to false, there will be no physical plan if TiFlash cop is banned. + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test 3 stage aggregation for single count distinct +func TestMPPSingleDistinct3Stage(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b bigint not null, c bigint, d date, e varchar(20) collate utf8mb4_general_ci)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// todo: some post optimization after resolveIndices will inject another projection below agg, which change the column name used in higher operator, +// +// since it doesn't change the schema out (index ref is still the right), so by now it's fine. SEE case: EXPLAIN select count(distinct a), count(distinct b), sum(c) from t. +func TestMPPMultiDistinct3Stage(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, d int);") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_opt_enable_three_stage_multi_distinct_agg=1") + defer tk.MustExec("set @@session.tidb_opt_enable_three_stage_multi_distinct_agg=0") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\";") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + tk.MustExec("set @@session.tidb_allow_mpp=ON;") + // todo: current mock regionCache won't scale the regions among tiFlash nodes. The under layer still collect data from only one of the nodes. + tk.MustExec("split table t BETWEEN (0) AND (5000) REGIONS 5;") + tk.MustExec("insert into t values(1000, 1000, 1000, 1)") + tk.MustExec("insert into t values(1000, 1000, 1000, 1)") + tk.MustExec("insert into t values(2000, 2000, 2000, 1)") + tk.MustExec("insert into t values(2000, 2000, 2000, 1)") + tk.MustExec("insert into t values(3000, 3000, 3000, 1)") + tk.MustExec("insert into t values(3000, 3000, 3000, 1)") + tk.MustExec("insert into t values(4000, 4000, 4000, 1)") + tk.MustExec("insert into t values(4000, 4000, 4000, 1)") + tk.MustExec("insert into t values(5000, 5000, 5000, 1)") + tk.MustExec("insert into t values(5000, 5000, 5000, 1)") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test null-aware semi join push down for MPP mode +func TestMPPNullAwareSemiJoinPushDown(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists s") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("create table s(a int, b int, c int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("alter table s set tiflash replica 1") + + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tb = external.GetTableByName(t, tk, "test", "s") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestMPPSharedCTEScan(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists s") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("create table s(a int, b int, c int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("alter table s set tiflash replica 1") + + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tb = external.GetTableByName(t, tk, "test", "s") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + + tk.MustExec("set @@tidb_enforce_mpp='on'") + tk.MustExec("set @@tidb_opt_enable_mpp_shared_cte_execution='on'") + + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestRollupMPP(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists s") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("create table s(a int, b int, c int)") + tk.MustExec("CREATE TABLE `sales` (`year` int(11) DEFAULT NULL, `country` varchar(20) DEFAULT NULL, `product` varchar(32) DEFAULT NULL, `profit` int(11) DEFAULT NULL)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("alter table s set tiflash replica 1") + tk.MustExec("alter table sales set tiflash replica 1") + + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tb = external.GetTableByName(t, tk, "test", "s") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tb = external.GetTableByName(t, tk, "test", "sales") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + // error test + err = tk.ExecToErr("explain format = 'brief' SELECT country, product, SUM(profit) AS profit FROM sales GROUP BY country, country, product with rollup order by grouping(year);") + require.Equal(t, err.Error(), "[planner:3602]Argument #0 of GROUPING function is not in GROUP BY") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + + tk.MustExec("set @@tidb_enforce_mpp='on'") + tk.Session().GetSessionVars().TiFlashFineGrainedShuffleStreamCount = -1 + + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} diff --git a/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_in.json b/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_in.json new file mode 100644 index 0000000000000..384d9a8c01b9b --- /dev/null +++ b/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_in.json @@ -0,0 +1,225 @@ +[ + { + "name": "TestEnforceMPP", + "cases": [ + "select @@tidb_allow_mpp", + "select @@tidb_enforce_mpp", + "select @@tidb_opt_tiflash_concurrency_factor", + "set @@tidb_allow_mpp=0", + "explain format='verbose' select count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "set @@tidb_allow_mpp=1;", + "set @@tidb_enforce_mpp=0;", + "explain format='verbose' select count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "set @@tidb_opt_tiflash_concurrency_factor = 1000000", + "explain format='verbose' select count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "set @@tidb_enforce_mpp=1;", + "explain format='verbose' select count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "explain select /*+ READ_FROM_STORAGE(TIFLASH[s]) */ a from s where a = 10 and b is null; -- index path huristic rule will prune tiflash path" + ] + }, + { + "name": "TestEnforceMPPWarning1", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "explain format = 'brief' select count(*) from t where a=1 -- 1. no replica", + "cmd: create-replica", + "explain select count(*) from t where a=1 -- 2. replica not ready", + "cmd: enable-replica", + "set @@session.tidb_isolation_read_engines = 'tikv';", + "explain select count(*) from t where a=1 -- 3. isolation_engine not match", + "set @@session.tidb_isolation_read_engines = 'tikv, tiflash';", + "explain format = 'brief' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", + "explain format = 'brief' SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", + "EXPLAIN format = 'brief' SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", + "EXPLAIN format = 'brief' SELECT count(b) from t where a=1; -- 7. agg func has virtual column", + "EXPLAIN format = 'brief' SELECT count(*) from t group by b; -- 8. group by virtual column", + "EXPLAIN format = 'brief' SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", + "EXPLAIN format = 'brief' SELECT count(a) from t where c=1; -- 11. type not supported", + "EXPLAIN format = 'brief' SELECT count(a) from t where d=1; -- 11.1. type not supported" + ] + }, + { + "name": "TestEnforceMPPWarning2", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "set @@tidb_partition_prune_mode=static;", + "EXPLAIN SELECT count(*) from t where a=1; -- 1. static partition prune", + "set @@tidb_partition_prune_mode=dynamic;" + + ] + }, + { + "name": "TestEnforceMPPWarning3", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_hash_exchange_with_new_collation=0;", + "cmd: enable-new-collation", + "EXPLAIN SELECT count(*) from t group by b; -- 1. new collation FIXME", + "EXPLAIN SELECT * from t t1 join t t2 on t1.b=t2.b; -- 2. new collation FIXME" + ] + }, + { + "name": "TestEnforceMPPWarning4", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1; -- test joins", + "EXPLAIN SELECT /*+ MERGE_JOIN(t,s) */ * from t join s using(a); -- 1. hint use MERGE_JOIN", + "EXPLAIN SELECT /*+ INL_JOIN(t,s) */ * from t, s where t.a=s.a; -- 2. hint use INL_JOIN", + "EXPLAIN SELECT /*+ INL_HASH_JOIN(t,s) */ * from t join s using(a); -- 3. hint use INL_HASH_JOIN", + "EXPLAIN SELECT /*+ HASH_JOIN(t,s) */ * from t join s using(a); -- 4. hint use INL_JOIN", + "set @@tidb_opt_broadcast_cartesian_join = 0", + "EXPLAIN SELECT * from t join s; -- 5. cartesian join, cartesian banned.", + "set @@tidb_broadcast_join_threshold_size = 0; set @@tidb_opt_broadcast_cartesian_join = 1", + "EXPLAIN SELECT * from t join s; -- 6. cartesian join, broadcast banned.", + "set @@tidb_broadcast_join_threshold_size = 104857600; set @@tidb_opt_broadcast_cartesian_join = 1", + "EXPLAIN SELECT * from t join s; -- can use mpp", + "set @@tidb_broadcast_join_threshold_size = 0; set @@tidb_opt_broadcast_cartesian_join = 2", + "EXPLAIN SELECT * from t join s; -- can use mpp", + "set @@tidb_broadcast_join_threshold_size = 104857600; set @@tidb_opt_broadcast_cartesian_join = 1;", + "explain select a from t where t.a>1 or t.a in (select a from t); -- 7. left outer semi join", + "explain select a from t where t.a>1 or t.a not in (select a from t); -- now it's supported -- 8. anti left outer semi join", + "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions" + ] + }, + { + "name": "TestMPP2PhaseAggPushDown", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_agg_push_down=1;", + "EXPLAIN select count(*) from c, o where c.c_id=o.c_id; -- 1. test agg push down, scalar aggregate", + "EXPLAIN select o.o_id, count(*) from c, o where c.c_id=o.c_id group by o.o_id; -- 2. test agg push down, group by non-join column", + "EXPLAIN select o.c_id, count(*) from c, o where c.c_id=o.c_id group by o.c_id; -- 3. test agg push down, group by join column", + "EXPLAIN format='brief' select a, count(*) from (select a, b from t union all select a, b from t) t group by a order by a limit 10" + ] + }, + { + "name": "TestMPPSkewedGroupDistinctRewrite", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_skew_distinct_agg=1;", + "EXPLAIN select count(distinct c) from t group by d;", + "EXPLAIN select count(distinct c), count(a) from t group by d;", + "EXPLAIN select count(distinct c) from t group by b+d;", + "EXPLAIN select count(distinct c) from t group by b+d, a+b;", + "EXPLAIN select count(distinct c), count(*) from t group by date_format(d,'%Y');", + "EXPLAIN select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y');", + "EXPLAIN select d, count(distinct c), sum(a), max(b), count(*) from t group by d;", + "EXPLAIN select date_format(d,'%Y') as df, count(distinct c) from t group by date_format(d,'%Y');", + "EXPLAIN select date_format(d,'%Y') as df, a, count(b), count(distinct c) from t group by date_format(d,'%Y'), a;", + "EXPLAIN select date_format(d,'%Y') as df, a, count(b), avg(distinct c) from t group by date_format(d,'%Y'), a;", + "EXPLAIN select d,e, min(b), count(distinct c), bit_or(a) from t group by e,d; -- bit agg func can't be pushed to TiFlash", + "EXPLAIN select a, count(b), avg(distinct c), count(distinct c) from t group by a; -- multi distinct funcs, bail out", + "EXPLAIN select count(b), count(distinct c) from t; -- single distinct func but no group key, bail out" + ] + }, + { + "name": "TestMPPSingleDistinct3Stage", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "EXPLAIN select count(distinct b) from t;", + "EXPLAIN select count(distinct c) from t;", + "EXPLAIN select count(distinct e) from t;", + "EXPLAIN select count(distinct a,b,c,e) from t;", + "EXPLAIN select count(distinct c), count(a), count(*) from t;", + "EXPLAIN select sum(b), count(a), count(*), count(distinct c) from t;", + "EXPLAIN select sum(b+a), count(*), count(distinct c), count(a) from t having count(distinct c) > 2;", + "EXPLAIN select sum(b+a), count(*), count(a) from t having count(distinct c) > 2;", + "EXPLAIN select sum(b+a), max(b), count(distinct c), count(*) from t having count(a) > 2;", + "EXPLAIN select sum(b), count(distinct a, b, e), count(a+b) from t;", + "EXPLAIN select count(distinct b), json_objectagg(d,c) from t;", + "EXPLAIN select count(distinct c+a), count(a) from t;", + "EXPLAIN select sum(b), count(distinct c+a, b, e), count(a+b) from t;" + ] + }, + { + "name": "TestMPPMultiDistinct3Stage", + "cases": [ + "EXPLAIN select count(distinct a) from t", + "select count(distinct a) from t", + "EXPLAIN select count(distinct a), count(distinct b) from t", + "select count(distinct a), count(distinct b) from t", + "EXPLAIN select count(distinct a), count(distinct b), count(c) from t", + "select count(distinct a), count(distinct b), count(c) from t", + "EXPLAIN select count(distinct a), count(distinct b), count(c+1) from t", + "select count(distinct a), count(distinct b), count(c+1) from t", + "EXPLAIN select count(distinct a), count(distinct b), sum(c) from t", + "select count(distinct a), count(distinct b), sum(c) from t", + "EXPLAIN select count(distinct a, b), count(distinct b), count(c), sum(d) from t", + "select count(distinct a, b), count(distinct b), count(c), sum(d) from t", + "EXPLAIN select count(distinct a+b), sum(c) from t", + "select count(distinct a+b), sum(c) from t", + "EXPLAIN select count(distinct a+b), count(distinct b+c), count(c) from t", + "select count(distinct a+b), count(distinct b+c), count(c) from t", + "explain select count(distinct a,c), count(distinct b,c), count(c) from t", + "select count(distinct a), count(distinct b), count(*) from t", + "explain select count(distinct a), count(distinct b), count(*) from t", + "select count(distinct a), count(distinct b), avg(c+d) from t", + "explain select count(distinct a), count(distinct b), avg(c+d) from t" + ] + }, + { + "name": "TestMPPNullAwareSemiJoinPushDown", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_enable_null_aware_anti_join=on;", + "EXPLAIN select * from t where t.a not in (select s.a from s); -- 1. anti semi join, one join key", + "EXPLAIN select * from t where t.a not in (select s.a from s where t.c > s.c); -- 2. anti semi join, one join key + other condition", + "EXPLAIN select * from t where (t.a, t.b) not in (select s.a, s.b from s); -- 3. anti semi join, two join key", + "EXPLAIN select * from t where (t.a, t.b) not in (select s.a, s.b from s where t.c < s.c); -- 4. anti semi join, two join key + other condition", + "EXPLAIN select *, t.a not in (select s.a from s) from t; -- 5. left anti semi join, one join key", + "EXPLAIN select *, t.a not in (select s.a from s where t.c > s.c) from t; -- 6. left anti semi join, one join key + other condition", + "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s) from t; -- 7. left anti semi join, two join key", + "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s where t.c < s.c) from t; -- 8. left anti semi join, two join key + other condition" + ] + }, + { + "name": "TestMPPSharedCTEScan", + "cases": [ + // The most simple case. + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "explain format = 'brief' with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + // Can work when there's global limit/topn + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + // The c2 references c1, c1 can mpp, and then c2 can mpp, so the main query can mpp. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + // The same SQL, c1 forces to read tikv. So c2 cannot MPP, then the whole SQL. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + // The two WITH satement can all be MPP. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The outer one will fail to use MPP. But the inner WITH statement can. But we haven't implemented the least common ancestor to detect the best position of the Sequence. So the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The inner one will fail. So the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select /*+ read_from_storage(tikv[t]) */ * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // A little change that the inner WITH statement references the outer's c1. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The outer one will fail to use MPP. Since the inner one is references the outer one, the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a" + ] + }, + { + "name": "TestRollupMPP", + "cases": [ + "explain format = 'brief' select count(1) from t group by a, b with rollup; -- 1. simple agg", + "explain format = 'brief' select sum(c), count(1) from t group by a, b with rollup; -- 2. non-grouping set col c", + "explain format = 'brief' select count(a) from t group by a, b with rollup; -- 3. should keep the original col a", + "explain format = 'brief' select grouping(a) from t group by a, b with rollup; -- 4. contain grouping function ref to grouping set column a", + "explain format = 'brief' select grouping(a,b) from t group by a, b with rollup; -- 5. grouping function contains grouping set column a,c", + "explain format = 'brief' select a, grouping(b,a) from t group by a,b with rollup; -- 6. resolve normal column a to grouping set column a'", + "explain format = 'brief' select a+1, grouping(b) from t group by a+1, b with rollup; -- 7. resolve field list a+1 to grouping set column a+1", + "explain format = 'brief' SELECT SUM(profit) AS profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP order by year+2; -- 8. order by item year+2 resolve to gby grouping expression", + "explain format = 'brief' SELECT year+2, SUM(profit) AS profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP order by year+2; -- 9. order by item year+2 resolve to select field", + "explain format = 'brief' SELECT year+2 as y, SUM(profit) as profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP having y > 2002 order by year+2, profit; -- 10. having (year+2) shouldn't be pushed down", + "explain format = 'brief' SELECT year+2 as y, SUM(profit) AS profit, grouping(year+2) FROM sales GROUP BY year+2, year+profit WITH ROLLUP having y > 2002 order by year+2, profit; -- 11. grouping function validation", + "explain format = 'brief' SELECT year, country, product, SUM(profit) AS profit FROM sales GROUP BY year, country, product with rollup order by grouping(year); -- 12. grouping function in order by clause", + "explain format = 'brief' SELECT country, product, SUM(profit) AS profit FROM sales GROUP BY country, country, product with rollup order by grouping(country); -- 13. 12 under gpos case", + "explain format = 'brief' SELECT year, country, product, SUM(profit) AS profit FROM sales GROUP BY year, country, product with rollup having grouping(year) > 0 order by grouping(year); -- 14. grouping function in having clause", + "explain format = 'brief' SELECT country, product, SUM(profit) AS profit FROM sales GROUP BY country, country, product with rollup having grouping(country) > 0 order by grouping(country); -- 15. 14 under gpos case", + "explain format = 'brief' SELECT year, country, product, grouping(year, country, product) from sales group by year, country, product with rollup having grouping(year, country, product) <> 0; -- 16. grouping function recreating fix" + ] + } +] diff --git a/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json b/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json new file mode 100644 index 0000000000000..2dc6c93121651 --- /dev/null +++ b/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json @@ -0,0 +1,2266 @@ +[ + { + "Name": "TestEnforceMPP", + "Cases": [ + { + "SQL": "select @@tidb_allow_mpp", + "Plan": [ + "1" + ], + "Warn": null + }, + { + "SQL": "select @@tidb_enforce_mpp", + "Plan": [ + "0" + ], + "Warn": null + }, + { + "SQL": "select @@tidb_opt_tiflash_concurrency_factor", + "Plan": [ + "24" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_allow_mpp=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='verbose' select count(*) from t where a=1", + "Plan": [ + "StreamAgg_24 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_25 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_23 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_17 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_18 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_16 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_20 1.00 63520.28 root funcs:count(Column#6)->Column#4", + "└─TableReader_21 1.00 63470.38 root data:StreamAgg_9", + " └─StreamAgg_9 1.00 952024.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─Selection_19 10.00 952000.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_18 10000.00 928000.00 batchCop[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_allow_mpp=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@tidb_enforce_mpp=0;", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='verbose' select count(*) from t where a=1", + "Plan": [ + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_27 1.00 63520.28 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 63470.38 root data:StreamAgg_11", + " └─StreamAgg_11 1.00 952024.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─Selection_26 10.00 952000.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_25 10000.00 928000.00 batchCop[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_tiflash_concurrency_factor = 1000000", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='verbose' select count(*) from t where a=1", + "Plan": [ + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_27 1.00 63520.28 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 63470.38 root data:StreamAgg_11", + " └─StreamAgg_11 1.00 952024.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─Selection_26 10.00 952000.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_25 10000.00 928000.00 batchCop[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_enforce_mpp=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='verbose' select count(*) from t where a=1", + "Plan": [ + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", + "Plan": [ + "StreamAgg_27 1.00 49.90 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 0.00 root data:StreamAgg_11", + " └─StreamAgg_11 1.00 952024.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─Selection_26 10.00 952000.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_25 10000.00 928000.00 batchCop[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select /*+ READ_FROM_STORAGE(TIFLASH[s]) */ a from s where a = 10 and b is null; -- index path huristic rule will prune tiflash path", + "Plan": [ + "TableReader_12 0.10 root MppVersion: 2, data:ExchangeSender_11", + "└─ExchangeSender_11 0.10 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_5 0.10 mpp[tiflash] test.s.a", + " └─Selection_10 0.10 mpp[tiflash] isnull(test.s.b)", + " └─TableFullScan_9 10.00 mpp[tiflash] table:s pushed down filter:eq(test.s.a, 10), keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestEnforceMPPWarning1", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format = 'brief' select count(*) from t where a=1 -- 1. no replica", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because there aren't tiflash replicas of table `t`." + ] + }, + { + "SQL": "cmd: create-replica", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain select count(*) from t where a=1 -- 2. replica not ready", + "Plan": [ + "StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader_18 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan_16 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because tiflash replicas of table `t` not ready." + ] + }, + { + "SQL": "cmd: enable-replica", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@session.tidb_isolation_read_engines = 'tikv';", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain select count(*) from t where a=1 -- 3. isolation_engine not match", + "Plan": [ + "StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader_18 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan_16 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because 'tidb_isolation_read_engines'(value: 'tikv') not match, need 'tiflash'." + ] + }, + { + "SQL": "set @@session.tidb_isolation_read_engines = 'tikv, tiflash';", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(order by test.t.a rows between current row and current row)", + " └─Sort 10000.00 mpp[tiflash] test.t.a", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root MppVersion: 2, data:ExchangeSender", + "│ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now.", + "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now." + ] + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT count(b) from t where a=1; -- 7. agg func has virtual column", + "Plan": [ + "StreamAgg 1.00 root funcs:count(test.t.b)->Column#6", + "└─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10.00 mpp[tiflash] table:t pushed down filter:eq(test.t.a, 1), keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to tiflash because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now", + "Aggregation can not be pushed to tikv because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now", + "Aggregation can not be pushed to tikv because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now", + "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now.", + "Aggregation can not be pushed to tikv because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now", + "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now.", + "Aggregation can not be pushed to tiflash because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now" + ] + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT count(*) from t group by b; -- 8. group by virtual column", + "Plan": [ + "HashAgg 8000.00 root group by:test.t.b, funcs:count(1)->Column#6", + "└─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to tiflash because groupByItems contain virtual columns, which is not supported now", + "Aggregation can not be pushed to tikv because groupByItems contain virtual columns, which is not supported now", + "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now.", + "Aggregation can not be pushed to tiflash because groupByItems contain virtual columns, which is not supported now" + ] + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", + "Plan": [ + "HashAgg 8000.00 root group by:Column#10, funcs:count(Column#9)->Column#6", + "└─Projection 10000.00 root test.t.a->Column#9, md5(cast(test.t.a, var_string(20)))->Column#10", + " └─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because groupByItems contain unsupported exprs", + "Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because groupByItems contain unsupported exprs" + ] + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where c=1; -- 11. type not supported", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#8", + " └─Selection 10.00 cop[tikv] eq(test.t.c, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Expression about 'test.t.c' can not be pushed to TiFlash because it contains unsupported calculation of type 'enum'.", + "Expression about 'test.t.c' can not be pushed to TiFlash because it contains unsupported calculation of type 'enum'.", + "Expression about 'test.t.c' can not be pushed to TiFlash because it contains unsupported calculation of type 'enum'." + ] + }, + { + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where d=1; -- 11.1. type not supported", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#8", + " └─Selection 10.00 cop[tikv] eq(test.t.d, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Expression about 'test.t.d' can not be pushed to TiFlash because it contains unsupported calculation of type 'bit'.", + "Expression about 'test.t.d' can not be pushed to TiFlash because it contains unsupported calculation of type 'bit'.", + "Expression about 'test.t.d' can not be pushed to TiFlash because it contains unsupported calculation of type 'bit'." + ] + } + ] + }, + { + "Name": "TestEnforceMPPWarning2", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@tidb_partition_prune_mode=static;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT count(*) from t where a=1; -- 1. static partition prune", + "Plan": [ + "StreamAgg_14 1.00 root funcs:count(1)->Column#4", + "└─TableReader_43 10.00 root MppVersion: 2, data:ExchangeSender_42", + " └─ExchangeSender_42 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_41 10.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_40 10000.00 mpp[tiflash] table:t, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_partition_prune_mode=dynamic;", + "Plan": null, + "Warn": null + } + ] + }, + { + "Name": "TestEnforceMPPWarning3", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_hash_exchange_with_new_collation=0;", + "Plan": null, + "Warn": null + }, + { + "SQL": "cmd: enable-new-collation", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT count(*) from t group by b; -- 1. new collation FIXME", + "Plan": [ + "HashAgg_25 8000.00 root group by:test.t.b, funcs:count(Column#7)->Column#4", + "└─TableReader_27 8000.00 root MppVersion: 2, data:ExchangeSender_26", + " └─ExchangeSender_26 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_11 8000.00 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#7", + " └─TableFullScan_22 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because when `new_collation_enabled` is true, HashJoin or HashAgg with string key is not supported now.", + "MPP mode may be blocked because when `new_collation_enabled` is true, HashJoin or HashAgg with string key is not supported now.", + "MPP mode may be blocked because when `new_collation_enabled` is true, HashJoin or HashAgg with string key is not supported now." + ] + }, + { + "SQL": "EXPLAIN SELECT * from t t1 join t t2 on t1.b=t2.b; -- 2. new collation FIXME", + "Plan": [ + "TableReader_36 12487.50 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_34 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver_15(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_13 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection_17(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan_16 10000.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestEnforceMPPWarning4", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1; -- test joins", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT /*+ MERGE_JOIN(t,s) */ * from t join s using(a); -- 1. hint use MERGE_JOIN", + "Plan": [ + "MergeJoin_9 12500.00 root inner join, left key:test.t.a, right key:test.s.a", + "├─TableReader_16(Build) 10000.00 root data:TableFullScan_15", + "│ └─TableFullScan_15 10000.00 cop[tikv] table:s keep order:true, stats:pseudo", + "└─TableReader_13(Probe) 10000.00 root data:TableFullScan_12", + " └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:true, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.", + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now." + ] + }, + { + "SQL": "EXPLAIN SELECT /*+ INL_JOIN(t,s) */ * from t, s where t.a=s.a; -- 2. hint use INL_JOIN", + "Plan": [ + "IndexJoin_16 12500.00 root inner join, inner:TableReader_13, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a)", + "├─TableReader_34(Build) 10000.00 root MppVersion: 2, data:ExchangeSender_33", + "│ └─ExchangeSender_33 10000.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_32 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + "└─TableReader_13(Probe) 10000.00 root data:TableRangeScan_12", + " └─TableRangeScan_12 10000.00 cop[tikv] table:s range: decided by [test.t.a], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.", + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now." + ] + }, + { + "SQL": "EXPLAIN SELECT /*+ INL_HASH_JOIN(t,s) */ * from t join s using(a); -- 3. hint use INL_HASH_JOIN", + "Plan": [ + "IndexHashJoin_17 12500.00 root inner join, inner:TableReader_12, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a)", + "├─TableReader_33(Build) 10000.00 root MppVersion: 2, data:ExchangeSender_32", + "│ └─ExchangeSender_32 10000.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_31 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + "└─TableReader_12(Probe) 10000.00 root data:TableRangeScan_11", + " └─TableRangeScan_11 10000.00 cop[tikv] table:s range: decided by [test.t.a], keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.", + "MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now." + ] + }, + { + "SQL": "EXPLAIN SELECT /*+ HASH_JOIN(t,s) */ * from t join s using(a); -- 4. hint use INL_JOIN", + "Plan": [ + "TableReader_47 12500.00 root MppVersion: 2, data:ExchangeSender_46", + "└─ExchangeSender_46 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 12500.00 mpp[tiflash] test.t.a", + " └─HashJoin_41 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.s.a)]", + " ├─ExchangeReceiver_44(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_43 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_42 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_45(Probe) 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_broadcast_cartesian_join = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT * from t join s; -- 5. cartesian join, cartesian banned.", + "Plan": [ + "HashJoin_8 100000000.00 root CARTESIAN inner join", + "├─TableReader_19(Build) 10000.00 root MppVersion: 2, data:ExchangeSender_18", + "│ └─ExchangeSender_18 10000.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_17 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + "└─TableReader_14(Probe) 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variable `tidb_opt_broadcast_cartesian_join`.", + "MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variable `tidb_opt_broadcast_cartesian_join`." + ] + }, + { + "SQL": "set @@tidb_broadcast_join_threshold_size = 0; set @@tidb_opt_broadcast_cartesian_join = 1", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT * from t join s; -- 6. cartesian join, broadcast banned.", + "Plan": [ + "HashJoin_8 100000000.00 root CARTESIAN inner join", + "├─TableReader_19(Build) 10000.00 root MppVersion: 2, data:ExchangeSender_18", + "│ └─ExchangeSender_18 10000.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_17 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + "└─TableReader_14(Probe) 10000.00 root MppVersion: 2, data:ExchangeSender_13", + " └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variables `tidb_broadcast_join_threshold_size` and `tidb_broadcast_join_threshold_count`.", + "MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variables `tidb_broadcast_join_threshold_size` and `tidb_broadcast_join_threshold_count`." + ] + }, + { + "SQL": "set @@tidb_broadcast_join_threshold_size = 104857600; set @@tidb_opt_broadcast_cartesian_join = 1", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT * from t join s; -- can use mpp", + "Plan": [ + "TableReader_29 100000000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 100000000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 100000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver_13(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_11 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_14(Probe) 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_broadcast_join_threshold_size = 0; set @@tidb_opt_broadcast_cartesian_join = 2", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN SELECT * from t join s; -- can use mpp", + "Plan": [ + "TableReader_29 100000000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 100000000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 100000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver_13(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_11 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_14(Probe) 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_broadcast_join_threshold_size = 104857600; set @@tidb_opt_broadcast_cartesian_join = 1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain select a from t where t.a>1 or t.a in (select a from t); -- 7. left outer semi join", + "Plan": [ + "TableReader_51 8000.00 root MppVersion: 2, data:ExchangeSender_50", + "└─ExchangeSender_50 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.a", + " └─Selection_49 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_46 10000.00 mpp[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_27(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_26 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_25 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_24(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select a from t where t.a>1 or t.a not in (select a from t); -- now it's supported -- 8. anti left outer semi join", + "Plan": [ + "TableReader_51 8000.00 root MppVersion: 2, data:ExchangeSender_50", + "└─ExchangeSender_50 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.a", + " └─Selection_49 8000.00 mpp[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_46 10000.00 mpp[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_27(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_26 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_25 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_24(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions", + "Plan": [ + "MergeJoin_10 8000.00 root anti semi join, left key:test.t.a, right key:test.s.a, left cond:[lt(test.t.a, 1)]", + "├─TableReader_26(Build) 10000.00 root data:TableFullScan_25", + "│ └─TableFullScan_25 10000.00 cop[tikv] table:s keep order:true, stats:pseudo", + "└─TableReader_23(Probe) 10000.00 root data:TableFullScan_22", + " └─TableFullScan_22 10000.00 cop[tikv] table:t keep order:true, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because there is a join that is not `left join` but has left conditions, which is not supported by mpp now, see github.com/pingcap/tidb/issues/26090 for more information.", + "MPP mode may be blocked because there is a join that is not `left join` but has left conditions, which is not supported by mpp now, see github.com/pingcap/tidb/issues/26090 for more information." + ] + } + ] + }, + { + "Name": "TestMPP2PhaseAggPushDown", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_agg_push_down=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select count(*) from c, o where c.c_id=o.c_id; -- 1. test agg push down, scalar aggregate", + "Plan": [ + "HashAgg_13 1.00 root funcs:count(Column#7)->Column#6", + "└─TableReader_35 9990.00 root MppVersion: 2, data:ExchangeSender_34", + " └─ExchangeSender_34 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_14 9990.00 mpp[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_33(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_32 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Projection_28 8000.00 mpp[tiflash] Column#7, test.o.c_id", + " │ └─HashAgg_29 8000.00 mpp[tiflash] group by:test.o.c_id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_31 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_30 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_20 8000.00 mpp[tiflash] group by:test.o.c_id, funcs:count(1)->Column#8", + " │ └─TableFullScan_27 10000.00 mpp[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_18(Probe) 9990.00 mpp[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_17 10000.00 mpp[tiflash] table:c pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select o.o_id, count(*) from c, o where c.c_id=o.c_id group by o.o_id; -- 2. test agg push down, group by non-join column", + "Plan": [ + "TableReader_80 8000.00 root MppVersion: 2, data:ExchangeSender_79", + "└─ExchangeSender_79 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_10 8000.00 mpp[tiflash] test.o.o_id, Column#6", + " └─Projection_78 8000.00 mpp[tiflash] Column#6, test.o.o_id", + " └─HashAgg_77 8000.00 mpp[tiflash] group by:Column#27, funcs:sum(Column#25)->Column#6, funcs:firstrow(Column#26)->test.o.o_id", + " └─Projection_81 9990.00 mpp[tiflash] cast(Column#7, decimal(20,0) BINARY)->Column#25, Column#8->Column#26, test.o.o_id->Column#27", + " └─ExchangeReceiver_73 9990.00 mpp[tiflash] ", + " └─ExchangeSender_72 9990.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.o.o_id, collate: binary]", + " └─HashJoin_71 9990.00 mpp[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_34(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_33 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Projection_29 8000.00 mpp[tiflash] Column#7, Column#8, test.o.o_id, test.o.c_id", + " │ └─HashAgg_30 8000.00 mpp[tiflash] group by:test.o.c_id, test.o.o_id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.o.o_id)->Column#8, funcs:firstrow(test.o.o_id)->test.o.o_id, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_32 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_31 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.o.o_id, collate: binary], [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_21 8000.00 mpp[tiflash] group by:test.o.c_id, test.o.o_id, funcs:count(1)->Column#9", + " │ └─TableFullScan_28 10000.00 mpp[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_19(Probe) 9990.00 mpp[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:c pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select o.c_id, count(*) from c, o where c.c_id=o.c_id group by o.c_id; -- 3. test agg push down, group by join column", + "Plan": [ + "TableReader_80 8000.00 root MppVersion: 2, data:ExchangeSender_79", + "└─ExchangeSender_79 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_10 8000.00 mpp[tiflash] test.o.c_id, Column#6", + " └─Projection_78 8000.00 mpp[tiflash] Column#6, test.o.c_id", + " └─HashAgg_77 8000.00 mpp[tiflash] group by:Column#23, funcs:sum(Column#21)->Column#6, funcs:firstrow(Column#22)->test.o.c_id", + " └─Projection_81 9990.00 mpp[tiflash] cast(Column#7, decimal(20,0) BINARY)->Column#21, Column#8->Column#22, test.o.c_id->Column#23", + " └─ExchangeReceiver_73 9990.00 mpp[tiflash] ", + " └─ExchangeSender_72 9990.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.o.c_id, collate: binary]", + " └─HashJoin_71 9990.00 mpp[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_34(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_33 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Projection_29 8000.00 mpp[tiflash] Column#7, Column#8, test.o.c_id", + " │ └─HashAgg_30 8000.00 mpp[tiflash] group by:test.o.c_id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.o.c_id)->Column#8, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_32 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender_31 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_21 8000.00 mpp[tiflash] group by:test.o.c_id, funcs:count(1)->Column#9", + " │ └─TableFullScan_28 10000.00 mpp[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_19(Probe) 9990.00 mpp[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:c pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN format='brief' select a, count(*) from (select a, b from t union all select a, b from t) t group by a order by a limit 10", + "Plan": [ + "Projection 10.00 root Column#7, Column#9", + "└─TopN 10.00 root Column#7, offset:0, count:10", + " └─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] Column#7, offset:0, count:10", + " └─Projection 16000.00 mpp[tiflash] Column#9, Column#7", + " └─HashAgg 16000.00 mpp[tiflash] group by:Column#38, funcs:sum(Column#36)->Column#9, funcs:firstrow(Column#37)->Column#7", + " └─Projection 16000.00 mpp[tiflash] cast(Column#10, decimal(20,0) BINARY)->Column#36, Column#11->Column#37, Column#7->Column#38", + " └─ExchangeReceiver 16000.00 mpp[tiflash] ", + " └─ExchangeSender 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#7, collate: binary]", + " └─Union 16000.00 mpp[tiflash] ", + " ├─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:sum(Column#30)->Column#10, funcs:firstrow(test.t.a)->Column#11, funcs:firstrow(test.t.a)->Column#7", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:count(1)->Column#30", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:sum(Column#33)->Column#10, funcs:firstrow(test.t.a)->Column#11, funcs:firstrow(test.t.a)->Column#7", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:count(1)->Column#33", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestMPPSkewedGroupDistinctRewrite", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_skew_distinct_agg=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c) from t group by d;", + "Plan": [ + "TableReader_56 8000.00 root MppVersion: 2, data:ExchangeSender_55", + "└─ExchangeSender_55 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_51 8000.00 mpp[tiflash] Column#7", + " └─HashAgg_52 8000.00 mpp[tiflash] group by:test.t.d, funcs:sum(Column#18)->Column#7", + " └─ExchangeReceiver_54 8000.00 mpp[tiflash] ", + " └─ExchangeSender_53 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary]", + " └─HashAgg_50 8000.00 mpp[tiflash] group by:test.t.d, funcs:count(test.t.c)->Column#18", + " └─Projection_37 8000.00 mpp[tiflash] test.t.c, test.t.d", + " └─HashAgg_38 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_40 8000.00 mpp[tiflash] ", + " └─ExchangeSender_39 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_36 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, ", + " └─TableFullScan_26 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c), count(a) from t group by d;", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] Column#7, cast(Column#10, bigint(21) BINARY)->Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#10", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:test.t.d, funcs:sum(Column#27)->Column#7, funcs:sum(Column#28)->Column#10", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#31, funcs:count(Column#29)->Column#27, funcs:sum(Column#30)->Column#28", + " └─Projection_58 8000.00 mpp[tiflash] test.t.c->Column#29, cast(Column#9, decimal(20,0) BINARY)->Column#30, test.t.d->Column#31", + " └─Projection_38 8000.00 mpp[tiflash] test.t.c, Column#9, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, funcs:firstrow(test.t.c)->test.t.c, funcs:sum(Column#21)->Column#9, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, funcs:count(test.t.a)->Column#21", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c) from t group by b+d;", + "Plan": [ + "TableReader_56 8000.00 root MppVersion: 2, data:ExchangeSender_55", + "└─ExchangeSender_55 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_51 8000.00 mpp[tiflash] Column#7", + " └─HashAgg_52 8000.00 mpp[tiflash] group by:Column#28, funcs:sum(Column#29)->Column#7", + " └─ExchangeReceiver_54 8000.00 mpp[tiflash] ", + " └─ExchangeSender_53 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#28, collate: binary]", + " └─HashAgg_50 8000.00 mpp[tiflash] group by:Column#35, funcs:count(Column#34)->Column#29", + " └─Projection_58 8000.00 mpp[tiflash] test.t.c->Column#34, plus(test.t.b, cast(test.t.d, bigint(10) BINARY))->Column#35", + " └─Projection_37 8000.00 mpp[tiflash] test.t.c, test.t.b, test.t.d", + " └─HashAgg_38 8000.00 mpp[tiflash] group by:Column#20, test.t.c, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#22)->test.t.b, funcs:firstrow(Column#23)->test.t.d", + " └─ExchangeReceiver_40 8000.00 mpp[tiflash] ", + " └─ExchangeSender_39 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_36 8000.00 mpp[tiflash] group by:Column#32, Column#33, funcs:firstrow(Column#30)->Column#22, funcs:firstrow(Column#31)->Column#23", + " └─Projection_57 10000.00 mpp[tiflash] test.t.b->Column#30, test.t.d->Column#31, plus(test.t.b, cast(test.t.d, bigint(10) BINARY))->Column#32, test.t.c->Column#33", + " └─TableFullScan_26 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c) from t group by b+d, a+b;", + "Plan": [ + "TableReader_56 8000.00 root MppVersion: 2, data:ExchangeSender_55", + "└─ExchangeSender_55 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_51 8000.00 mpp[tiflash] Column#7", + " └─HashAgg_52 8000.00 mpp[tiflash] group by:Column#42, Column#43, funcs:sum(Column#44)->Column#7", + " └─ExchangeReceiver_54 8000.00 mpp[tiflash] ", + " └─ExchangeSender_53 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#42, collate: binary], [name: Column#43, collate: binary]", + " └─HashAgg_50 8000.00 mpp[tiflash] group by:Column#53, Column#54, funcs:count(Column#52)->Column#44", + " └─Projection_58 8000.00 mpp[tiflash] test.t.c->Column#52, plus(test.t.b, cast(test.t.d, bigint(10) BINARY))->Column#53, plus(test.t.a, test.t.b)->Column#54", + " └─Projection_37 8000.00 mpp[tiflash] test.t.c, test.t.b, test.t.d, test.t.a, test.t.b", + " └─HashAgg_38 8000.00 mpp[tiflash] group by:Column#29, Column#30, test.t.c, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#32)->test.t.b, funcs:firstrow(Column#33)->test.t.d, funcs:firstrow(Column#34)->test.t.a, funcs:firstrow(Column#35)->test.t.b", + " └─ExchangeReceiver_40 8000.00 mpp[tiflash] ", + " └─ExchangeSender_39 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_36 8000.00 mpp[tiflash] group by:Column#49, Column#50, Column#51, funcs:firstrow(Column#45)->Column#32, funcs:firstrow(Column#46)->Column#33, funcs:firstrow(Column#47)->Column#34, funcs:firstrow(Column#48)->Column#35", + " └─Projection_57 10000.00 mpp[tiflash] test.t.b->Column#45, test.t.d->Column#46, test.t.a->Column#47, test.t.b->Column#48, plus(test.t.b, cast(test.t.d, bigint(10) BINARY))->Column#49, plus(test.t.a, test.t.b)->Column#50, test.t.c->Column#51", + " └─TableFullScan_26 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c), count(*) from t group by date_format(d,'%Y');", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] Column#7, cast(Column#10, bigint(21) BINARY)->Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#10", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#33, funcs:sum(Column#34)->Column#7, funcs:sum(Column#35)->Column#10", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#33, collate: utf8mb4_bin]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#41, funcs:count(Column#39)->Column#34, funcs:sum(Column#40)->Column#35", + " └─Projection_59 8000.00 mpp[tiflash] test.t.c->Column#39, cast(Column#9, decimal(20,0) BINARY)->Column#40, date_format(test.t.d, %Y)->Column#41", + " └─Projection_38 8000.00 mpp[tiflash] test.t.c, Column#9, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#23, test.t.c, funcs:firstrow(test.t.c)->test.t.c, funcs:sum(Column#25)->Column#9, funcs:firstrow(Column#26)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#37, Column#38, funcs:count(1)->Column#25, funcs:firstrow(Column#36)->Column#26", + " └─Projection_58 10000.00 mpp[tiflash] test.t.d->Column#36, date_format(test.t.d, %Y)->Column#37, test.t.c->Column#38", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y');", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#10, Column#7, cast(Column#14, bigint(21) BINARY)->Column#8, Column#9", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#14, Column#9, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#45, funcs:sum(Column#46)->Column#7, funcs:sum(Column#47)->Column#14, funcs:sum(Column#48)->Column#9, funcs:firstrow(Column#49)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#45, collate: utf8mb4_bin]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#59, funcs:sum(Column#55)->Column#46, funcs:sum(Column#56)->Column#47, funcs:count(Column#57)->Column#48, funcs:firstrow(Column#58)->Column#49", + " └─Projection_59 8000.00 mpp[tiflash] Column#12->Column#55, cast(Column#13, decimal(20,0) BINARY)->Column#56, test.t.c->Column#57, test.t.d->Column#58, date_format(test.t.d, %Y)->Column#59", + " └─Projection_38 8000.00 mpp[tiflash] Column#12, Column#13, test.t.c, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#30, test.t.c, funcs:sum(Column#31)->Column#12, funcs:sum(Column#32)->Column#13, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#34)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#53, Column#54, funcs:sum(Column#50)->Column#31, funcs:count(Column#51)->Column#32, funcs:firstrow(Column#52)->Column#34", + " └─Projection_58 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#50, test.t.b->Column#51, test.t.d->Column#52, date_format(test.t.d, %Y)->Column#53, test.t.c->Column#54", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select d, count(distinct c), sum(a), max(b), count(*) from t group by d;", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.d, Column#7, Column#8, Column#9, cast(Column#14, bigint(21) BINARY)->Column#10", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#14, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:test.t.d, funcs:sum(Column#45)->Column#7, funcs:sum(Column#46)->Column#8, funcs:max(Column#47)->Column#9, funcs:sum(Column#48)->Column#14, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#58, funcs:count(Column#54)->Column#45, funcs:sum(Column#55)->Column#46, funcs:max(Column#56)->Column#47, funcs:sum(Column#57)->Column#48", + " └─Projection_59 8000.00 mpp[tiflash] test.t.c->Column#54, Column#11->Column#55, Column#12->Column#56, cast(Column#13, decimal(20,0) BINARY)->Column#57, test.t.d->Column#58", + " └─Projection_38 8000.00 mpp[tiflash] test.t.c, Column#11, Column#12, Column#13, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, funcs:firstrow(test.t.c)->test.t.c, funcs:sum(Column#31)->Column#11, funcs:max(Column#32)->Column#12, funcs:sum(Column#33)->Column#13, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.d, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#52, Column#53, funcs:sum(Column#50)->Column#31, funcs:max(Column#51)->Column#32, funcs:count(1)->Column#33", + " └─Projection_58 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#50, test.t.b->Column#51, test.t.d->Column#52, test.t.c->Column#53", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, count(distinct c) from t group by date_format(d,'%Y');", + "Plan": [ + "TableReader_56 8000.00 root MppVersion: 2, data:ExchangeSender_55", + "└─ExchangeSender_55 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#8, Column#7", + " └─Projection_51 8000.00 mpp[tiflash] Column#7, test.t.d", + " └─HashAgg_52 8000.00 mpp[tiflash] group by:Column#28, funcs:sum(Column#29)->Column#7, funcs:firstrow(Column#30)->test.t.d", + " └─ExchangeReceiver_54 8000.00 mpp[tiflash] ", + " └─ExchangeSender_53 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#28, collate: utf8mb4_bin]", + " └─HashAgg_50 8000.00 mpp[tiflash] group by:Column#36, funcs:count(Column#34)->Column#29, funcs:firstrow(Column#35)->Column#30", + " └─Projection_58 8000.00 mpp[tiflash] test.t.c->Column#34, test.t.d->Column#35, date_format(test.t.d, %Y)->Column#36", + " └─Projection_37 8000.00 mpp[tiflash] test.t.c, test.t.d", + " └─HashAgg_38 8000.00 mpp[tiflash] group by:Column#19, test.t.c, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#21)->test.t.d", + " └─ExchangeReceiver_40 8000.00 mpp[tiflash] ", + " └─ExchangeSender_39 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_36 8000.00 mpp[tiflash] group by:Column#32, Column#33, funcs:firstrow(Column#31)->Column#21", + " └─Projection_57 10000.00 mpp[tiflash] test.t.d->Column#31, date_format(test.t.d, %Y)->Column#32, test.t.c->Column#33", + " └─TableFullScan_26 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, a, count(b), count(distinct c) from t group by date_format(d,'%Y'), a;", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#9, test.t.a, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#12, Column#8, test.t.a, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#43, test.t.a, funcs:sum(Column#44)->Column#12, funcs:sum(Column#45)->Column#8, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#47)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#56, Column#57, funcs:sum(Column#53)->Column#44, funcs:count(Column#54)->Column#45, funcs:firstrow(Column#55)->Column#47", + " └─Projection_59 8000.00 mpp[tiflash] cast(Column#11, decimal(20,0) BINARY)->Column#53, test.t.c->Column#54, test.t.d->Column#55, date_format(test.t.d, %Y)->Column#56, test.t.a->Column#57", + " └─Projection_38 8000.00 mpp[tiflash] Column#11, test.t.c, test.t.a, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#28, test.t.a, test.t.c, funcs:sum(Column#29)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#32)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#50, Column#51, Column#52, funcs:count(Column#48)->Column#29, funcs:firstrow(Column#49)->Column#32", + " └─Projection_58 10000.00 mpp[tiflash] test.t.b->Column#48, test.t.d->Column#49, date_format(test.t.d, %Y)->Column#50, test.t.a->Column#51, test.t.c->Column#52", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, a, count(b), avg(distinct c) from t group by date_format(d,'%Y'), a;", + "Plan": [ + "TableReader_57 8000.00 root MppVersion: 2, data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#9, test.t.a, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#12, div(Column#8, cast(case(eq(Column#46, 0), 1, Column#46), decimal(20,0) BINARY))->Column#8, test.t.a, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#47, test.t.a, funcs:sum(Column#48)->Column#12, funcs:sum(Column#49)->Column#46, funcs:sum(Column#50)->Column#8, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#52)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#62, Column#63, funcs:sum(Column#58)->Column#48, funcs:count(Column#59)->Column#49, funcs:sum(Column#60)->Column#50, funcs:firstrow(Column#61)->Column#52", + " └─Projection_59 8000.00 mpp[tiflash] cast(Column#11, decimal(20,0) BINARY)->Column#58, test.t.c->Column#59, cast(test.t.c, decimal(20,0) BINARY)->Column#60, test.t.d->Column#61, date_format(test.t.d, %Y)->Column#62, test.t.a->Column#63", + " └─Projection_38 8000.00 mpp[tiflash] Column#11, test.t.c, test.t.a, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#28, test.t.a, test.t.c, funcs:sum(Column#29)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#32)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#55, Column#56, Column#57, funcs:count(Column#53)->Column#29, funcs:firstrow(Column#54)->Column#32", + " └─Projection_58 10000.00 mpp[tiflash] test.t.b->Column#53, test.t.d->Column#54, date_format(test.t.d, %Y)->Column#55, test.t.a->Column#56, test.t.c->Column#57", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select d,e, min(b), count(distinct c), bit_or(a) from t group by e,d; -- bit agg func can't be pushed to TiFlash", + "Plan": [ + "Projection_4 8000.00 root test.t.d, test.t.e, Column#7, Column#8, Column#9", + "└─HashAgg_6 8000.00 root group by:test.t.d, test.t.e, funcs:min(test.t.b)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:bit_or(test.t.a)->Column#9, funcs:firstrow(test.t.d)->test.t.d, funcs:firstrow(test.t.e)->test.t.e", + " └─TableReader_10 10000.00 root data:TableFullScan_9", + " └─TableFullScan_9 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to tiflash because AggFunc `bit_or` is not supported now", + "Aggregation can not be pushed to tiflash because AggFunc `bit_or` is not supported now" + ] + }, + { + "SQL": "EXPLAIN select a, count(b), avg(distinct c), count(distinct c) from t group by a; -- multi distinct funcs, bail out", + "Plan": [ + "Projection_4 8000.00 root test.t.a, Column#7, Column#8, Column#9", + "└─HashAgg_6 8000.00 root group by:Column#14, funcs:count(Column#10)->Column#7, funcs:avg(distinct Column#11)->Column#8, funcs:count(distinct Column#12)->Column#9, funcs:firstrow(Column#13)->test.t.a", + " └─Projection_11 10000.00 root test.t.b->Column#10, cast(test.t.c, decimal(20,0) BINARY)->Column#11, test.t.c->Column#12, test.t.a->Column#13, test.t.a->Column#14", + " └─TableReader_10 10000.00 root data:TableFullScan_9", + " └─TableFullScan_9 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct", + "Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct" + ] + }, + { + "SQL": "EXPLAIN select count(b), count(distinct c) from t; -- single distinct func but no group key, bail out", + "Plan": [ + "TableReader_34 1.00 root MppVersion: 2, data:ExchangeSender_33", + "└─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_27 1.00 mpp[tiflash] Column#7, Column#8", + " └─HashAgg_28 1.00 mpp[tiflash] funcs:sum(Column#13)->Column#7, funcs:sum(Column#14)->Column#8", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_28 1.00 mpp[tiflash] funcs:sum(Column#12)->Column#13, funcs:count(distinct test.t.c)->Column#14", + " └─ExchangeReceiver_30 1.00 mpp[tiflash] ", + " └─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_26 1.00 mpp[tiflash] group by:test.t.c, funcs:count(test.t.b)->Column#12", + " └─TableFullScan_13 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestMPPSingleDistinct3Stage", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct b) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#7", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:count(distinct test.t.b)->Column#9", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:test.t.b, ", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#7", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:count(distinct test.t.c)->Column#9", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:test.t.c, ", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct e) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#7", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:count(distinct test.t.e)->Column#9", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.e, collate: utf8mb4_general_ci]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:test.t.e, ", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a,b,c,e) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#7", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:count(distinct test.t.a, test.t.b, test.t.c, test.t.e)->Column#9", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: test.t.c, collate: binary], [name: test.t.e, collate: utf8mb4_general_ci]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:test.t.a, test.t.b, test.t.c, test.t.e, ", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct c), count(a), count(*) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7, Column#8, Column#9", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#17)->Column#7, funcs:sum(Column#18)->Column#8, funcs:sum(Column#19)->Column#9", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:count(distinct test.t.c)->Column#17, funcs:sum(Column#15)->Column#18, funcs:sum(Column#16)->Column#19", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:test.t.c, funcs:count(test.t.a)->Column#15, funcs:count(1)->Column#16", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b), count(a), count(*), count(distinct c) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#10", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#21)->Column#7, funcs:sum(Column#22)->Column#8, funcs:sum(Column#23)->Column#9, funcs:sum(Column#24)->Column#10", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#21, funcs:sum(Column#19)->Column#22, funcs:sum(Column#20)->Column#23, funcs:count(distinct test.t.c)->Column#24", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:Column#27, funcs:sum(Column#25)->Column#18, funcs:count(Column#26)->Column#19, funcs:count(1)->Column#20", + " └─Projection_37 10000.00 mpp[tiflash] cast(test.t.b, decimal(20,0) BINARY)->Column#25, test.t.a->Column#26, test.t.c->Column#27", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b+a), count(*), count(distinct c), count(a) from t having count(distinct c) > 2;", + "Plan": [ + "TableReader_43 0.80 root MppVersion: 2, data:ExchangeSender_42", + "└─ExchangeSender_42 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_41 0.80 mpp[tiflash] gt(Column#9, 2)", + " └─Projection_33 1.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#10", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#25)->Column#7, funcs:sum(Column#26)->Column#8, funcs:sum(Column#27)->Column#9, funcs:sum(Column#28)->Column#10", + " └─ExchangeReceiver_38 1.00 mpp[tiflash] ", + " └─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#22)->Column#25, funcs:sum(Column#23)->Column#26, funcs:count(distinct test.t.c)->Column#27, funcs:sum(Column#24)->Column#28", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_32 1.00 mpp[tiflash] group by:Column#31, funcs:sum(Column#29)->Column#22, funcs:count(1)->Column#23, funcs:count(Column#30)->Column#24", + " └─Projection_44 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#29, test.t.a->Column#30, test.t.c->Column#31", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b+a), count(*), count(a) from t having count(distinct c) > 2;", + "Plan": [ + "TableReader_43 0.80 root MppVersion: 2, data:ExchangeSender_42", + "└─ExchangeSender_42 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 0.80 mpp[tiflash] Column#7->Column#11, Column#8->Column#12, Column#9->Column#13", + " └─Selection_41 0.80 mpp[tiflash] gt(Column#10, 2)", + " └─Projection_33 1.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#10", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#24)->Column#7, funcs:sum(Column#25)->Column#8, funcs:sum(Column#26)->Column#9, funcs:sum(Column#27)->Column#10", + " └─ExchangeReceiver_38 1.00 mpp[tiflash] ", + " └─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#21)->Column#24, funcs:sum(Column#22)->Column#25, funcs:sum(Column#23)->Column#26, funcs:count(distinct test.t.c)->Column#27", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_32 1.00 mpp[tiflash] group by:Column#30, funcs:sum(Column#28)->Column#21, funcs:count(1)->Column#22, funcs:count(Column#29)->Column#23", + " └─Projection_44 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#28, test.t.a->Column#29, test.t.c->Column#30", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b+a), max(b), count(distinct c), count(*) from t having count(a) > 2;", + "Plan": [ + "TableReader_43 0.80 root MppVersion: 2, data:ExchangeSender_42", + "└─ExchangeSender_42 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 0.80 mpp[tiflash] Column#7->Column#12, Column#8->Column#13, Column#9->Column#14, Column#10->Column#15", + " └─Selection_41 0.80 mpp[tiflash] gt(Column#11, 2)", + " └─Projection_33 1.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#10, Column#11", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#29)->Column#7, funcs:max(Column#30)->Column#8, funcs:sum(Column#31)->Column#9, funcs:sum(Column#32)->Column#10, funcs:sum(Column#33)->Column#11", + " └─ExchangeReceiver_38 1.00 mpp[tiflash] ", + " └─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_34 1.00 mpp[tiflash] funcs:sum(Column#25)->Column#29, funcs:max(Column#26)->Column#30, funcs:count(distinct test.t.c)->Column#31, funcs:sum(Column#27)->Column#32, funcs:sum(Column#28)->Column#33", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_32 1.00 mpp[tiflash] group by:Column#37, funcs:sum(Column#34)->Column#25, funcs:max(Column#35)->Column#26, funcs:count(1)->Column#27, funcs:count(Column#36)->Column#28", + " └─Projection_44 10000.00 mpp[tiflash] cast(plus(test.t.b, test.t.a), decimal(20,0) BINARY)->Column#34, test.t.b->Column#35, test.t.a->Column#36, test.t.c->Column#37", + " └─TableFullScan_18 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b), count(distinct a, b, e), count(a+b) from t;", + "Plan": [ + "TableReader_36 1.00 root MppVersion: 2, data:ExchangeSender_35", + "└─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_29 1.00 mpp[tiflash] Column#7, Column#8, Column#9", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#17)->Column#7, funcs:sum(Column#18)->Column#8, funcs:sum(Column#19)->Column#9", + " └─ExchangeReceiver_34 1.00 mpp[tiflash] ", + " └─ExchangeSender_33 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_30 1.00 mpp[tiflash] funcs:sum(Column#15)->Column#17, funcs:count(distinct test.t.a, test.t.b, test.t.e)->Column#18, funcs:sum(Column#16)->Column#19", + " └─ExchangeReceiver_32 1.00 mpp[tiflash] ", + " └─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: test.t.e, collate: utf8mb4_general_ci]", + " └─HashAgg_28 1.00 mpp[tiflash] group by:Column#22, Column#23, Column#24, funcs:sum(Column#20)->Column#15, funcs:count(Column#21)->Column#16", + " └─Projection_37 10000.00 mpp[tiflash] cast(test.t.b, decimal(20,0) BINARY)->Column#20, plus(test.t.a, test.t.b)->Column#21, test.t.a->Column#22, test.t.b->Column#23, test.t.e->Column#24", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct b), json_objectagg(d,c) from t;", + "Plan": [ + "HashAgg_6 1.00 root funcs:count(distinct Column#9)->Column#7, funcs:json_objectagg(Column#10, Column#11)->Column#8", + "└─Projection_18 10000.00 root test.t.b->Column#9, cast(test.t.d, var_string(10))->Column#10, test.t.c->Column#11", + " └─TableReader_12 10000.00 root MppVersion: 2, data:ExchangeSender_11", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to tiflash because AggFunc `json_objectagg` is not supported now", + "Aggregation can not be pushed to tiflash because AggFunc `json_objectagg` is not supported now" + ] + }, + { + "SQL": "EXPLAIN select count(distinct c+a), count(a) from t;", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_27 1.00 mpp[tiflash] Column#7, Column#8", + " └─HashAgg_28 1.00 mpp[tiflash] funcs:count(distinct Column#11)->Column#7, funcs:sum(Column#12)->Column#8", + " └─ExchangeReceiver_30 1.00 mpp[tiflash] ", + " └─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_26 1.00 mpp[tiflash] group by:Column#14, funcs:count(Column#13)->Column#12", + " └─Projection_33 10000.00 mpp[tiflash] test.t.a->Column#13, plus(test.t.c, test.t.a)->Column#14", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select sum(b), count(distinct c+a, b, e), count(a+b) from t;", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_27 1.00 mpp[tiflash] Column#7, Column#8, Column#9", + " └─HashAgg_28 1.00 mpp[tiflash] funcs:sum(Column#13)->Column#7, funcs:count(distinct Column#14, test.t.b, test.t.e)->Column#8, funcs:sum(Column#15)->Column#9", + " └─ExchangeReceiver_30 1.00 mpp[tiflash] ", + " └─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_26 1.00 mpp[tiflash] group by:Column#18, Column#19, Column#20, funcs:sum(Column#16)->Column#13, funcs:count(Column#17)->Column#15", + " └─Projection_33 10000.00 mpp[tiflash] cast(test.t.b, decimal(20,0) BINARY)->Column#16, plus(test.t.a, test.t.b)->Column#17, plus(test.t.c, test.t.a)->Column#18, test.t.b->Column#19, test.t.e->Column#20", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestMPPMultiDistinct3Stage", + "Cases": [ + { + "SQL": "EXPLAIN select count(distinct a) from t", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_25 1.00 mpp[tiflash] Column#6", + " └─HashAgg_26 1.00 mpp[tiflash] funcs:sum(Column#8)->Column#6", + " └─ExchangeReceiver_30 1.00 mpp[tiflash] ", + " └─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_26 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#8", + " └─ExchangeReceiver_28 1.00 mpp[tiflash] ", + " └─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg_24 1.00 mpp[tiflash] group by:test.t.a, ", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a) from t", + "Plan": [ + "5" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a), count(distinct b) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#12)->Column#6, funcs:sum(Column#13)->Column#7", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#12, funcs:count(distinct test.t.b)->Column#13", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#11, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#11, test.t.a, test.t.b, ", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#11, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b) from t", + "Plan": [ + "5 5" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a), count(distinct b), count(c) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#6, funcs:sum(Column#19)->Column#7, funcs:sum(Column#20)->Column#8", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#18, funcs:count(distinct test.t.b)->Column#19, funcs:sum(Column#15)->Column#20", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#16, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#16, test.t.a, test.t.b, funcs:count(Column#17)->Column#15", + " └─Projection_32 20000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c, Column#16, case(eq(Column#16, 1), test.t.c, )->Column#17", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#16, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b), count(c) from t", + "Plan": [ + "5 5 10" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a), count(distinct b), count(c+1) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#6, funcs:sum(Column#19)->Column#7, funcs:sum(Column#20)->Column#8", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#18, funcs:count(distinct test.t.b)->Column#19, funcs:sum(Column#15)->Column#20", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#16, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#16, test.t.a, test.t.b, funcs:count(Column#17)->Column#15", + " └─Projection_32 20000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c, Column#16, case(eq(Column#16, 1), plus(test.t.c, 1), )->Column#17", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#16, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b), count(c+1) from t", + "Plan": [ + "5 5 10" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a), count(distinct b), sum(c) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#6, funcs:sum(Column#19)->Column#7, funcs:sum(Column#20)->Column#8", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#18, funcs:count(distinct test.t.b)->Column#19, funcs:sum(Column#15)->Column#20", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#16, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#22, Column#23, Column#24, funcs:sum(Column#21)->Column#15", + " └─Projection_39 20000.00 mpp[tiflash] cast(Column#17, decimal(10,0) BINARY)->Column#21, test.t.a->Column#22, test.t.b->Column#23, Column#16->Column#24", + " └─Projection_32 20000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c, Column#16, case(eq(Column#16, 1), test.t.c, )->Column#17", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#16, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b), sum(c) from t", + "Plan": [ + "5 5 30000" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a, b), count(distinct b), count(c), sum(d) from t", + "Plan": [ + "TableReader_28 1.00 root MppVersion: 2, data:ExchangeSender_27", + "└─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_23 1.00 mpp[tiflash] Column#6, Column#7, Column#8, Column#9", + " └─HashAgg_24 1.00 mpp[tiflash] funcs:count(distinct test.t.a, test.t.b)->Column#6, funcs:count(distinct test.t.b)->Column#7, funcs:sum(Column#12)->Column#8, funcs:sum(Column#13)->Column#9", + " └─ExchangeReceiver_26 1.00 mpp[tiflash] ", + " └─ExchangeSender_25 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_22 1.00 mpp[tiflash] group by:Column#16, Column#17, funcs:count(Column#14)->Column#12, funcs:sum(Column#15)->Column#13", + " └─Projection_29 10000.00 mpp[tiflash] test.t.c->Column#14, cast(test.t.d, decimal(10,0) BINARY)->Column#15, test.t.a->Column#16, test.t.b->Column#17", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Some grouping sets should be merged", + "Some grouping sets should be merged" + ] + }, + { + "SQL": "select count(distinct a, b), count(distinct b), count(c), sum(d) from t", + "Plan": [ + "5 5 10 10" + ], + "Warn": [ + "Some grouping sets should be merged", + "Some grouping sets should be merged" + ] + }, + { + "SQL": "EXPLAIN select count(distinct a+b), sum(c) from t", + "Plan": [ + "TableReader_28 1.00 root MppVersion: 2, data:ExchangeSender_27", + "└─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_23 1.00 mpp[tiflash] Column#6, Column#7", + " └─HashAgg_24 1.00 mpp[tiflash] funcs:count(distinct Column#10)->Column#6, funcs:sum(Column#11)->Column#7", + " └─ExchangeReceiver_26 1.00 mpp[tiflash] ", + " └─ExchangeSender_25 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_22 1.00 mpp[tiflash] group by:Column#13, funcs:sum(Column#12)->Column#11", + " └─Projection_29 10000.00 mpp[tiflash] cast(test.t.c, decimal(10,0) BINARY)->Column#12, plus(test.t.a, test.t.b)->Column#13", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a+b), sum(c) from t", + "Plan": [ + "5 30000" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select count(distinct a+b), count(distinct b+c), count(c) from t", + "Plan": [ + "TableReader_28 1.00 root MppVersion: 2, data:ExchangeSender_27", + "└─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_23 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_24 1.00 mpp[tiflash] funcs:count(distinct Column#12)->Column#6, funcs:count(distinct Column#13)->Column#7, funcs:sum(Column#14)->Column#8", + " └─ExchangeReceiver_26 1.00 mpp[tiflash] ", + " └─ExchangeSender_25 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_22 1.00 mpp[tiflash] group by:Column#16, Column#17, funcs:count(Column#15)->Column#14", + " └─Projection_29 10000.00 mpp[tiflash] test.t.c->Column#15, plus(test.t.a, test.t.b)->Column#16, plus(test.t.b, test.t.c)->Column#17", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a+b), count(distinct b+c), count(c) from t", + "Plan": [ + "5 5 10" + ], + "Warn": null + }, + { + "SQL": "explain select count(distinct a,c), count(distinct b,c), count(c) from t", + "Plan": [ + "TableReader_28 1.00 root MppVersion: 2, data:ExchangeSender_27", + "└─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_23 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_24 1.00 mpp[tiflash] funcs:count(distinct test.t.a, test.t.c)->Column#6, funcs:count(distinct test.t.b, test.t.c)->Column#7, funcs:sum(Column#10)->Column#8", + " └─ExchangeReceiver_26 1.00 mpp[tiflash] ", + " └─ExchangeSender_25 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_22 1.00 mpp[tiflash] group by:test.t.a, test.t.b, test.t.c, funcs:count(test.t.c)->Column#10", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b), count(*) from t", + "Plan": [ + "5 5 10" + ], + "Warn": null + }, + { + "SQL": "explain select count(distinct a), count(distinct b), count(*) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7, Column#8", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#18)->Column#6, funcs:sum(Column#19)->Column#7, funcs:sum(Column#20)->Column#8", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#18, funcs:count(distinct test.t.b)->Column#19, funcs:sum(Column#15)->Column#20", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#16, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#16, test.t.a, test.t.b, funcs:count(Column#17)->Column#15", + " └─Projection_32 20000.00 mpp[tiflash] test.t.a, test.t.b, Column#16, case(eq(Column#16, 1), 1, )->Column#17", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#16, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select count(distinct a), count(distinct b), avg(c+d) from t", + "Plan": [ + "5 5 3001.0000" + ], + "Warn": null + }, + { + "SQL": "explain select count(distinct a), count(distinct b), avg(c+d) from t", + "Plan": [ + "TableReader_38 1.00 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_28 1.00 mpp[tiflash] Column#6, Column#7, div(Column#8, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#8", + " └─HashAgg_29 1.00 mpp[tiflash] funcs:sum(Column#25)->Column#6, funcs:sum(Column#26)->Column#7, funcs:sum(Column#27)->Column#19, funcs:sum(Column#28)->Column#8", + " └─ExchangeReceiver_36 1.00 mpp[tiflash] ", + " └─ExchangeSender_35 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg_31 1.00 mpp[tiflash] funcs:count(distinct test.t.a)->Column#25, funcs:count(distinct test.t.b)->Column#26, funcs:sum(Column#20)->Column#27, funcs:sum(Column#21)->Column#28", + " └─ExchangeReceiver_34 16000.00 mpp[tiflash] ", + " └─ExchangeSender_33 16000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.b, collate: binary], [name: Column#22, collate: binary]", + " └─HashAgg_27 16000.00 mpp[tiflash] group by:Column#31, Column#32, Column#33, funcs:count(Column#29)->Column#20, funcs:sum(Column#30)->Column#21", + " └─Projection_39 20000.00 mpp[tiflash] Column#23->Column#29, cast(Column#24, decimal(20,0) BINARY)->Column#30, test.t.a->Column#31, test.t.b->Column#32, Column#22->Column#33", + " └─Projection_32 20000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c, test.t.d, Column#22, case(eq(Column#22, 1), plus(test.t.c, test.t.d), )->Column#23, case(eq(Column#22, 1), plus(test.t.c, test.t.d), )->Column#24", + " └─Expand_30 20000.00 mpp[tiflash] group set num:2, groupingID:Column#22, [{},{}]", + " └─TableFullScan_12 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestMPPNullAwareSemiJoinPushDown", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_enable_null_aware_anti_join=on;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select * from t where t.a not in (select s.a from s); -- 1. anti semi join, one join key", + "Plan": [ + "TableReader_29 8000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 8000.00 mpp[tiflash] Null-aware anti semi join, equal:[eq(test.t.a, test.s.a)]", + " ├─ExchangeReceiver_14(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_11(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select * from t where t.a not in (select s.a from s where t.c > s.c); -- 2. anti semi join, one join key + other condition", + "Plan": [ + "TableReader_30 8000.00 root MppVersion: 2, data:ExchangeSender_29", + "└─ExchangeSender_29 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_28 8000.00 mpp[tiflash] Null-aware anti semi join, equal:[eq(test.t.a, test.s.a)], other cond:gt(test.t.c, test.s.c)", + " ├─ExchangeReceiver_15(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_13 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_12(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select * from t where (t.a, t.b) not in (select s.a, s.b from s); -- 3. anti semi join, two join key", + "Plan": [ + "TableReader_29 8000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 8000.00 mpp[tiflash] Null-aware anti semi join, equal:[eq(test.t.a, test.s.a) eq(test.t.b, test.s.b)]", + " ├─ExchangeReceiver_14(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_11(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select * from t where (t.a, t.b) not in (select s.a, s.b from s where t.c < s.c); -- 4. anti semi join, two join key + other condition", + "Plan": [ + "TableReader_30 8000.00 root MppVersion: 2, data:ExchangeSender_29", + "└─ExchangeSender_29 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_28 8000.00 mpp[tiflash] Null-aware anti semi join, equal:[eq(test.t.a, test.s.a) eq(test.t.b, test.s.b)], other cond:lt(test.t.c, test.s.c)", + " ├─ExchangeReceiver_15(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_13 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_12(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select *, t.a not in (select s.a from s) from t; -- 5. left anti semi join, one join key", + "Plan": [ + "TableReader_29 10000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 10000.00 mpp[tiflash] Null-aware anti left outer semi join, equal:[eq(test.t.a, test.s.a)]", + " ├─ExchangeReceiver_14(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_11(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select *, t.a not in (select s.a from s where t.c > s.c) from t; -- 6. left anti semi join, one join key + other condition", + "Plan": [ + "TableReader_30 10000.00 root MppVersion: 2, data:ExchangeSender_29", + "└─ExchangeSender_29 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_28 10000.00 mpp[tiflash] Null-aware anti left outer semi join, equal:[eq(test.t.a, test.s.a)], other cond:gt(test.t.c, test.s.c)", + " ├─ExchangeReceiver_15(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_13 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_12(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s) from t; -- 7. left anti semi join, two join key", + "Plan": [ + "TableReader_29 10000.00 root MppVersion: 2, data:ExchangeSender_28", + "└─ExchangeSender_28 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_27 10000.00 mpp[tiflash] Null-aware anti left outer semi join, equal:[eq(test.t.a, test.s.a) eq(test.t.b, test.s.b)]", + " ├─ExchangeReceiver_14(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_12 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_11(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s where t.c < s.c) from t; -- 8. left anti semi join, two join key + other condition", + "Plan": [ + "TableReader_30 10000.00 root MppVersion: 2, data:ExchangeSender_29", + "└─ExchangeSender_29 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_28 10000.00 mpp[tiflash] Null-aware anti left outer semi join, equal:[eq(test.t.a, test.s.a) eq(test.t.b, test.s.b)], other cond:lt(test.t.c, test.s.c)", + " ├─ExchangeReceiver_15(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender_14 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_13 10000.00 mpp[tiflash] table:s keep order:false, stats:pseudo", + " └─TableFullScan_12(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestMPPSharedCTEScan", + "Cases": [ + { + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "Plan": [ + "TableReader 9999.99 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 9999.99 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "Plan": [ + "HashJoin 9999.99 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection(Build) 7999.99 root not(isnull(test.t.b))", + "│ └─CTEFullScan 9999.99 root CTE:c1 AS c2 data:CTE_0", + "└─Selection(Probe) 7999.99 root not(isnull(test.t.a))", + " └─CTEFullScan 9999.99 root CTE:c1 data:CTE_0", + "CTE_0 9999.99 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.99 root data:Selection", + " └─Selection 9999.99 cop[tikv] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + "Plan": [ + "TableReader 9999.99 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 9999.99 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9999.99 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.c))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "Plan": [ + "Limit 10.00 root offset:0, count:10", + "└─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 10.00 mpp[tiflash] offset:0, count:10", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + "Plan": [ + "TopN 10.00 root test.t.a, offset:0, count:10", + "└─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] test.t.a, offset:0, count:10", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + "Plan": [ + "TableReader 12500.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence 12500.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + "Plan": [ + "HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + "│ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + "└─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "TableReader 19531.25 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19531.25 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19531.25 mpp[tiflash] test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c", + " └─Sequence 19531.25 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 19531.25 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9999.98 mpp[tiflash] ", + " │ └─ExchangeSender 9999.98 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Sequence 9999.98 mpp[tiflash] Sequence Node", + " │ ├─CTE_2 9999.98 mpp[tiflash] Non-Recursive CTE Storage", + " │ │ └─Selection 9999.98 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─HashJoin 9999.98 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", + " │ ├─ExchangeReceiver(Build) 7999.98 mpp[tiflash] ", + " │ │ └─ExchangeSender 7999.98 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 7999.98 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", + " │ │ └─CTEFullScan 9999.98 mpp[tiflash] CTE:c3 data:CTE_2", + " │ └─Selection(Probe) 7999.98 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 9999.98 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", + " └─HashJoin(Probe) 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 9999.98 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 7999.98 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 9999.98 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 7999.98 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 9999.98 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 9999.98 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.98 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 9999.98 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 9999.98 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select /*+ read_from_storage(tikv[t]) */ * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 9999.98 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 7999.98 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 9999.98 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 7999.98 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 9999.98 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 9999.98 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.98 root data:Selection", + " └─Selection 9999.98 cop[tikv] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "TableReader 19531.25 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 19531.25 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19531.25 mpp[tiflash] test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c", + " └─Sequence 19531.25 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 19531.25 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Sequence 8000.00 mpp[tiflash] Sequence Node", + " │ ├─CTE_2 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ │ └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", + " │ ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", + " │ │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 data:CTE_2", + " │ └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", + " └─HashJoin(Probe) 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 8000.00 root Non-Recursive CTE", + "└─Selection(Seed Part) 8000.00 root or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + } + ] + }, + { + "Name": "TestRollupMPP", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(1) from t group by a, b with rollup; -- 1. simple agg", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:count(1)->Column#8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#5, ->Column#6, 0->gid],[Column#5, ->Column#6, 1->gid],[Column#5, Column#6, 3->gid]; schema: [Column#5,Column#6,gid]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select sum(c), count(1) from t group by a, b with rollup; -- 2. non-grouping set col c", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#8, Column#9", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#19, Column#20, Column#21, funcs:sum(Column#18)->Column#8, funcs:count(1)->Column#9", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.c, decimal(10,0) BINARY)->Column#18, Column#5->Column#19, Column#6->Column#20, gid->Column#21", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.t.c, ->Column#5, ->Column#6, 0->gid],[test.t.c, Column#5, ->Column#6, 1->gid],[test.t.c, Column#5, Column#6, 3->gid]; schema: [test.t.c,Column#5,Column#6,gid]", + " └─Projection 10000.00 mpp[tiflash] test.t.c, test.t.a->Column#5, test.t.b->Column#6", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select count(a) from t group by a, b with rollup; -- 3. should keep the original col a", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:sum(Column#12)->Column#8", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:count(test.t.a)->Column#12", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.t.a, ->Column#5, ->Column#6, 0->gid],[test.t.a, Column#5, ->Column#6, 1->gid],[test.t.a, Column#5, Column#6, 3->gid]; schema: [test.t.a,Column#5,Column#6,gid]", + " └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.a->Column#5, test.t.b->Column#6", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select grouping(a) from t group by a, b with rollup; -- 4. contain grouping function ref to grouping set column a", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] grouping(gid)->Column#9", + " └─Projection 8000.00 mpp[tiflash] gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:firstrow(gid)->gid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, ", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#5, ->Column#6, 0->gid],[Column#5, ->Column#6, 1->gid],[Column#5, Column#6, 3->gid]; schema: [Column#5,Column#6,gid]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select grouping(a,b) from t group by a, b with rollup; -- 5. grouping function contains grouping set column a,c", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] grouping(gid)->Column#9", + " └─Projection 8000.00 mpp[tiflash] gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:firstrow(gid)->gid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, ", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#5, ->Column#6, 0->gid],[Column#5, ->Column#6, 1->gid],[Column#5, Column#6, 3->gid]; schema: [Column#5,Column#6,gid]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select a, grouping(b,a) from t group by a,b with rollup; -- 6. resolve normal column a to grouping set column a'", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5->Column#9, grouping(gid)->Column#10", + " └─Projection 8000.00 mpp[tiflash] Column#5, gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:firstrow(Column#5)->Column#5, funcs:firstrow(gid)->gid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, ", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#5, ->Column#6, 0->gid],[Column#5, ->Column#6, 1->gid],[Column#5, Column#6, 3->gid]; schema: [Column#5,Column#6,gid]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select a+1, grouping(b) from t group by a+1, b with rollup; -- 7. resolve field list a+1 to grouping set column a+1", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5->Column#9, grouping(gid)->Column#10", + " └─Projection 8000.00 mpp[tiflash] Column#5, gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, funcs:firstrow(Column#5)->Column#5, funcs:firstrow(gid)->gid", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#5, collate: binary], [name: Column#6, collate: binary], [name: gid, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#5, Column#6, gid, ", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#5, ->Column#6, 0->gid],[Column#5, ->Column#6, 1->gid],[Column#5, Column#6, 3->gid]; schema: [Column#5,Column#6,gid]", + " └─Projection 10000.00 mpp[tiflash] plus(test.t.a, 1)->Column#5, test.t.b->Column#6", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT SUM(profit) AS profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP order by year+2; -- 8. order by item year+2 resolve to gby grouping expression", + "Plan": [ + "Projection 8000.00 root Column#9", + "└─Sort 8000.00 root Column#6", + " └─TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#9, Column#6", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#19, Column#20, Column#21, funcs:sum(Column#17)->Column#9, funcs:firstrow(Column#18)->Column#6", + " └─Projection 10000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#17, Column#6->Column#18, Column#6->Column#19, Column#7->Column#20, gid->Column#21", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: binary], [name: gid, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid],[test.sales.profit, Column#6, ->Column#7, 1->gid],[test.sales.profit, Column#6, Column#7, 3->gid]; schema: [test.sales.profit,Column#6,Column#7,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, plus(test.sales.year, 2)->Column#6, plus(test.sales.year, test.sales.profit)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year+2, SUM(profit) AS profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP order by year+2; -- 9. order by item year+2 resolve to select field", + "Plan": [ + "Projection 8000.00 root Column#6->Column#10, Column#9", + "└─Sort 8000.00 root Column#6", + " └─TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#9, Column#6", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, Column#21, Column#22, funcs:sum(Column#18)->Column#9, funcs:firstrow(Column#19)->Column#6", + " └─Projection 10000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#18, Column#6->Column#19, Column#6->Column#20, Column#7->Column#21, gid->Column#22", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: binary], [name: gid, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid],[test.sales.profit, Column#6, ->Column#7, 1->gid],[test.sales.profit, Column#6, Column#7, 3->gid]; schema: [test.sales.profit,Column#6,Column#7,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, plus(test.sales.year, 2)->Column#6, plus(test.sales.year, test.sales.profit)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year+2 as y, SUM(profit) as profit FROM sales GROUP BY year+2, year+profit WITH ROLLUP having y > 2002 order by year+2, profit; -- 10. having (year+2) shouldn't be pushed down", + "Plan": [ + "Projection 6400.00 root Column#6, Column#9", + "└─Sort 6400.00 root Column#6, Column#9", + " └─TableReader 6400.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#9, Column#6", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#20, Column#21, Column#22, funcs:sum(Column#18)->Column#9, funcs:firstrow(Column#19)->Column#6", + " └─Projection 8000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#18, Column#6->Column#19, Column#6->Column#20, Column#7->Column#21, gid->Column#22", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: binary], [name: gid, collate: binary]", + " └─Selection 8000.00 mpp[tiflash] gt(Column#6, 2002)", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid],[test.sales.profit, Column#6, ->Column#7, 1->gid],[test.sales.profit, Column#6, Column#7, 3->gid]; schema: [test.sales.profit,Column#6,Column#7,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, plus(test.sales.year, 2)->Column#6, plus(test.sales.year, test.sales.profit)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year+2 as y, SUM(profit) AS profit, grouping(year+2) FROM sales GROUP BY year+2, year+profit WITH ROLLUP having y > 2002 order by year+2, profit; -- 11. grouping function validation", + "Plan": [ + "Sort 6400.00 root Column#6, Column#9", + "└─TableReader 6400.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#6, Column#9, grouping(gid)->Column#10", + " └─Projection 6400.00 mpp[tiflash] Column#9, Column#6, gid", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#29, Column#30, Column#31, funcs:sum(Column#26)->Column#9, funcs:firstrow(Column#27)->Column#6, funcs:firstrow(Column#28)->gid", + " └─Projection 8000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#26, Column#6->Column#27, gid->Column#28, Column#6->Column#29, Column#7->Column#30, gid->Column#31", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: binary], [name: gid, collate: binary]", + " └─Selection 8000.00 mpp[tiflash] gt(Column#6, 2002)", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid],[test.sales.profit, Column#6, ->Column#7, 1->gid],[test.sales.profit, Column#6, Column#7, 3->gid]; schema: [test.sales.profit,Column#6,Column#7,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, plus(test.sales.year, 2)->Column#6, plus(test.sales.year, test.sales.profit)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year, country, product, SUM(profit) AS profit FROM sales GROUP BY year, country, product with rollup order by grouping(year); -- 12. grouping function in order by clause", + "Plan": [ + "Projection 8000.00 root Column#6, Column#7->Column#12, Column#8->Column#13, Column#10", + "└─Projection 8000.00 root Column#10, Column#6, Column#7, Column#8, gid", + " └─Sort 8000.00 root Column#34", + " └─Projection 8000.00 root Column#10, Column#6, Column#7, Column#8, gid, grouping(gid)->Column#34", + " └─TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#10, Column#6, Column#7, Column#8, gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#30, Column#31, Column#32, Column#33, funcs:sum(Column#25)->Column#10, funcs:firstrow(Column#26)->Column#6, funcs:firstrow(Column#27)->Column#7, funcs:firstrow(Column#28)->Column#8, funcs:firstrow(Column#29)->gid", + " └─Projection 10000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#25, Column#6->Column#26, Column#7->Column#27, Column#8->Column#28, gid->Column#29, Column#6->Column#30, Column#7->Column#31, Column#8->Column#32, gid->Column#33", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: utf8mb4_bin], [name: Column#8, collate: utf8mb4_bin], [name: gid, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, ->Column#8, 0->gid],[test.sales.profit, Column#6, ->Column#7, ->Column#8, 1->gid],[test.sales.profit, Column#6, Column#7, ->Column#8, 3->gid],[test.sales.profit, Column#6, Column#7, Column#8, 7->gid]; schema: [test.sales.profit,Column#6,Column#7,Column#8,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, test.sales.year->Column#6, test.sales.country->Column#7, test.sales.product->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT country, product, SUM(profit) AS profit FROM sales GROUP BY country, country, product with rollup order by grouping(country); -- 13. 12 under gpos case", + "Plan": [ + "Projection 8000.00 root Column#6, Column#7->Column#12, Column#10", + "└─Projection 8000.00 root Column#10, Column#6, Column#7, gid", + " └─Sort 8000.00 root Column#31", + " └─Projection 8000.00 root Column#10, Column#6, Column#7, gid, grouping(gid)->Column#31", + " └─TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#10, Column#6, Column#7, gid", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#26, Column#27, Column#28, Column#29, Column#30, funcs:sum(Column#22)->Column#10, funcs:firstrow(Column#23)->Column#6, funcs:firstrow(Column#24)->Column#7, funcs:firstrow(Column#25)->gid", + " └─Projection 10000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#22, Column#6->Column#23, Column#7->Column#24, gid->Column#25, Column#6->Column#26, Column#6->Column#27, Column#7->Column#28, gid->Column#29, gpos->Column#30", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: utf8mb4_bin], [name: Column#6, collate: utf8mb4_bin], [name: Column#7, collate: utf8mb4_bin], [name: gid, collate: binary], [name: gpos, collate: binary]", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid, 0->gpos],[test.sales.profit, Column#6, ->Column#7, 1->gid, 1->gpos],[test.sales.profit, Column#6, ->Column#7, 1->gid, 2->gpos],[test.sales.profit, Column#6, Column#7, 3->gid, 3->gpos]; schema: [test.sales.profit,Column#6,Column#7,gid,gpos]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, test.sales.country->Column#6, test.sales.product->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year, country, product, SUM(profit) AS profit FROM sales GROUP BY year, country, product with rollup having grouping(year) > 0 order by grouping(year); -- 14. grouping function in having clause", + "Plan": [ + "Projection 6400.00 root Column#6, Column#7->Column#12, Column#8->Column#13, Column#10", + "└─Projection 6400.00 root Column#10, Column#6, Column#7, Column#8, gid", + " └─Sort 6400.00 root Column#34", + " └─Projection 6400.00 root Column#10, Column#6, Column#7, Column#8, gid, grouping(gid)->Column#34", + " └─TableReader 6400.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#10, Column#6, Column#7, Column#8, gid", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#30, Column#31, Column#32, Column#33, funcs:sum(Column#25)->Column#10, funcs:firstrow(Column#26)->Column#6, funcs:firstrow(Column#27)->Column#7, funcs:firstrow(Column#28)->Column#8, funcs:firstrow(Column#29)->gid", + " └─Projection 8000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#25, Column#6->Column#26, Column#7->Column#27, Column#8->Column#28, gid->Column#29, Column#6->Column#30, Column#7->Column#31, Column#8->Column#32, gid->Column#33", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: utf8mb4_bin], [name: Column#8, collate: utf8mb4_bin], [name: gid, collate: binary]", + " └─Selection 8000.00 mpp[tiflash] gt(grouping(gid), 0)", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, ->Column#8, 0->gid],[test.sales.profit, Column#6, ->Column#7, ->Column#8, 1->gid],[test.sales.profit, Column#6, Column#7, ->Column#8, 3->gid],[test.sales.profit, Column#6, Column#7, Column#8, 7->gid]; schema: [test.sales.profit,Column#6,Column#7,Column#8,gid]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, test.sales.year->Column#6, test.sales.country->Column#7, test.sales.product->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT country, product, SUM(profit) AS profit FROM sales GROUP BY country, country, product with rollup having grouping(country) > 0 order by grouping(country); -- 15. 14 under gpos case", + "Plan": [ + "Projection 6400.00 root Column#6, Column#7->Column#12, Column#10", + "└─Projection 6400.00 root Column#10, Column#6, Column#7, gid", + " └─Sort 6400.00 root Column#31", + " └─Projection 6400.00 root Column#10, Column#6, Column#7, gid, grouping(gid)->Column#31", + " └─TableReader 6400.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#10, Column#6, Column#7, gid", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#26, Column#27, Column#28, Column#29, Column#30, funcs:sum(Column#22)->Column#10, funcs:firstrow(Column#23)->Column#6, funcs:firstrow(Column#24)->Column#7, funcs:firstrow(Column#25)->gid", + " └─Projection 8000.00 mpp[tiflash] cast(test.sales.profit, decimal(10,0) BINARY)->Column#22, Column#6->Column#23, Column#7->Column#24, gid->Column#25, Column#6->Column#26, Column#6->Column#27, Column#7->Column#28, gid->Column#29, gpos->Column#30", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: utf8mb4_bin], [name: Column#6, collate: utf8mb4_bin], [name: Column#7, collate: utf8mb4_bin], [name: gid, collate: binary], [name: gpos, collate: binary]", + " └─Selection 8000.00 mpp[tiflash] gt(grouping(gid), 0)", + " └─Expand 10000.00 mpp[tiflash] level-projection:[test.sales.profit, ->Column#6, ->Column#7, 0->gid, 0->gpos],[test.sales.profit, Column#6, ->Column#7, 1->gid, 1->gpos],[test.sales.profit, Column#6, ->Column#7, 1->gid, 2->gpos],[test.sales.profit, Column#6, Column#7, 3->gid, 3->gpos]; schema: [test.sales.profit,Column#6,Column#7,gid,gpos]", + " └─Projection 10000.00 mpp[tiflash] test.sales.profit, test.sales.country->Column#6, test.sales.product->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' SELECT year, country, product, grouping(year, country, product) from sales group by year, country, product with rollup having grouping(year, country, product) <> 0; -- 16. grouping function recreating fix", + "Plan": [ + "TableReader 6400.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 mpp[tiflash] Column#6->Column#11, Column#7->Column#12, Column#8->Column#13, grouping(gid)->Column#14", + " └─Projection 6400.00 mpp[tiflash] Column#6, Column#7, Column#8, gid", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#6, Column#7, Column#8, gid, funcs:firstrow(Column#6)->Column#6, funcs:firstrow(Column#7)->Column#7, funcs:firstrow(Column#8)->Column#8, funcs:firstrow(gid)->gid", + " └─ExchangeReceiver 6400.00 mpp[tiflash] ", + " └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], [name: Column#7, collate: utf8mb4_bin], [name: Column#8, collate: utf8mb4_bin], [name: gid, collate: binary]", + " └─HashAgg 6400.00 mpp[tiflash] group by:Column#6, Column#7, Column#8, gid, ", + " └─Selection 8000.00 mpp[tiflash] ne(grouping(gid), 0)", + " └─Expand 10000.00 mpp[tiflash] level-projection:[->Column#6, ->Column#7, ->Column#8, 0->gid],[Column#6, ->Column#7, ->Column#8, 1->gid],[Column#6, Column#7, ->Column#8, 3->gid],[Column#6, Column#7, Column#8, 7->gid]; schema: [Column#6,Column#7,Column#8,gid]", + " └─TableFullScan 10000.00 mpp[tiflash] table:sales keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + } +] diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index f07e15e288af3..edb118f57a8af 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2555,6 +2555,16 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // Is this aggregate a final stage aggregate? // Final agg can't be split into multi-stage aggregate hasFinalAgg := len(la.AggFuncs) > 0 && la.AggFuncs[0].Mode == aggregation.FinalMode + // count final agg should become sum for MPP execution path. + // In the traditional case, TiDB take up the final agg role and push partial agg to TiKV, + // while TiDB can tell the partialMode and do the sum computation rather than counting but MPP doesn't + finalAggAdjust := func(aggFuncs []*aggregation.AggFuncDesc) { + for i, agg := range aggFuncs { + if agg.Mode == aggregation.FinalMode && agg.Name == ast.AggFuncCount { + aggFuncs[i], _ = aggregation.NewAggFuncDesc(la.SCtx(), ast.AggFuncSum, agg.Args, false) + } + } + } if len(la.GroupByItems) > 0 { partitionCols := la.GetPotentialPartitionKeys() @@ -2576,6 +2586,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp1Phase + finalAggAdjust(agg.AggFuncs) hashAggs = append(hashAggs, agg) }