Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: control Chunk size for StreamAgg&HashAgg (#9512) #10133

Merged
merged 5 commits into from
Apr 15, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
42 changes: 19 additions & 23 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,6 @@ type HashAggFinalWorker struct {
type AfFinalResult struct {
chk *chunk.Chunk
err error

giveBackCh chan *chunk.Chunk
}

// HashAggExec deals with all the aggregate functions.
Expand Down Expand Up @@ -152,6 +150,7 @@ type HashAggExec struct {

finishCh chan struct{}
finalOutputCh chan *AfFinalResult
finalInputCh chan *chunk.Chunk
partialOutputChs []chan *HashAggIntermData
inputCh chan *HashAggInput
partialInputChs []chan *chunk.Chunk
Expand Down Expand Up @@ -247,6 +246,7 @@ func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) {
partialConcurrency := sessionVars.HashAggPartialConcurrency
e.isChildReturnEmpty = true
e.finalOutputCh = make(chan *AfFinalResult, finalConcurrency)
e.finalInputCh = make(chan *chunk.Chunk, finalConcurrency)
e.inputCh = make(chan *HashAggInput, partialConcurrency)
e.finishCh = make(chan struct{}, 1)

Expand Down Expand Up @@ -291,11 +291,10 @@ func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) {
groupSet: set.NewStringSet(),
inputCh: e.partialOutputChs[i],
outputCh: e.finalOutputCh,
finalResultHolderCh: make(chan *chunk.Chunk, 1),
finalResultHolderCh: e.finalInputCh,
rowBuffer: make([]types.Datum, 0, e.Schema().Len()),
mutableRow: chunk.MutRowFromTypes(e.retTypes()),
}
e.finalWorkers[i].finalResultHolderCh <- e.newFirstChunk()
}
}

Expand Down Expand Up @@ -469,7 +468,6 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) {
if finished {
return
}
result.Reset()
for groupKey := range w.groupSet {
partialResults := w.getPartialResult(sctx.GetSessionVars().StmtCtx, []byte(groupKey), w.partialResultMap)
for i, af := range w.aggFuncs {
Expand All @@ -480,18 +478,15 @@ func (w *HashAggFinalWorker) getFinalResult(sctx sessionctx.Context) {
if len(w.aggFuncs) == 0 {
result.SetNumVirtualRows(result.NumRows() + 1)
}
if result.NumRows() == w.maxChunkSize {
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
if result.IsFull() {
w.outputCh <- &AfFinalResult{chk: result}
result, finished = w.receiveFinalResultHolder()
if finished {
return
}
result.Reset()
}
}
if result.NumRows() > 0 {
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
}
w.outputCh <- &AfFinalResult{chk: result}
}

func (w *HashAggFinalWorker) receiveFinalResultHolder() (*chunk.Chunk, bool) {
Expand Down Expand Up @@ -606,24 +601,25 @@ func (e *HashAggExec) parallelExec(ctx context.Context, chk *chunk.Chunk) error
e.prepare4ParallelExec(ctx)
e.prepared = true
}
for {

for !chk.IsFull() {
e.finalInputCh <- chk
result, ok := <-e.finalOutputCh
if !ok || result.err != nil || result.chk.NumRows() == 0 {
if result != nil {
return errors.Trace(result.err)
if !ok { // all finalWorkers exited
if chk.NumRows() > 0 { // but there are some data left
return nil
}
if e.isChildReturnEmpty && e.defaultVal != nil {
chk.Append(e.defaultVal, 0, 1)
}
e.isChildReturnEmpty = false
return nil
}
e.isChildReturnEmpty = false
chk.SwapColumns(result.chk)
// Put result.chk back to the corresponded final worker's finalResultHolderCh.
result.giveBackCh <- result.chk
if result.err != nil {
return result.err
}
if chk.NumRows() > 0 {
break
e.isChildReturnEmpty = false
}
}
return nil
Expand Down Expand Up @@ -657,11 +653,11 @@ func (e *HashAggExec) unparallelExec(ctx context.Context, chk *chunk.Chunk) erro
chk.SetNumVirtualRows(chk.NumRows() + 1)
}
for i, af := range e.PartialAggFuncs {
if err := (af.AppendFinalResult2Chunk(e.ctx, partialResults[i], chk)); err != nil {
if err := af.AppendFinalResult2Chunk(e.ctx, partialResults[i], chk); err != nil {
return err
}
}
if chk.NumRows() == e.maxChunkSize {
if chk.IsFull() {
e.cursor4GroupKey++
return nil
}
Expand Down Expand Up @@ -787,7 +783,7 @@ func (e *StreamAggExec) Next(ctx context.Context, chk *chunk.Chunk) error {
defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }()
}
chk.Reset()
for !e.executed && chk.NumRows() < e.maxChunkSize {
for !e.executed && !chk.IsFull() {
err := e.consumeOneGroup(ctx, chk)
if err != nil {
e.executed = true
Expand Down
168 changes: 168 additions & 0 deletions executor/executor_required_rows_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
Expand Down Expand Up @@ -589,3 +590,170 @@ func buildProjectionExec(ctx sessionctx.Context, exprs []expression.Expression,
evaluatorSuit: expression.NewEvaluatorSuite(exprs, false),
}
}

func buildHashAggExecutor(ctx sessionctx.Context, src Executor, schema *expression.Schema,
aggFuncs []*aggregation.AggFuncDesc, groupItems []expression.Expression) Executor {
plan := new(plannercore.PhysicalHashAgg)
plan.AggFuncs = aggFuncs
plan.GroupByItems = groupItems
plan.SetSchema(schema)
plan.Init(ctx, nil)
plan.SetChildren(nil)
b := newExecutorBuilder(ctx, nil)
exec := b.build(plan)
hashAgg := exec.(*HashAggExec)
hashAgg.children[0] = src
return exec
}

func buildStreamAggExecutor(ctx sessionctx.Context, src Executor, schema *expression.Schema,
aggFuncs []*aggregation.AggFuncDesc, groupItems []expression.Expression) Executor {
plan := new(plannercore.PhysicalStreamAgg)
plan.AggFuncs = aggFuncs
plan.GroupByItems = groupItems
plan.SetSchema(schema)
plan.Init(ctx, nil)
plan.SetChildren(nil)
b := newExecutorBuilder(ctx, nil)
exec := b.build(plan)
streamAgg := exec.(*StreamAggExec)
streamAgg.children[0] = src
return exec
}

func divGenerator(factor int) func(valType *types.FieldType) interface{} {
closureCountInt := 0
closureCountDouble := 0
return func(valType *types.FieldType) interface{} {
switch valType.Tp {
case mysql.TypeLong, mysql.TypeLonglong:
ret := int64(closureCountInt / factor)
closureCountInt++
return ret
case mysql.TypeDouble:
ret := float64(closureCountInt / factor)
closureCountDouble++
return ret
default:
panic("not implement")
}
}
}

func (s *testExecSuite) TestStreamAggRequiredRows(c *C) {
maxChunkSize := defaultCtx().GetSessionVars().MaxChunkSize
testCases := []struct {
totalRows int
aggFunc string
requiredRows []int
expectedRows []int
expectedRowsDS []int
gen func(valType *types.FieldType) interface{}
}{
{
totalRows: 1000000,
aggFunc: ast.AggFuncSum,
requiredRows: []int{1, 2, 3, 4, 5, 6, 7},
expectedRows: []int{1, 2, 3, 4, 5, 6, 7},
expectedRowsDS: []int{maxChunkSize},
gen: divGenerator(1),
},
{
totalRows: maxChunkSize * 3,
aggFunc: ast.AggFuncAvg,
requiredRows: []int{1, 3},
expectedRows: []int{1, 2},
expectedRowsDS: []int{maxChunkSize, maxChunkSize, maxChunkSize, 0},
gen: divGenerator(maxChunkSize),
},
{
totalRows: maxChunkSize*2 - 1,
aggFunc: ast.AggFuncMax,
requiredRows: []int{maxChunkSize/2 + 1},
expectedRows: []int{maxChunkSize/2 + 1},
expectedRowsDS: []int{maxChunkSize, maxChunkSize - 1},
gen: divGenerator(2),
},
}

for _, testCase := range testCases {
sctx := defaultCtx()
ctx := context.Background()
ds := newRequiredRowsDataSourceWithGenerator(sctx, testCase.totalRows, testCase.expectedRowsDS, testCase.gen)
childCols := ds.Schema().Columns
schema := expression.NewSchema(childCols...)
groupBy := []expression.Expression{childCols[1]}
aggFunc := aggregation.NewAggFuncDesc(sctx, testCase.aggFunc, []expression.Expression{childCols[0]}, true)
aggFuncs := []*aggregation.AggFuncDesc{aggFunc}
exec := buildStreamAggExecutor(sctx, ds, schema, aggFuncs, groupBy)
c.Assert(exec.Open(ctx), IsNil)
chk := exec.newFirstChunk()
for i := range testCase.requiredRows {
chk.SetRequiredRows(testCase.requiredRows[i], maxChunkSize)
c.Assert(exec.Next(ctx, chk), IsNil)
c.Assert(chk.NumRows(), Equals, testCase.expectedRows[i])
}
c.Assert(exec.Close(), IsNil)
c.Assert(ds.checkNumNextCalled(), IsNil)
}
}

func (s *testExecSuite) TestHashAggParallelRequiredRows(c *C) {
maxChunkSize := defaultCtx().GetSessionVars().MaxChunkSize
testCases := []struct {
totalRows int
aggFunc string
requiredRows []int
expectedRows []int
expectedRowsDS []int
gen func(valType *types.FieldType) interface{}
}{
{
totalRows: maxChunkSize,
aggFunc: ast.AggFuncSum,
requiredRows: []int{1, 2, 3, 4, 5, 6, 7},
expectedRows: []int{1, 2, 3, 4, 5, 6, 7},
expectedRowsDS: []int{maxChunkSize, 0},
gen: divGenerator(1),
},
{
totalRows: maxChunkSize * 3,
aggFunc: ast.AggFuncAvg,
requiredRows: []int{1, 3},
expectedRows: []int{1, 2},
expectedRowsDS: []int{maxChunkSize, maxChunkSize, maxChunkSize, 0},
gen: divGenerator(maxChunkSize),
},
{
totalRows: maxChunkSize * 3,
aggFunc: ast.AggFuncAvg,
requiredRows: []int{maxChunkSize, maxChunkSize},
expectedRows: []int{maxChunkSize, maxChunkSize / 2},
expectedRowsDS: []int{maxChunkSize, maxChunkSize, maxChunkSize, 0},
gen: divGenerator(2),
},
}

for _, hasDistinct := range []bool{false, true} {
for _, testCase := range testCases {
sctx := defaultCtx()
ctx := context.Background()
ds := newRequiredRowsDataSourceWithGenerator(sctx, testCase.totalRows, testCase.expectedRowsDS, testCase.gen)
childCols := ds.Schema().Columns
schema := expression.NewSchema(childCols...)
groupBy := []expression.Expression{childCols[1]}
aggFunc := aggregation.NewAggFuncDesc(sctx, testCase.aggFunc, []expression.Expression{childCols[0]}, hasDistinct)
aggFuncs := []*aggregation.AggFuncDesc{aggFunc}
exec := buildHashAggExecutor(sctx, ds, schema, aggFuncs, groupBy)
c.Assert(exec.Open(ctx), IsNil)
chk := exec.newFirstChunk()
for i := range testCase.requiredRows {
chk.SetRequiredRows(testCase.requiredRows[i], maxChunkSize)
c.Assert(exec.Next(ctx, chk), IsNil)
c.Assert(chk.NumRows(), Equals, testCase.expectedRows[i])
}
c.Assert(exec.Close(), IsNil)
c.Assert(ds.checkNumNextCalled(), IsNil)
}
}
}
2 changes: 1 addition & 1 deletion planner/core/initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,7 @@ func (p PhysicalMergeJoin) init(ctx sessionctx.Context, stats *property.StatsInf
return &p
}

func (base basePhysicalAgg) init(ctx sessionctx.Context, stats *property.StatsInfo) *basePhysicalAgg {
func (base basePhysicalAgg) Init(ctx sessionctx.Context, stats *property.StatsInfo) *basePhysicalAgg {
base.basePhysicalPlan = newBasePhysicalPlan(ctx, TypeHashAgg, &base)
base.stats = stats
return &base
Expand Down