// 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 ( "math/rand" "sync" "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor/aggfuncs" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/logutil" "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 partialResultMap aggfuncs.AggPartialResultMapper BInMap int inputCh chan *aggfuncs.AggPartialResultMapper outputCh chan *AfFinalResult finalResultHolderCh chan *chunk.Chunk spillHelper *parallelHashAggSpillHelper restoredAggResultMapperMem int64 } func (w *HashAggFinalWorker) getInputFromDisk(sctx sessionctx.Context) (ret aggfuncs.AggPartialResultMapper, restoredMem int64, err error) { ret, restoredMem, err = w.spillHelper.restoreOnePartition(sctx) w.intestDuringFinalWorkerRun(&err) return ret, restoredMem, err } func (w *HashAggFinalWorker) getPartialInput() (input *aggfuncs.AggPartialResultMapper, ok bool) { waitStart := time.Now() defer updateWaitTime(w.stats, waitStart) 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< (1<