Skip to content

Commit 069631e

Browse files
authored
*: fix wrong result when to concurrency merge global stats (#48852) (#48864)
close #48713
1 parent 96f4883 commit 069631e

File tree

2 files changed

+51
-57
lines changed

2 files changed

+51
-57
lines changed

pkg/statistics/handle/globalstats/merge_worker.go

Lines changed: 26 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -44,8 +44,11 @@ type topnStatsMergeWorker struct {
4444
respCh chan<- *TopnStatsMergeResponse
4545
// the stats in the wrapper should only be read during the worker
4646
statsWrapper *StatsWrapper
47+
// Different TopN structures may hold the same value, we have to merge them.
48+
counter map[hack.MutableString]float64
4749
// shardMutex is used to protect `statsWrapper.AllHg`
4850
shardMutex []sync.Mutex
51+
mu sync.Mutex
4952
}
5053

5154
// NewTopnStatsMergeWorker returns topn merge worker
@@ -55,8 +58,9 @@ func NewTopnStatsMergeWorker(
5558
wrapper *StatsWrapper,
5659
killed *uint32) *topnStatsMergeWorker {
5760
worker := &topnStatsMergeWorker{
58-
taskCh: taskCh,
59-
respCh: respCh,
61+
taskCh: taskCh,
62+
respCh: respCh,
63+
counter: make(map[hack.MutableString]float64),
6064
}
6165
worker.statsWrapper = wrapper
6266
worker.shardMutex = make([]sync.Mutex, len(wrapper.AllHg))
@@ -80,33 +84,24 @@ func NewTopnStatsMergeTask(start, end int) *TopnStatsMergeTask {
8084

8185
// TopnStatsMergeResponse indicates topn merge worker response
8286
type TopnStatsMergeResponse struct {
83-
Err error
84-
TopN *statistics.TopN
85-
PopedTopn []statistics.TopNMeta
87+
Err error
8688
}
8789

8890
// Run runs topn merge like statistics.MergePartTopN2GlobalTopN
89-
func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool,
90-
n uint32,
91-
version int) {
91+
func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, version int) {
9292
for task := range worker.taskCh {
9393
start := task.start
9494
end := task.end
9595
checkTopNs := worker.statsWrapper.AllTopN[start:end]
9696
allTopNs := worker.statsWrapper.AllTopN
9797
allHists := worker.statsWrapper.AllHg
9898
resp := &TopnStatsMergeResponse{}
99-
if statistics.CheckEmptyTopNs(checkTopNs) {
100-
worker.respCh <- resp
101-
return
102-
}
99+
103100
partNum := len(allTopNs)
104-
// Different TopN structures may hold the same value, we have to merge them.
105-
counter := make(map[hack.MutableString]float64)
101+
106102
// datumMap is used to store the mapping from the string type to datum type.
107103
// The datum is used to find the value in the histogram.
108104
datumMap := statistics.NewDatumMapCache()
109-
110105
for i, topN := range checkTopNs {
111106
i = i + start
112107
if atomic.LoadUint32(worker.killed) == 1 {
@@ -119,12 +114,15 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool,
119114
}
120115
for _, val := range topN.TopN {
121116
encodedVal := hack.String(val.Encoded)
122-
_, exists := counter[encodedVal]
123-
counter[encodedVal] += float64(val.Count)
117+
worker.mu.Lock()
118+
_, exists := worker.counter[encodedVal]
119+
worker.counter[encodedVal] += float64(val.Count)
124120
if exists {
121+
worker.mu.Unlock()
125122
// We have already calculated the encodedVal from the histogram, so just continue to next topN value.
126123
continue
127124
}
125+
worker.mu.Unlock()
128126
// We need to check whether the value corresponding to encodedVal is contained in other partition-level stats.
129127
// 1. Check the topN first.
130128
// 2. If the topN doesn't contain the value corresponding to encodedVal. We should check the histogram.
@@ -148,31 +146,26 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool,
148146
}
149147
datum = d
150148
}
149+
worker.shardMutex[j].Lock()
151150
// Get the row count which the value is equal to the encodedVal from histogram.
152151
count, _ := allHists[j].EqualRowCount(nil, datum, isIndex)
153152
if count != 0 {
154-
counter[encodedVal] += count
155153
// Remove the value corresponding to encodedVal from the histogram.
156-
worker.shardMutex[j].Lock()
157154
worker.statsWrapper.AllHg[j].BinarySearchRemoveVal(statistics.TopNMeta{Encoded: datum.GetBytes(), Count: uint64(count)})
158-
worker.shardMutex[j].Unlock()
155+
}
156+
worker.shardMutex[j].Unlock()
157+
if count != 0 {
158+
worker.mu.Lock()
159+
worker.counter[encodedVal] += count
160+
worker.mu.Unlock()
159161
}
160162
}
161163
}
162164
}
163-
numTop := len(counter)
164-
if numTop == 0 {
165-
worker.respCh <- resp
166-
continue
167-
}
168-
sorted := make([]statistics.TopNMeta, 0, numTop)
169-
for value, cnt := range counter {
170-
data := hack.Slice(string(value))
171-
sorted = append(sorted, statistics.TopNMeta{Encoded: data, Count: uint64(cnt)})
172-
}
173-
globalTopN, leftTopN := statistics.GetMergedTopNFromSortedSlice(sorted, n)
174-
resp.TopN = globalTopN
175-
resp.PopedTopn = leftTopN
176165
worker.respCh <- resp
177166
}
178167
}
168+
169+
func (worker *topnStatsMergeWorker) Result() map[hack.MutableString]float64 {
170+
return worker.counter
171+
}

pkg/statistics/handle/globalstats/topn.go

Lines changed: 25 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,9 @@ import (
3030
func mergeGlobalStatsTopN(gp *gp.Pool, sc sessionctx.Context, wrapper *StatsWrapper,
3131
timeZone *time.Location, version int, n uint32, isIndex bool) (*statistics.TopN,
3232
[]statistics.TopNMeta, []*statistics.Histogram, error) {
33+
if statistics.CheckEmptyTopNs(wrapper.AllTopN) {
34+
return nil, nil, wrapper.AllHg, nil
35+
}
3336
mergeConcurrency := sc.GetSessionVars().AnalyzePartitionMergeConcurrency
3437
killed := &sc.GetSessionVars().Killed
3538
// use original method if concurrency equals 1 or for version1
@@ -69,12 +72,12 @@ func MergeGlobalStatsTopNByConcurrency(gp *gp.Pool, mergeConcurrency, mergeBatch
6972
taskNum := len(tasks)
7073
taskCh := make(chan *TopnStatsMergeTask, taskNum)
7174
respCh := make(chan *TopnStatsMergeResponse, taskNum)
75+
worker := NewTopnStatsMergeWorker(taskCh, respCh, wrapper, killed)
7276
for i := 0; i < mergeConcurrency; i++ {
73-
worker := NewTopnStatsMergeWorker(taskCh, respCh, wrapper, killed)
7477
wg.Add(1)
7578
gp.Go(func() {
7679
defer wg.Done()
77-
worker.Run(timeZone, isIndex, n, version)
80+
worker.Run(timeZone, isIndex, version)
7881
})
7982
}
8083
for _, task := range tasks {
@@ -83,8 +86,6 @@ func MergeGlobalStatsTopNByConcurrency(gp *gp.Pool, mergeConcurrency, mergeBatch
8386
close(taskCh)
8487
wg.Wait()
8588
close(respCh)
86-
resps := make([]*TopnStatsMergeResponse, 0)
87-
8889
// handle Error
8990
hasErr := false
9091
errMsg := make([]string, 0)
@@ -93,27 +94,21 @@ func MergeGlobalStatsTopNByConcurrency(gp *gp.Pool, mergeConcurrency, mergeBatch
9394
hasErr = true
9495
errMsg = append(errMsg, resp.Err.Error())
9596
}
96-
resps = append(resps, resp)
9797
}
9898
if hasErr {
9999
return nil, nil, nil, errors.New(strings.Join(errMsg, ","))
100100
}
101101

102102
// fetch the response from each worker and merge them into global topn stats
103-
sorted := make([]statistics.TopNMeta, 0, mergeConcurrency)
104-
leftTopn := make([]statistics.TopNMeta, 0)
105-
for _, resp := range resps {
106-
if resp.TopN != nil {
107-
sorted = append(sorted, resp.TopN.TopN...)
108-
}
109-
leftTopn = append(leftTopn, resp.PopedTopn...)
103+
counter := worker.Result()
104+
numTop := len(counter)
105+
sorted := make([]statistics.TopNMeta, 0, numTop)
106+
for value, cnt := range counter {
107+
data := hack.Slice(string(value))
108+
sorted = append(sorted, statistics.TopNMeta{Encoded: data, Count: uint64(cnt)})
110109
}
111-
112110
globalTopN, popedTopn := statistics.GetMergedTopNFromSortedSlice(sorted, n)
113-
114-
result := append(leftTopn, popedTopn...)
115-
statistics.SortTopnMeta(result)
116-
return globalTopN, result, wrapper.AllHg, nil
111+
return globalTopN, popedTopn, wrapper.AllHg, nil
117112
}
118113

119114
// MergePartTopN2GlobalTopN is used to merge the partition-level topN to global-level topN.
@@ -124,13 +119,19 @@ func MergeGlobalStatsTopNByConcurrency(gp *gp.Pool, mergeConcurrency, mergeBatch
124119
//
125120
// The output parameters:
126121
// 1. `*TopN` is the final global-level topN.
127-
// 2. `[]TopNMeta` is the left topN value from the partition-level TopNs, but is not placed to global-level TopN. We should put them back to histogram latter.
128-
// 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN.
129-
func MergePartTopN2GlobalTopN(loc *time.Location, version int, topNs []*statistics.TopN, n uint32, hists []*statistics.Histogram,
130-
isIndex bool, killed *uint32) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) {
131-
if statistics.CheckEmptyTopNs(topNs) {
132-
return nil, nil, hists, nil
133-
}
122+
// 2. `[]TopNMeta` is the left topN value from the partition-level TopNs,
123+
// but is not placed to global-level TopN. We should put them back to histogram latter.
124+
// 3. `[]*Histogram` are the partition-level histograms which
125+
// just delete some values when we merge the global-level topN.
126+
func MergePartTopN2GlobalTopN(
127+
loc *time.Location,
128+
version int,
129+
topNs []*statistics.TopN,
130+
n uint32,
131+
hists []*statistics.Histogram,
132+
isIndex bool,
133+
killed *uint32,
134+
) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) {
134135
partNum := len(topNs)
135136
// Different TopN structures may hold the same value, we have to merge them.
136137
counter := make(map[hack.MutableString]float64)

0 commit comments

Comments
 (0)