Skip to content

Commit

Permalink
executor: Support spill of parallel hash aggregate (#49074)
Browse files Browse the repository at this point in the history
ref #47733
  • Loading branch information
xzhangxian1008 authored Feb 4, 2024
1 parent ba77807 commit 402c52e
Show file tree
Hide file tree
Showing 14 changed files with 1,488 additions and 213 deletions.
28 changes: 27 additions & 1 deletion pkg/executor/aggregate/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "aggregate",
Expand Down Expand Up @@ -40,3 +40,29 @@ go_library(
"@org_uber_go_zap//:zap",
],
)

go_test(
name = "aggregate_test",
timeout = "short",
srcs = ["agg_spill_test.go"],
flaky = True,
shard_count = 3,
deps = [
":aggregate",
"//pkg/executor/aggfuncs",
"//pkg/executor/internal/exec",
"//pkg/executor/internal/testutil",
"//pkg/expression",
"//pkg/expression/aggregation",
"//pkg/parser/ast",
"//pkg/parser/mysql",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/types",
"//pkg/util/chunk",
"//pkg/util/memory",
"//pkg/util/mock",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
],
)
42 changes: 9 additions & 33 deletions pkg/executor/aggregate/agg_hash_base_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,8 @@
package aggregate

import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/executor/aggfuncs"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/memory"
)

Expand Down Expand Up @@ -49,40 +46,19 @@ func newBaseHashAggWorker(ctx sessionctx.Context, finishCh <-chan struct{}, aggF
return baseWorker
}

func (w *baseHashAggWorker) getPartialResult(_ *stmtctx.StatementContext, groupKey [][]byte, mapper aggfuncs.AggPartialResultMapper) [][]aggfuncs.PartialResult {
n := len(groupKey)
partialResults := make([][]aggfuncs.PartialResult, n)
allMemDelta := int64(0)
partialResultSize := w.getPartialResultSliceLenConsiderByteAlign()
for i := 0; i < n; i++ {
var ok bool
if partialResults[i], ok = mapper[string(groupKey[i])]; ok {
continue
}
partialResults[i] = make([]aggfuncs.PartialResult, partialResultSize)
for j, af := range w.aggFuncs {
partialResult, memDelta := af.AllocPartialResult()
partialResults[i][j] = partialResult
allMemDelta += memDelta // the memory usage of PartialResult
}
allMemDelta += int64(partialResultSize * 8)
// Map will expand when count > bucketNum * loadFactor. The memory usage will double.
if len(mapper)+1 > (1<<w.BInMap)*hack.LoadFactorNum/hack.LoadFactorDen {
w.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice * (1 << w.BInMap))
w.BInMap++
}
mapper[string(groupKey[i])] = partialResults[i]
allMemDelta += int64(len(groupKey[i]))
}
failpoint.Inject("ConsumeRandomPanic", nil)
w.memTracker.Consume(allMemDelta)
return partialResults
}

func (w *baseHashAggWorker) getPartialResultSliceLenConsiderByteAlign() int {
length := len(w.aggFuncs)
if length == 1 {
return 1
}
return length + length&1
}

func (w *baseHashAggWorker) checkFinishChClosed() bool {
select {
case <-w.finishCh:
return true
default:
}
return false
}
Loading

0 comments on commit 402c52e

Please sign in to comment.