Files
tidb/pkg/executor/aggregate/agg_hash_final_worker.go

193 lines
5.1 KiB
Go

// 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 aggregate
import (
"sync"
"time"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/executor/aggfuncs"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/set"
"go.uber.org/zap"
)
// AfFinalResult indicates aggregation functions final result.
type AfFinalResult struct {
chk *chunk.Chunk
err error
giveBackCh chan *chunk.Chunk
}
// HashAggFinalWorker indicates the final workers of parallel hash agg execution,
// the number of the worker can be set by `tidb_hashagg_final_concurrency`.
type HashAggFinalWorker struct {
baseHashAggWorker
rowBuffer []types.Datum
mutableRow chunk.MutRow
partialResultMap aggfuncs.AggPartialResultMapper
BInMap int
isFirstInput bool
groupSet set.StringSetWithMemoryUsage
inputCh chan *aggfuncs.AggPartialResultMapper
outputCh chan *AfFinalResult
finalResultHolderCh chan *chunk.Chunk
groupKeys [][]byte
}
func (w *HashAggFinalWorker) getPartialInput() (input *aggfuncs.AggPartialResultMapper, ok bool) {
select {
case <-w.finishCh:
return nil, false
case input, ok = <-w.inputCh:
if !ok {
return nil, false
}
}
return
}
func (w *HashAggFinalWorker) initBInMap() {
w.BInMap = 0
mapLen := len(w.partialResultMap)
for mapLen > (1<<w.BInMap)*hack.LoadFactorNum/hack.LoadFactorDen {
w.BInMap++
}
}
func (w *HashAggFinalWorker) consumeIntermData(sctx sessionctx.Context) (err error) {
for {
waitStart := time.Now()
input, ok := w.getPartialInput()
if w.stats != nil {
w.stats.WaitTime += int64(time.Since(waitStart))
}
if !ok {
return nil
}
// As the w.partialResultMap is empty when we get the first input.
// So it's better to directly assign the input to w.partialResultMap
if w.isFirstInput {
w.isFirstInput = false
w.partialResultMap = *input
w.initBInMap()
continue
}
failpoint.Inject("ConsumeRandomPanic", nil)
execStart := time.Now()
allMemDelta := int64(0)
for key, value := range *input {
dstVal, ok := w.partialResultMap[key]
if !ok {
// Map will expand when count > bucketNum * loadFactor. The memory usage will double.
if len(w.partialResultMap)+1 > (1<<w.BInMap)*hack.LoadFactorNum/hack.LoadFactorDen {
w.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice * (1 << w.BInMap))
w.BInMap++
}
w.partialResultMap[key] = value
continue
}
for j, af := range w.aggFuncs {
memDelta, err := af.MergePartialResult(sctx, value[j], dstVal[j])
if err != nil {
return err
}
allMemDelta += memDelta
}
}
w.memTracker.Consume(allMemDelta)
if w.stats != nil {
w.stats.ExecTime += int64(time.Since(execStart))
w.stats.TaskNum++
}
}
}
func (w *HashAggFinalWorker) loadFinalResult(sctx sessionctx.Context) {
waitStart := time.Now()
result, finished := w.receiveFinalResultHolder()
if w.stats != nil {
w.stats.WaitTime += int64(time.Since(waitStart))
}
if finished {
return
}
failpoint.Inject("ConsumeRandomPanic", nil)
execStart := time.Now()
for _, results := range w.partialResultMap {
for j, af := range w.aggFuncs {
if err := af.AppendFinalResult2Chunk(sctx, results[j], result); err != nil {
logutil.BgLogger().Error("HashAggFinalWorker failed to append final result to Chunk", zap.Error(err))
}
}
if len(w.aggFuncs) == 0 {
result.SetNumVirtualRows(result.NumRows() + 1)
}
if result.IsFull() {
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
result, finished = w.receiveFinalResultHolder()
if finished {
return
}
}
}
w.outputCh <- &AfFinalResult{chk: result, giveBackCh: w.finalResultHolderCh}
if w.stats != nil {
w.stats.ExecTime += int64(time.Since(execStart))
}
}
func (w *HashAggFinalWorker) receiveFinalResultHolder() (*chunk.Chunk, bool) {
select {
case <-w.finishCh:
return nil, true
case result, ok := <-w.finalResultHolderCh:
return result, !ok
}
}
func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup) {
start := time.Now()
defer func() {
if r := recover(); r != nil {
recoveryHashAgg(w.outputCh, r)
}
if w.stats != nil {
w.stats.WorkerTime += int64(time.Since(start))
}
waitGroup.Done()
}()
if err := w.consumeIntermData(ctx); err != nil {
w.outputCh <- &AfFinalResult{err: err}
}
w.loadFinalResult(ctx)
}