Skip to content

Commit

Permalink
executor: fix data race in hash agg ut (#51342)
Browse files Browse the repository at this point in the history
close #51254
  • Loading branch information
xzhangxian1008 authored Feb 27, 2024
1 parent 961159c commit 218f3af
Showing 1 changed file with 9 additions and 7 deletions.
16 changes: 9 additions & 7 deletions pkg/executor/aggregate/agg_hash_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ type HashAggExec struct {
IsUnparallelExec bool
parallelExecValid bool
prepared atomic.Bool
executed bool
executed atomic.Bool

memTracker *memory.Tracker // track memory usage.
diskTracker *disk.Tracker
Expand Down Expand Up @@ -201,7 +201,7 @@ func (e *HashAggExec) Close() error {
channel.Clear(ch)
}
channel.Clear(e.finalOutputCh)
e.executed = false
e.executed.Store(false)
if e.memTracker != nil {
e.memTracker.ReplaceBytesUsed(0)
}
Expand Down Expand Up @@ -257,7 +257,8 @@ func (e *HashAggExec) initForUnparallelExec() {
e.memTracker.Consume(e.childResult.MemoryUsage())

e.offsetOfSpilledChks, e.numOfSpilledChks = 0, 0
e.executed, e.isChildDrained = false, false
e.executed.Store(false)
e.isChildDrained = false
e.dataInDisk = chunk.NewDataInDiskByChunks(exec.RetTypes(e.Children(0)))

e.tmpChkForSpill = exec.TryNewCacheChunk(e.Children(0))
Expand Down Expand Up @@ -398,6 +399,7 @@ func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) error {
e.initPartialWorkers(partialConcurrency, finalConcurrency, ctx)
e.initFinalWorkers(finalConcurrency)
e.parallelExecValid = true
e.executed.Store(false)
return nil
}

Expand Down Expand Up @@ -593,14 +595,14 @@ func (e *HashAggExec) parallelExec(ctx context.Context, chk *chunk.Chunk) error
}
})

if e.executed {
if e.executed.Load() {
return nil
}

for {
result, ok := <-e.finalOutputCh
if !ok {
e.executed = true
e.executed.Store(true)
if e.IsChildReturnEmpty && e.DefaultVal != nil {
chk.Append(e.DefaultVal, 0, 1)
}
Expand Down Expand Up @@ -646,7 +648,7 @@ func (e *HashAggExec) unparallelExec(ctx context.Context, chk *chunk.Chunk) erro
}
e.resetSpillMode()
}
if e.executed {
if e.executed.Load() {
return nil
}
if err := e.execute(ctx); err != nil {
Expand All @@ -671,7 +673,7 @@ func (e *HashAggExec) resetSpillMode() {
e.partialResultMap = make(aggfuncs.AggPartialResultMapper)
e.bInMap = 0
e.prepared.Store(false)
e.executed = e.numOfSpilledChks == e.dataInDisk.NumChunks() // No data is spilling again, all data have been processed.
e.executed.Store(e.numOfSpilledChks == e.dataInDisk.NumChunks()) // No data is spilling again, all data have been processed.
e.numOfSpilledChks = e.dataInDisk.NumChunks()
e.memTracker.ReplaceBytesUsed(setSize)
atomic.StoreUint32(&e.inSpillMode, 0)
Expand Down

0 comments on commit 218f3af

Please sign in to comment.