planner, executor: allow build root task type of expand operator and implement expand executor (#54536)
close pingcap/tidb#42631
This commit is contained in:
@ -31,6 +31,7 @@ go_library(
|
||||
"detach.go",
|
||||
"distsql.go",
|
||||
"executor.go",
|
||||
"expand.go",
|
||||
"explain.go",
|
||||
"foreign_key.go",
|
||||
"grant.go",
|
||||
|
||||
@ -314,6 +314,8 @@ func (b *executorBuilder) build(p base.Plan) exec.Executor {
|
||||
return b.buildCompactTable(v)
|
||||
case *plannercore.AdminShowBDRRole:
|
||||
return b.buildAdminShowBDRRole(v)
|
||||
case *plannercore.PhysicalExpand:
|
||||
return b.buildExpand(v)
|
||||
default:
|
||||
if mp, ok := p.(testutil.MockPhysicalPlan); ok {
|
||||
return mp.GetExecutor()
|
||||
@ -1970,6 +1972,41 @@ func (b *executorBuilder) buildSelection(v *plannercore.PhysicalSelection) exec.
|
||||
return e
|
||||
}
|
||||
|
||||
func (b *executorBuilder) buildExpand(v *plannercore.PhysicalExpand) exec.Executor {
|
||||
childExec := b.build(v.Children()[0])
|
||||
if b.err != nil {
|
||||
return nil
|
||||
}
|
||||
levelES := make([]*expression.EvaluatorSuite, 0, len(v.LevelExprs))
|
||||
for _, exprs := range v.LevelExprs {
|
||||
// column evaluator can always refer others inside expand.
|
||||
// grouping column's nullability change should be seen as a new column projecting.
|
||||
// since input inside expand logic should be targeted and reused for N times.
|
||||
// column evaluator's swapping columns logic will pollute the input data.
|
||||
levelE := expression.NewEvaluatorSuite(exprs, true)
|
||||
levelES = append(levelES, levelE)
|
||||
}
|
||||
e := &ExpandExec{
|
||||
BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID(), childExec),
|
||||
numWorkers: int64(b.ctx.GetSessionVars().ProjectionConcurrency()),
|
||||
levelEvaluatorSuits: levelES,
|
||||
}
|
||||
|
||||
// If the calculation row count for this Projection operator is smaller
|
||||
// than a Chunk size, we turn back to the un-parallel Projection
|
||||
// implementation to reduce the goroutine overhead.
|
||||
if int64(v.StatsCount()) < int64(b.ctx.GetSessionVars().MaxChunkSize) {
|
||||
e.numWorkers = 0
|
||||
}
|
||||
|
||||
// Use un-parallel projection for query that write on memdb to avoid data race.
|
||||
// See also https://github.com/pingcap/tidb/issues/26832
|
||||
if b.inUpdateStmt || b.inDeleteStmt || b.inInsertStmt || b.hasLock {
|
||||
e.numWorkers = 0
|
||||
}
|
||||
return e
|
||||
}
|
||||
|
||||
func (b *executorBuilder) buildProjection(v *plannercore.PhysicalProjection) exec.Executor {
|
||||
childExec := b.build(v.Children()[0])
|
||||
if b.err != nil {
|
||||
|
||||
133
pkg/executor/expand.go
Normal file
133
pkg/executor/expand.go
Normal file
@ -0,0 +1,133 @@
|
||||
// Copyright 2024 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 executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/tidb/pkg/executor/internal/exec"
|
||||
"github.com/pingcap/tidb/pkg/expression"
|
||||
"github.com/pingcap/tidb/pkg/util/chunk"
|
||||
"github.com/pingcap/tidb/pkg/util/execdetails"
|
||||
"github.com/pingcap/tidb/pkg/util/memory"
|
||||
)
|
||||
|
||||
// ExpandExec is used to execute expand logical plan.
|
||||
type ExpandExec struct {
|
||||
exec.BaseExecutor
|
||||
|
||||
numWorkers int64
|
||||
childResult *chunk.Chunk
|
||||
memTracker *memory.Tracker
|
||||
|
||||
// levelIterOffset is responsible for the level iteration offset.
|
||||
levelIterOffset int64
|
||||
|
||||
// levelEvaluatorSuits is responsible for the level projections.
|
||||
// each level is an implicit projection helped by a evaluatorSuit.
|
||||
levelEvaluatorSuits []*expression.EvaluatorSuite
|
||||
}
|
||||
|
||||
// Open implements the Executor Open interface.
|
||||
func (e *ExpandExec) Open(ctx context.Context) error {
|
||||
if err := e.BaseExecutor.Open(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
return e.open(ctx)
|
||||
}
|
||||
|
||||
func (e *ExpandExec) open(_ context.Context) error {
|
||||
if e.memTracker != nil {
|
||||
e.memTracker.Reset()
|
||||
} else {
|
||||
e.memTracker = memory.NewTracker(e.ID(), -1)
|
||||
}
|
||||
e.memTracker.AttachTo(e.Ctx().GetSessionVars().StmtCtx.MemTracker)
|
||||
|
||||
// todo: implement the parallel execution logic
|
||||
e.numWorkers = 0
|
||||
|
||||
if e.isUnparalleled() {
|
||||
// levelIterOffset = -1 means we should cache a child chunk first.
|
||||
e.levelIterOffset = -1
|
||||
e.childResult = exec.TryNewCacheChunk(e.Children(0))
|
||||
e.memTracker.Consume(e.childResult.MemoryUsage())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Next implements the Executor Next interface.
|
||||
func (e *ExpandExec) Next(ctx context.Context, req *chunk.Chunk) error {
|
||||
req.GrowAndReset(e.MaxChunkSize())
|
||||
if e.isUnparalleled() {
|
||||
return e.unParallelExecute(ctx, req)
|
||||
}
|
||||
return e.parallelExecute(ctx, req)
|
||||
}
|
||||
|
||||
func (e *ExpandExec) isUnparalleled() bool {
|
||||
return e.numWorkers <= 0
|
||||
}
|
||||
|
||||
func (e *ExpandExec) unParallelExecute(ctx context.Context, chk *chunk.Chunk) error {
|
||||
// for one cache input chunk, if it has already been processed N times, we need to fetch a new one.
|
||||
if e.levelIterOffset == -1 || e.levelIterOffset > int64(len(e.levelEvaluatorSuits)-1) {
|
||||
e.childResult.SetRequiredRows(chk.RequiredRows(), e.MaxChunkSize())
|
||||
mSize := e.childResult.MemoryUsage()
|
||||
err := exec.Next(ctx, e.Children(0), e.childResult)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
e.memTracker.Consume(e.childResult.MemoryUsage() - mSize)
|
||||
if e.childResult.NumRows() == 0 {
|
||||
return nil
|
||||
}
|
||||
// when cache a new child chunk, rewind the levelIterOffset.
|
||||
e.levelIterOffset = 0
|
||||
}
|
||||
evalCtx := e.Ctx().GetExprCtx().GetEvalCtx()
|
||||
enableVectorized := e.Ctx().GetSessionVars().EnableVectorizedExpression
|
||||
err := e.levelEvaluatorSuits[e.levelIterOffset].Run(evalCtx, enableVectorized, e.childResult, chk)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
e.levelIterOffset++
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*ExpandExec) parallelExecute(_ context.Context, _ *chunk.Chunk) error {
|
||||
return errors.New("parallel expand eval logic not implemented")
|
||||
}
|
||||
|
||||
// Close implements the Executor Close interface.
|
||||
func (e *ExpandExec) Close() error {
|
||||
// if e.BaseExecutor.Open returns error, e.childResult will be nil, see https://github.com/pingcap/tidb/issues/24210
|
||||
// for more information
|
||||
if e.isUnparalleled() && e.childResult != nil {
|
||||
e.memTracker.Consume(-e.childResult.MemoryUsage())
|
||||
e.childResult = nil
|
||||
}
|
||||
if e.BaseExecutor.RuntimeStats() != nil {
|
||||
runtimeStats := &execdetails.RuntimeStatsWithConcurrencyInfo{}
|
||||
if e.isUnparalleled() {
|
||||
runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", 0))
|
||||
} else {
|
||||
runtimeStats.SetConcurrencyInfo(execdetails.NewConcurrencyInfo("Concurrency", int(e.numWorkers)))
|
||||
}
|
||||
e.Ctx().GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.ID(), runtimeStats)
|
||||
}
|
||||
return e.BaseExecutor.Close()
|
||||
}
|
||||
@ -150,7 +150,7 @@ func (e *ProjectionExec) open(_ context.Context) error {
|
||||
//
|
||||
// 1. "projectionInputFetcher" gets its input and output resources from its
|
||||
// "inputCh" and "outputCh" channel, once the input and output resources are
|
||||
// abtained, it fetches child's result into "input.chk" and:
|
||||
// obtained, it fetches child's result into "input.chk" and:
|
||||
// a. Dispatches this input to the worker specified in "input.targetWorker"
|
||||
// b. Dispatches this output to the main thread: "ProjectionExec.Next"
|
||||
// c. Dispatches this output to the worker specified in "input.targetWorker"
|
||||
|
||||
@ -10,7 +10,7 @@ go_test(
|
||||
data = glob(["testdata/**"]),
|
||||
flaky = True,
|
||||
race = "on",
|
||||
shard_count = 33,
|
||||
shard_count = 34,
|
||||
deps = [
|
||||
"//pkg/config",
|
||||
"//pkg/domain",
|
||||
|
||||
@ -1464,3 +1464,38 @@ func TestAlwaysTruePredicateWithSubquery(t *testing.T) {
|
||||
tk.MustQuery(ts).Check(testkit.Rows(output[i].Plan...))
|
||||
}
|
||||
}
|
||||
|
||||
// TestExplainExpand
|
||||
func TestExplainExpand(t *testing.T) {
|
||||
var (
|
||||
input []string
|
||||
output []struct {
|
||||
SQL string
|
||||
Plan []string
|
||||
Warning []string
|
||||
}
|
||||
)
|
||||
planSuiteData := GetPlanSuiteData()
|
||||
planSuiteData.LoadTestCases(t, &input, &output)
|
||||
store := testkit.CreateMockStore(t)
|
||||
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, d int, e int)")
|
||||
tk.MustExec("create table s(a int, b int, c int, d int, e 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, `whatever` int)")
|
||||
|
||||
// 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")
|
||||
|
||||
for i, ts := range input {
|
||||
testdata.OnRecord(func() {
|
||||
output[i].SQL = ts
|
||||
output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Rows())
|
||||
})
|
||||
tk.MustQuery(ts).Check(testkit.Rows(output[i].Plan...))
|
||||
}
|
||||
}
|
||||
|
||||
@ -619,5 +619,26 @@
|
||||
"explain select * from t WHERE TRUE = ALL ( SELECT TRUE GROUP BY 1 LIMIT 1 ) IS NULL IS NOT NULL;",
|
||||
"explain select * from t WHERE TRUE = ALL ( SELECT TRUE from t GROUP BY 1 LIMIT 1 ) is null is not null;"
|
||||
]
|
||||
},
|
||||
{
|
||||
"name": "TestExplainExpand",
|
||||
"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"
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
@ -3778,5 +3778,224 @@
|
||||
"Warning": null
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Name": "TestExplainExpand",
|
||||
"Cases": [
|
||||
{
|
||||
"SQL": "explain format = 'brief' select count(1) from t group by a, b with rollup; -- 1. simple agg",
|
||||
"Plan": [
|
||||
"HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:count(1)->Column#10",
|
||||
"└─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, 0->gid],[Column#7, <nil>->Column#8, 1->gid],[Column#7, Column#8, 3->gid]; schema: [Column#7,Column#8,gid]",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"HashAgg 8000.00 root group by:Column#15, Column#16, Column#17, funcs:sum(Column#14)->Column#10, funcs:count(1)->Column#11",
|
||||
"└─Projection 10000.00 root cast(test.t.c, decimal(10,0) BINARY)->Column#14, Column#7->Column#15, Column#8->Column#16, gid->Column#17",
|
||||
" └─Expand 10000.00 root level-projection:[test.t.c, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.t.c, Column#7, <nil>->Column#8, 1->gid],[test.t.c, Column#7, Column#8, 3->gid]; schema: [test.t.c,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.t.c, test.t.a->Column#7, test.t.b->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": null
|
||||
},
|
||||
{
|
||||
"SQL": "explain format = 'brief' select count(a) from t group by a, b with rollup; -- 3. should keep the original col a",
|
||||
"Plan": [
|
||||
"HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:count(test.t.a)->Column#10",
|
||||
"└─Expand 10000.00 root level-projection:[test.t.a, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.t.a, Column#7, <nil>->Column#8, 1->gid],[test.t.a, Column#7, Column#8, 3->gid]; schema: [test.t.a,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.t.a, test.t.a->Column#7, test.t.b->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"Projection 8000.00 root grouping(gid)->Column#11",
|
||||
"└─HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:firstrow(gid)->gid",
|
||||
" └─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, 0->gid],[Column#7, <nil>->Column#8, 1->gid],[Column#7, Column#8, 3->gid]; schema: [Column#7,Column#8,gid]",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"Projection 8000.00 root grouping(gid)->Column#11",
|
||||
"└─HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:firstrow(gid)->gid",
|
||||
" └─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, 0->gid],[Column#7, <nil>->Column#8, 1->gid],[Column#7, Column#8, 3->gid]; schema: [Column#7,Column#8,gid]",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"Projection 8000.00 root Column#7->Column#11, grouping(gid)->Column#12",
|
||||
"└─HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:firstrow(Column#7)->Column#7, funcs:firstrow(gid)->gid",
|
||||
" └─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, 0->gid],[Column#7, <nil>->Column#8, 1->gid],[Column#7, Column#8, 3->gid]; schema: [Column#7,Column#8,gid]",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"Projection 8000.00 root Column#7->Column#11, grouping(gid)->Column#12",
|
||||
"└─HashAgg 8000.00 root group by:Column#7, Column#8, gid, funcs:firstrow(Column#7)->Column#7, funcs:firstrow(gid)->gid",
|
||||
" └─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, 0->gid],[Column#7, <nil>->Column#8, 1->gid],[Column#7, Column#8, 3->gid]; schema: [Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root plus(test.t.a, 1)->Column#7, test.t.b->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#10",
|
||||
"└─Sort 8000.00 root Column#7",
|
||||
" └─HashAgg 8000.00 root group by:Column#15, Column#16, Column#17, funcs:sum(Column#14)->Column#10, funcs:firstrow(Column#15)->Column#7",
|
||||
" └─Projection 10000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#14, Column#7->Column#15, Column#8->Column#16, gid->Column#17",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid],[test.sales.profit, Column#7, Column#8, 3->gid]; schema: [test.sales.profit,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, plus(test.sales.year, 2)->Column#7, plus(test.sales.year, test.sales.profit)->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7->Column#11, Column#10",
|
||||
"└─Sort 8000.00 root Column#7",
|
||||
" └─HashAgg 8000.00 root group by:Column#16, Column#17, Column#18, funcs:sum(Column#15)->Column#10, funcs:firstrow(Column#16)->Column#7",
|
||||
" └─Projection 10000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#15, Column#7->Column#16, Column#8->Column#17, gid->Column#18",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid],[test.sales.profit, Column#7, Column#8, 3->gid]; schema: [test.sales.profit,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, plus(test.sales.year, 2)->Column#7, plus(test.sales.year, test.sales.profit)->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#10",
|
||||
"└─Sort 6400.00 root Column#7, Column#10",
|
||||
" └─HashAgg 6400.00 root group by:Column#16, Column#17, Column#18, funcs:sum(Column#15)->Column#10, funcs:firstrow(Column#16)->Column#7",
|
||||
" └─Projection 8000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#15, Column#7->Column#16, Column#8->Column#17, gid->Column#18",
|
||||
" └─Selection 8000.00 root gt(Column#7, 2002)",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid],[test.sales.profit, Column#7, Column#8, 3->gid]; schema: [test.sales.profit,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, plus(test.sales.year, 2)->Column#7, plus(test.sales.year, test.sales.profit)->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#10",
|
||||
"└─Projection 6400.00 root Column#7, Column#10, grouping(gid)->Column#11",
|
||||
" └─HashAgg 6400.00 root group by:Column#19, Column#20, Column#21, funcs:sum(Column#18)->Column#10, funcs:firstrow(Column#19)->Column#7, funcs:firstrow(Column#20)->gid",
|
||||
" └─Projection 8000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#18, Column#7->Column#19, gid->Column#20, Column#8->Column#21",
|
||||
" └─Selection 8000.00 root gt(Column#7, 2002)",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid],[test.sales.profit, Column#7, Column#8, 3->gid]; schema: [test.sales.profit,Column#7,Column#8,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, plus(test.sales.year, 2)->Column#7, plus(test.sales.year, test.sales.profit)->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#8->Column#13, Column#9->Column#14, Column#11",
|
||||
"└─Projection 8000.00 root Column#11, Column#7, Column#8, Column#9, gid",
|
||||
" └─Sort 8000.00 root Column#21",
|
||||
" └─Projection 8000.00 root Column#11, Column#7, Column#8, Column#9, gid, grouping(gid)->Column#21",
|
||||
" └─HashAgg 8000.00 root group by:Column#17, Column#18, Column#19, Column#20, funcs:sum(Column#16)->Column#11, funcs:firstrow(Column#17)->Column#7, funcs:firstrow(Column#18)->Column#8, funcs:firstrow(Column#19)->Column#9, funcs:firstrow(Column#20)->gid",
|
||||
" └─Projection 10000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#16, Column#7->Column#17, Column#8->Column#18, Column#9->Column#19, gid->Column#20",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, <nil>->Column#9, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, <nil>->Column#9, 1->gid],[test.sales.profit, Column#7, Column#8, <nil>->Column#9, 3->gid],[test.sales.profit, Column#7, Column#8, Column#9, 7->gid]; schema: [test.sales.profit,Column#7,Column#8,Column#9,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, test.sales.year->Column#7, test.sales.country->Column#8, test.sales.product->Column#9",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#8->Column#13, Column#11",
|
||||
"└─Projection 8000.00 root Column#11, Column#7, Column#8, gid",
|
||||
" └─Sort 8000.00 root Column#20",
|
||||
" └─Projection 8000.00 root Column#11, Column#7, Column#8, gid, grouping(gid)->Column#20",
|
||||
" └─HashAgg 8000.00 root group by:Column#16, Column#16, Column#17, Column#18, Column#19, funcs:sum(Column#15)->Column#11, funcs:firstrow(Column#16)->Column#7, funcs:firstrow(Column#17)->Column#8, funcs:firstrow(Column#18)->gid",
|
||||
" └─Projection 10000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#15, Column#7->Column#16, Column#8->Column#17, gid->Column#18, gpos->Column#19",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid, 0->gpos],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid, 1->gpos],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid, 2->gpos],[test.sales.profit, Column#7, Column#8, 3->gid, 3->gpos]; schema: [test.sales.profit,Column#7,Column#8,gid,gpos]",
|
||||
" └─Projection 10000.00 root test.sales.profit, test.sales.country->Column#7, test.sales.product->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#8->Column#13, Column#9->Column#14, Column#11",
|
||||
"└─Projection 6400.00 root Column#11, Column#7, Column#8, Column#9, gid",
|
||||
" └─Sort 6400.00 root Column#21",
|
||||
" └─Projection 6400.00 root Column#11, Column#7, Column#8, Column#9, gid, grouping(gid)->Column#21",
|
||||
" └─HashAgg 6400.00 root group by:Column#17, Column#18, Column#19, Column#20, funcs:sum(Column#16)->Column#11, funcs:firstrow(Column#17)->Column#7, funcs:firstrow(Column#18)->Column#8, funcs:firstrow(Column#19)->Column#9, funcs:firstrow(Column#20)->gid",
|
||||
" └─Projection 8000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#16, Column#7->Column#17, Column#8->Column#18, Column#9->Column#19, gid->Column#20",
|
||||
" └─Selection 8000.00 root gt(grouping(gid), 0)",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, <nil>->Column#9, 0->gid],[test.sales.profit, Column#7, <nil>->Column#8, <nil>->Column#9, 1->gid],[test.sales.profit, Column#7, Column#8, <nil>->Column#9, 3->gid],[test.sales.profit, Column#7, Column#8, Column#9, 7->gid]; schema: [test.sales.profit,Column#7,Column#8,Column#9,gid]",
|
||||
" └─Projection 10000.00 root test.sales.profit, test.sales.year->Column#7, test.sales.country->Column#8, test.sales.product->Column#9",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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#7, Column#8->Column#13, Column#11",
|
||||
"└─Projection 6400.00 root Column#11, Column#7, Column#8, gid",
|
||||
" └─Sort 6400.00 root Column#20",
|
||||
" └─Projection 6400.00 root Column#11, Column#7, Column#8, gid, grouping(gid)->Column#20",
|
||||
" └─HashAgg 6400.00 root group by:Column#16, Column#16, Column#17, Column#18, Column#19, funcs:sum(Column#15)->Column#11, funcs:firstrow(Column#16)->Column#7, funcs:firstrow(Column#17)->Column#8, funcs:firstrow(Column#18)->gid",
|
||||
" └─Projection 8000.00 root cast(test.sales.profit, decimal(10,0) BINARY)->Column#15, Column#7->Column#16, Column#8->Column#17, gid->Column#18, gpos->Column#19",
|
||||
" └─Selection 8000.00 root gt(grouping(gid), 0)",
|
||||
" └─Expand 10000.00 root level-projection:[test.sales.profit, <nil>->Column#7, <nil>->Column#8, 0->gid, 0->gpos],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid, 1->gpos],[test.sales.profit, Column#7, <nil>->Column#8, 1->gid, 2->gpos],[test.sales.profit, Column#7, Column#8, 3->gid, 3->gpos]; schema: [test.sales.profit,Column#7,Column#8,gid,gpos]",
|
||||
" └─Projection 10000.00 root test.sales.profit, test.sales.country->Column#7, test.sales.product->Column#8",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": 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": [
|
||||
"Projection 6400.00 root Column#7->Column#12, Column#8->Column#13, Column#9->Column#14, grouping(gid)->Column#15",
|
||||
"└─HashAgg 6400.00 root group by:Column#7, Column#8, Column#9, gid, funcs:firstrow(Column#7)->Column#7, funcs:firstrow(Column#8)->Column#8, funcs:firstrow(Column#9)->Column#9, funcs:firstrow(gid)->gid",
|
||||
" └─Selection 8000.00 root ne(grouping(gid), 0)",
|
||||
" └─Expand 10000.00 root level-projection:[<nil>->Column#7, <nil>->Column#8, <nil>->Column#9, 0->gid],[Column#7, <nil>->Column#8, <nil>->Column#9, 1->gid],[Column#7, Column#8, <nil>->Column#9, 3->gid],[Column#7, Column#8, Column#9, 7->gid]; schema: [Column#7,Column#8,Column#9,gid]",
|
||||
" └─TableReader 10000.00 root data:TableFullScan",
|
||||
" └─TableFullScan 10000.00 cop[tikv] table:sales keep order:false, stats:pseudo"
|
||||
],
|
||||
"Warning": null
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
@ -2589,14 +2589,19 @@ func choosePartitionKeys(keys []*property.MPPPartitionColumn, matches []int) []*
|
||||
return newKeys
|
||||
}
|
||||
|
||||
// ExhaustPhysicalPlans enumerate all the possible physical plan for expand operator (currently only mpp case is supported)
|
||||
// ExhaustPhysicalPlans enumerate all the possible physical plan for expand operator.
|
||||
// The second boolean means whether we should resort to enforcer to satisfy prop requirement.
|
||||
// false means we should, while true means we should not.
|
||||
func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
|
||||
// under the mpp task type, if the sort item is not empty, refuse it, cause expanded data doesn't support any sort items.
|
||||
if !prop.IsSortItemEmpty() {
|
||||
// false, meaning we can add a sort enforcer.
|
||||
return nil, false, nil
|
||||
}
|
||||
// RootTaskType is the default one, meaning no option. (we can give them a mpp choice)
|
||||
// when TiDB Expand execution is introduced: we can deal with two kind of physical plans.
|
||||
// RootTaskType means expand should be run at TiDB node.
|
||||
// (RootTaskType is the default option, we can also generate a mpp candidate for it)
|
||||
// MPPTaskType means expand should be run at TiFlash node.
|
||||
if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.MppTaskType {
|
||||
return nil, true, nil
|
||||
}
|
||||
@ -2606,8 +2611,10 @@ func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([
|
||||
if prop.TaskTp == property.MppTaskType && prop.MPPPartitionTp != property.AnyType {
|
||||
return nil, true, nil
|
||||
}
|
||||
var physicalExpands []base.PhysicalPlan
|
||||
// for property.RootTaskType and property.MppTaskType with no partition option, we can give an MPP Expand.
|
||||
if p.SCtx().GetSessionVars().IsMPPAllowed() {
|
||||
canPushToTiFlash := p.CanPushToCop(kv.TiFlash)
|
||||
if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash {
|
||||
mppProp := prop.CloneEssentialFields()
|
||||
mppProp.TaskTp = property.MppTaskType
|
||||
expand := PhysicalExpand{
|
||||
@ -2616,10 +2623,29 @@ func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([
|
||||
ExtraGroupingColNames: p.ExtraGroupingColNames,
|
||||
}.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.QueryBlockOffset(), mppProp)
|
||||
expand.SetSchema(p.Schema())
|
||||
return []base.PhysicalPlan{expand}, true, nil
|
||||
physicalExpands = append(physicalExpands, expand)
|
||||
// when the MppTaskType is required, we can return the physical plan directly.
|
||||
if prop.TaskTp == property.MppTaskType {
|
||||
return physicalExpands, true, nil
|
||||
}
|
||||
}
|
||||
// if MPP switch is shutdown, nothing can be generated.
|
||||
return nil, true, nil
|
||||
// for property.RootTaskType, we can give a TiDB Expand.
|
||||
{
|
||||
taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopMultiReadTaskType, property.MppTaskType, property.RootTaskType}
|
||||
for _, taskType := range taskTypes {
|
||||
// require cop task type for children.F
|
||||
tidbProp := prop.CloneEssentialFields()
|
||||
tidbProp.TaskTp = taskType
|
||||
expand := PhysicalExpand{
|
||||
GroupingSets: p.RollupGroupingSets,
|
||||
LevelExprs: p.LevelExprs,
|
||||
ExtraGroupingColNames: p.ExtraGroupingColNames,
|
||||
}.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.QueryBlockOffset(), tidbProp)
|
||||
expand.SetSchema(p.Schema())
|
||||
physicalExpands = append(physicalExpands, expand)
|
||||
}
|
||||
}
|
||||
return physicalExpands, true, nil
|
||||
}
|
||||
|
||||
func exhaustPhysicalPlans4Projection(p *LogicalProjection, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
|
||||
|
||||
@ -1011,13 +1011,21 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task {
|
||||
// Attach2Task implements the PhysicalPlan interface.
|
||||
func (p *PhysicalExpand) Attach2Task(tasks ...base.Task) base.Task {
|
||||
t := tasks[0].Copy()
|
||||
// current expand can only be run in MPP TiFlash mode.
|
||||
// current expand can only be run in MPP TiFlash mode or Root Tidb mode.
|
||||
// if expr inside could not be pushed down to tiFlash, it will error in converting to pb side.
|
||||
if mpp, ok := t.(*MppTask); ok {
|
||||
p.SetChildren(mpp.p)
|
||||
mpp.p = p
|
||||
return mpp
|
||||
}
|
||||
return base.InvalidTask
|
||||
// For root task
|
||||
// since expand should be in root side accordingly, convert to root task now.
|
||||
root := t.ConvertToRootTask(p.SCtx())
|
||||
t = attachPlan2Task(p, root)
|
||||
if root, ok := tasks[0].(*RootTask); ok && root.IsEmpty() {
|
||||
t.(*RootTask).SetEmpty(true)
|
||||
}
|
||||
return t
|
||||
}
|
||||
|
||||
// Attach2Task implements PhysicalPlan interface.
|
||||
|
||||
260
tests/integrationtest/r/executor/expand.result
Normal file
260
tests/integrationtest/r/executor/expand.result
Normal file
@ -0,0 +1,260 @@
|
||||
CREATE TABLE t1(
|
||||
product VARCHAR(32),
|
||||
country_id INTEGER NOT NULL,
|
||||
year INTEGER,
|
||||
profit INTEGER);
|
||||
INSERT INTO t1 VALUES ( 'Computer', 2,2000, 1200),
|
||||
( 'TV', 1, 1999, 150),
|
||||
( 'Calculator', 1, 1999,50),
|
||||
( 'Computer', 1, 1999,1500),
|
||||
( 'Computer', 1, 2000,1500),
|
||||
( 'TV', 1, 2000, 150),
|
||||
( 'TV', 2, 2000, 100),
|
||||
( 'TV', 2, 2000, 100),
|
||||
( 'Calculator', 1, 2000,75),
|
||||
( 'Calculator', 2, 2000,75),
|
||||
( 'TV', 1, 1999, 100),
|
||||
( 'Computer', 1, 1999,1200),
|
||||
( 'Computer', 2, 2000,1500),
|
||||
( 'Calculator', 2, 2000,75),
|
||||
( 'Phone', 3, 2003,10)
|
||||
;
|
||||
CREATE TABLE t2 (
|
||||
country_id INTEGER PRIMARY KEY,
|
||||
country CHAR(20) NOT NULL);
|
||||
INSERT INTO t2 VALUES (1, 'USA'),(2,'India'), (3,'Finland');
|
||||
# First simple rollups, with just grand total
|
||||
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY product;
|
||||
product SUM(profit)
|
||||
Calculator 275
|
||||
Computer 6900
|
||||
Phone 10
|
||||
TV 600
|
||||
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
product SUM(profit)
|
||||
NULL 7785
|
||||
Calculator 275
|
||||
Computer 6900
|
||||
Phone 10
|
||||
TV 600
|
||||
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY 1 WITH ROLLUP;
|
||||
product SUM(profit)
|
||||
NULL 7785
|
||||
Calculator 275
|
||||
Computer 6900
|
||||
Phone 10
|
||||
TV 600
|
||||
|
||||
SELECT product, SUM(profit),AVG(profit) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
product SUM(profit) AVG(profit)
|
||||
NULL 7785 519.0000
|
||||
Calculator 275 68.7500
|
||||
Computer 6900 1380.0000
|
||||
Phone 10 10.0000
|
||||
TV 600 120.0000
|
||||
|
||||
# Sub totals
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year;
|
||||
product country_id year SUM(profit)
|
||||
Calculator 1 1999 50
|
||||
Calculator 1 2000 75
|
||||
Calculator 2 2000 150
|
||||
Computer 1 1999 2700
|
||||
Computer 1 2000 1500
|
||||
Computer 2 2000 2700
|
||||
Phone 3 2003 10
|
||||
TV 1 1999 250
|
||||
TV 1 2000 150
|
||||
TV 2 2000 200
|
||||
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP;
|
||||
product country_id year SUM(profit)
|
||||
NULL NULL NULL 7785
|
||||
Calculator NULL NULL 275
|
||||
Calculator 1 NULL 125
|
||||
Calculator 1 1999 50
|
||||
Calculator 1 2000 75
|
||||
Calculator 2 NULL 150
|
||||
Calculator 2 2000 150
|
||||
Computer NULL NULL 6900
|
||||
Computer 1 NULL 4200
|
||||
Computer 1 1999 2700
|
||||
Computer 1 2000 1500
|
||||
Computer 2 NULL 2700
|
||||
Computer 2 2000 2700
|
||||
Phone NULL NULL 10
|
||||
Phone 3 NULL 10
|
||||
Phone 3 2003 10
|
||||
TV NULL NULL 600
|
||||
TV 1 NULL 400
|
||||
TV 1 1999 250
|
||||
TV 1 2000 150
|
||||
TV 2 NULL 200
|
||||
TV 2 2000 200
|
||||
|
||||
ANALYZE TABLE t1;
|
||||
|
||||
EXPLAIN FORMAT='brief' SELECT product, country_id , year, SUM(profit)
|
||||
FROM t1 GROUP BY product, country_id, year WITH ROLLUP;
|
||||
id estRows task access object operator info
|
||||
Projection 8000.00 root Column#6->Column#11, Column#7->Column#12, Column#8->Column#13, Column#10
|
||||
└─HashAgg 8000.00 root group by:Column#16, Column#17, Column#18, Column#19, funcs:sum(Column#15)->Column#10, funcs:firstrow(Column#16)->Column#6, funcs:firstrow(Column#17)->Column#7, funcs:firstrow(Column#18)->Column#8
|
||||
└─Projection 10000.00 root cast(executor__expand.t1.profit, decimal(10,0) BINARY)->Column#15, Column#6->Column#16, Column#7->Column#17, Column#8->Column#18, gid->Column#19
|
||||
└─Expand 10000.00 root level-projection:[executor__expand.t1.profit, <nil>->Column#6, <nil>->Column#7, <nil>->Column#8, 0->gid],[executor__expand.t1.profit, Column#6, <nil>->Column#7, <nil>->Column#8, 1->gid],[executor__expand.t1.profit, Column#6, Column#7, <nil>->Column#8, 3->gid],[executor__expand.t1.profit, Column#6, Column#7, Column#8, 7->gid]; schema: [executor__expand.t1.profit,Column#6,Column#7,Column#8,gid]
|
||||
└─Projection 10000.00 root executor__expand.t1.profit, executor__expand.t1.product->Column#6, executor__expand.t1.country_id->Column#7, executor__expand.t1.year->Column#8
|
||||
└─TableReader 10000.00 root data:TableFullScan
|
||||
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
|
||||
|
||||
SELECT product, country_id , SUM(profit) FROM t1
|
||||
GROUP BY product, country_id WITH ROLLUP
|
||||
ORDER BY product DESC, country_id;
|
||||
product country_id SUM(profit)
|
||||
TV NULL 600
|
||||
TV 1 400
|
||||
TV 2 200
|
||||
Phone NULL 10
|
||||
Phone 3 10
|
||||
Computer NULL 6900
|
||||
Computer 1 4200
|
||||
Computer 2 2700
|
||||
Calculator NULL 275
|
||||
Calculator 1 125
|
||||
Calculator 2 150
|
||||
NULL NULL 7785
|
||||
|
||||
# limit
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP
|
||||
ORDER BY product, country_id, year LIMIT 5;
|
||||
product country_id year SUM(profit)
|
||||
NULL NULL NULL 7785
|
||||
Calculator NULL NULL 275
|
||||
Calculator 1 NULL 125
|
||||
Calculator 1 1999 50
|
||||
Calculator 1 2000 75
|
||||
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP
|
||||
ORDER BY product, country_id, year limit 3,3;
|
||||
product country_id year SUM(profit)
|
||||
Calculator 1 1999 50
|
||||
Calculator 1 2000 75
|
||||
Calculator 2 NULL 150
|
||||
|
||||
SELECT product, country_id, COUNT(*), COUNT(distinct year)
|
||||
FROM t1 GROUP BY product, country_id;
|
||||
product country_id COUNT(*) COUNT(distinct year)
|
||||
Calculator 1 2 2
|
||||
Calculator 2 2 1
|
||||
Computer 1 3 2
|
||||
Computer 2 2 1
|
||||
Phone 3 1 1
|
||||
TV 1 3 2
|
||||
TV 2 2 1
|
||||
|
||||
SELECT product, country_id, COUNT(*), COUNT(distinct year)
|
||||
FROM t1 GROUP BY product, country_id WITH ROLLUP;
|
||||
product country_id COUNT(*) COUNT(distinct year)
|
||||
NULL NULL 15 3
|
||||
Calculator NULL 4 2
|
||||
Calculator 1 2 2
|
||||
Calculator 2 2 1
|
||||
Computer NULL 5 2
|
||||
Computer 1 3 2
|
||||
Computer 2 2 1
|
||||
Phone NULL 1 1
|
||||
Phone 3 1 1
|
||||
TV NULL 5 2
|
||||
TV 1 3 2
|
||||
TV 2 2 1
|
||||
|
||||
# Test of having
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING country_id = 1;
|
||||
product country_id year SUM(profit)
|
||||
Calculator 1 NULL 125
|
||||
Calculator 1 1999 50
|
||||
Calculator 1 2000 75
|
||||
Computer 1 NULL 4200
|
||||
Computer 1 1999 2700
|
||||
Computer 1 2000 1500
|
||||
TV 1 NULL 400
|
||||
TV 1 1999 250
|
||||
TV 1 2000 150
|
||||
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING SUM(profit) > 200;
|
||||
product country_id year SUM(profit)
|
||||
NULL NULL NULL 7785
|
||||
Calculator NULL NULL 275
|
||||
Computer NULL NULL 6900
|
||||
Computer 1 NULL 4200
|
||||
Computer 1 1999 2700
|
||||
Computer 1 2000 1500
|
||||
Computer 2 NULL 2700
|
||||
Computer 2 2000 2700
|
||||
TV NULL NULL 600
|
||||
TV 1 NULL 400
|
||||
TV 1 1999 250
|
||||
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING SUM(profit) > 7000;
|
||||
product country_id year SUM(profit)
|
||||
NULL NULL NULL 7785
|
||||
|
||||
# Functions
|
||||
SELECT CONCAT(product,':',country_id) AS 'prod', CONCAT(":",year,":") AS 'year',
|
||||
1+1, SUM(profit)/COUNT(*) FROM t1 GROUP BY 1,2 WITH ROLLUP;
|
||||
prod year 1+1 SUM(profit)/COUNT(*)
|
||||
NULL NULL 2 519.0000
|
||||
Calculator:1 NULL 2 62.5000
|
||||
Calculator:1 :1999: 2 50.0000
|
||||
Calculator:1 :2000: 2 75.0000
|
||||
Calculator:2 NULL 2 75.0000
|
||||
Calculator:2 :2000: 2 75.0000
|
||||
Computer:1 NULL 2 1400.0000
|
||||
Computer:1 :1999: 2 1350.0000
|
||||
Computer:1 :2000: 2 1500.0000
|
||||
Computer:2 NULL 2 1350.0000
|
||||
Computer:2 :2000: 2 1350.0000
|
||||
Phone:3 NULL 2 10.0000
|
||||
Phone:3 :2003: 2 10.0000
|
||||
TV:1 NULL 2 133.3333
|
||||
TV:1 :1999: 2 125.0000
|
||||
TV:1 :2000: 2 150.0000
|
||||
TV:2 NULL 2 100.0000
|
||||
TV:2 :2000: 2 100.0000
|
||||
|
||||
SELECT product, SUM(profit)/COUNT(*) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
product SUM(profit)/COUNT(*)
|
||||
NULL 519.0000
|
||||
Calculator 68.7500
|
||||
Computer 1380.0000
|
||||
Phone 10.0000
|
||||
TV 120.0000
|
||||
|
||||
SELECT LEFT(product,4) AS prod, SUM(profit)/COUNT(*) FROM t1
|
||||
GROUP BY prod WITH ROLLUP;
|
||||
prod SUM(profit)/COUNT(*)
|
||||
NULL 519.0000
|
||||
Calc 68.7500
|
||||
Comp 1380.0000
|
||||
Phon 10.0000
|
||||
TV 120.0000
|
||||
|
||||
SELECT CONCAT(product,':',country_id), 1+1, SUM(profit)/COUNT(*) FROM t1
|
||||
GROUP BY CONCAT(product,':',country_id) WITH ROLLUP;
|
||||
CONCAT(product,':',country_id) 1+1 SUM(profit)/COUNT(*)
|
||||
NULL 2 519.0000
|
||||
Calculator:1 2 62.5000
|
||||
Calculator:2 2 75.0000
|
||||
Computer:1 2 1400.0000
|
||||
Computer:2 2 1350.0000
|
||||
Phone:3 2 10.0000
|
||||
TV:1 2 133.3333
|
||||
TV:2 2 100.0000
|
||||
112
tests/integrationtest/t/executor/expand.test
Normal file
112
tests/integrationtest/t/executor/expand.test
Normal file
@ -0,0 +1,112 @@
|
||||
CREATE TABLE t1(
|
||||
product VARCHAR(32),
|
||||
country_id INTEGER NOT NULL,
|
||||
year INTEGER,
|
||||
profit INTEGER);
|
||||
|
||||
INSERT INTO t1 VALUES ( 'Computer', 2,2000, 1200),
|
||||
( 'TV', 1, 1999, 150),
|
||||
( 'Calculator', 1, 1999,50),
|
||||
( 'Computer', 1, 1999,1500),
|
||||
( 'Computer', 1, 2000,1500),
|
||||
( 'TV', 1, 2000, 150),
|
||||
( 'TV', 2, 2000, 100),
|
||||
( 'TV', 2, 2000, 100),
|
||||
( 'Calculator', 1, 2000,75),
|
||||
( 'Calculator', 2, 2000,75),
|
||||
( 'TV', 1, 1999, 100),
|
||||
( 'Computer', 1, 1999,1200),
|
||||
( 'Computer', 2, 2000,1500),
|
||||
( 'Calculator', 2, 2000,75),
|
||||
( 'Phone', 3, 2003,10)
|
||||
;
|
||||
|
||||
CREATE TABLE t2 (
|
||||
country_id INTEGER PRIMARY KEY,
|
||||
country CHAR(20) NOT NULL);
|
||||
|
||||
INSERT INTO t2 VALUES (1, 'USA'),(2,'India'), (3,'Finland');
|
||||
|
||||
--echo # First simple rollups, with just grand total
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY product;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, SUM(profit) FROM t1 GROUP BY 1 WITH ROLLUP;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, SUM(profit),AVG(profit) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
|
||||
--echo
|
||||
--echo # Sub totals
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP;
|
||||
--echo
|
||||
ANALYZE TABLE t1;
|
||||
--echo
|
||||
EXPLAIN FORMAT='brief' SELECT product, country_id , year, SUM(profit)
|
||||
FROM t1 GROUP BY product, country_id, year WITH ROLLUP;
|
||||
--echo
|
||||
SELECT product, country_id , SUM(profit) FROM t1
|
||||
GROUP BY product, country_id WITH ROLLUP
|
||||
ORDER BY product DESC, country_id;
|
||||
|
||||
--echo
|
||||
--echo # limit
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP
|
||||
ORDER BY product, country_id, year LIMIT 5;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP
|
||||
ORDER BY product, country_id, year limit 3,3;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id, COUNT(*), COUNT(distinct year)
|
||||
FROM t1 GROUP BY product, country_id;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id, COUNT(*), COUNT(distinct year)
|
||||
FROM t1 GROUP BY product, country_id WITH ROLLUP;
|
||||
|
||||
--echo
|
||||
--echo # Test of having
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING country_id = 1;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING SUM(profit) > 200;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, country_id , year, SUM(profit) FROM t1
|
||||
GROUP BY product, country_id, year WITH ROLLUP HAVING SUM(profit) > 7000;
|
||||
|
||||
--echo
|
||||
--echo # Functions
|
||||
--sorted_result
|
||||
SELECT CONCAT(product,':',country_id) AS 'prod', CONCAT(":",year,":") AS 'year',
|
||||
1+1, SUM(profit)/COUNT(*) FROM t1 GROUP BY 1,2 WITH ROLLUP;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT product, SUM(profit)/COUNT(*) FROM t1 GROUP BY product WITH ROLLUP;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT LEFT(product,4) AS prod, SUM(profit)/COUNT(*) FROM t1
|
||||
GROUP BY prod WITH ROLLUP;
|
||||
--echo
|
||||
--sorted_result
|
||||
SELECT CONCAT(product,':',country_id), 1+1, SUM(profit)/COUNT(*) FROM t1
|
||||
GROUP BY CONCAT(product,':',country_id) WITH ROLLUP;
|
||||
Reference in New Issue
Block a user