Skip to content

Commit 69b3885

Browse files
committed
address comments
Signed-off-by: Wenqi Mou <[email protected]>
1 parent 168bc8e commit 69b3885

File tree

7 files changed

+260
-92
lines changed

7 files changed

+260
-92
lines changed

pkg/statistics/BUILD.bazel

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ go_library(
1818
"row_sampler.go",
1919
"sample.go",
2020
"scalar.go",
21-
"sort_utils.go",
2221
"table.go",
2322
],
2423
importpath = "github.com/pingcap/tidb/pkg/statistics",
@@ -48,6 +47,7 @@ go_library(
4847
"//pkg/util/context",
4948
"//pkg/util/dbterror",
5049
"//pkg/util/fastrand",
50+
"//pkg/util/generic",
5151
"//pkg/util/hack",
5252
"//pkg/util/intest",
5353
"//pkg/util/logutil",
@@ -78,7 +78,6 @@ go_test(
7878
"main_test.go",
7979
"sample_test.go",
8080
"scalar_test.go",
81-
"sort_utils_test.go",
8281
"statistics_test.go",
8382
"table_test.go",
8483
],

pkg/statistics/builder.go

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import (
2525
"github.com/pingcap/tidb/pkg/types"
2626
"github.com/pingcap/tidb/pkg/util/codec"
2727
"github.com/pingcap/tidb/pkg/util/collate"
28+
"github.com/pingcap/tidb/pkg/util/generic"
2829
"github.com/pingcap/tidb/pkg/util/memory"
2930
)
3031

@@ -84,11 +85,11 @@ func (s *SequentialRangeChecker) IsIndexInTopNRange(idx int64) bool {
8485
return false
8586
}
8687

87-
// processTopNValueHeap handles the logic for a complete TopN value count using existing heap with range tracking.
88-
func processTopNValueHeap(topNHeap *TopNHeap[TopNWithRange], encoded []byte, curCnt float64,
88+
// processTopNValue handles the logic for a complete TopN value count using existing priority queue with range tracking.
89+
func processTopNValue(topNPriorityQueue *generic.PriorityQueue[TopNWithRange], encoded []byte, curCnt float64,
8990
startIdx, endIdx int64, numTopN int, allowPruning bool, sampleFactor float64) {
9091
// case 1: prune values with count of 1 under certain conditions
91-
if curCnt == 1 && allowPruning && (topNHeap.Len() >= (numTopN/topNPruningThreshold) || sampleFactor > 1) {
92+
if curCnt == 1 && allowPruning && (topNPriorityQueue.Len() >= (numTopN/topNPruningThreshold) || sampleFactor > 1) {
9293
return
9394
}
9495

@@ -98,7 +99,7 @@ func processTopNValueHeap(topNHeap *TopNHeap[TopNWithRange], encoded []byte, cur
9899
startIdx: startIdx,
99100
endIdx: endIdx,
100101
}
101-
topNHeap.Add(newItem)
102+
topNPriorityQueue.Add(newItem)
102103
}
103104

104105
// SortedBuilder is used to build histograms for PK and index.
@@ -430,7 +431,7 @@ func BuildHistAndTopN(
430431

431432
// Step1: collect topn from samples using heap and track their index ranges
432433
// use heap for efficient TopN maintenance - O(log N) insertions
433-
topNHeap := NewTopNHeap(numTopN, func(a, b TopNWithRange) int {
434+
topNHeap := generic.NewPriorityQueue(numTopN, func(a, b TopNWithRange) int {
434435
if a.Count < b.Count {
435436
return -1 // min-heap: smaller counts at root
436437
} else if a.Count > b.Count {
@@ -472,7 +473,7 @@ func BuildHistAndTopN(
472473
// case 2, meet a different value: counting for the "current" is complete
473474
sampleNDV++
474475
// process the completed value using heap with range tracking
475-
processTopNValueHeap(topNHeap, cur, curCnt, curStartIdx, int64(i)-1, numTopN, allowPruning, sampleFactor)
476+
processTopNValue(topNHeap, cur, curCnt, curStartIdx, i-1, numTopN, allowPruning, sampleFactor)
476477

477478
cur, curCnt = sampleBytes, 1
478479
curStartIdx = i // new value group starts at current index
@@ -485,7 +486,7 @@ func BuildHistAndTopN(
485486

486487
// handle the counting for the last value
487488
if numTopN != 0 {
488-
processTopNValueHeap(topNHeap, cur, curCnt, curStartIdx, sampleNum-1, numTopN, allowPruning, sampleFactor)
489+
processTopNValue(topNHeap, cur, curCnt, curStartIdx, sampleNum-1, numTopN, allowPruning, sampleFactor)
489490
}
490491

491492
// convert heap to sorted slice

pkg/statistics/builder_test.go

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -130,3 +130,37 @@ func BenchmarkBuildHistAndTopNWithLowNDV(b *testing.B) {
130130
_, _, _ = BuildHistAndTopN(ctx, 256, 500, 0, collector, filedType, true, memoryTracker, false)
131131
}
132132
}
133+
134+
// SequentialRangeChecker tests
135+
func TestSequentialRangeChecker(t *testing.T) {
136+
ranges := []TopNWithRange{
137+
{TopNMeta: TopNMeta{Count: 10}, startIdx: 5, endIdx: 8}, // range [5,8]
138+
{TopNMeta: TopNMeta{Count: 15}, startIdx: 12, endIdx: 15}, // range [12,15]
139+
{TopNMeta: TopNMeta{Count: 8}, startIdx: 20, endIdx: 22}, // range [20,22]
140+
}
141+
142+
checker := NewSequentialRangeChecker(ranges)
143+
144+
// test basic functionality
145+
require.False(t, checker.IsIndexInTopNRange(4)) // before first range
146+
require.True(t, checker.IsIndexInTopNRange(6)) // in first range
147+
require.False(t, checker.IsIndexInTopNRange(10)) // between ranges
148+
require.True(t, checker.IsIndexInTopNRange(13)) // in second range
149+
require.True(t, checker.IsIndexInTopNRange(21)) // in third range
150+
require.False(t, checker.IsIndexInTopNRange(25)) // after all ranges
151+
152+
// test edge cases
153+
emptyChecker := NewSequentialRangeChecker([]TopNWithRange{})
154+
require.False(t, emptyChecker.IsIndexInTopNRange(0))
155+
156+
// test unsorted input - should be sorted internally
157+
unsortedRanges := []TopNWithRange{
158+
{TopNMeta: TopNMeta{Count: 8}, startIdx: 20, endIdx: 22}, // third
159+
{TopNMeta: TopNMeta{Count: 10}, startIdx: 5, endIdx: 8}, // first
160+
{TopNMeta: TopNMeta{Count: 15}, startIdx: 12, endIdx: 15}, // second
161+
}
162+
unsortedChecker := NewSequentialRangeChecker(unsortedRanges)
163+
require.True(t, unsortedChecker.IsIndexInTopNRange(6))
164+
require.True(t, unsortedChecker.IsIndexInTopNRange(13))
165+
require.True(t, unsortedChecker.IsIndexInTopNRange(21))
166+
}

pkg/statistics/sort_utils_test.go

Lines changed: 0 additions & 59 deletions
This file was deleted.

pkg/util/generic/BUILD.bazel

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2,18 +2,22 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
22

33
go_library(
44
name = "generic",
5-
srcs = ["sync_map.go"],
5+
srcs = [
6+
"priority_queue.go",
7+
"sync_map.go",
8+
],
69
importpath = "github.com/pingcap/tidb/pkg/util/generic",
710
visibility = ["//visibility:public"],
811
)
912

1013
go_test(
1114
name = "generic_test",
1215
timeout = "short",
13-
srcs = ["sync_map_test.go"],
14-
flaky = True,
15-
deps = [
16-
":generic",
17-
"@com_github_stretchr_testify//require",
16+
srcs = [
17+
"priority_queue_test.go",
18+
"sync_map_test.go",
1819
],
20+
embed = [":generic"],
21+
flaky = True,
22+
deps = ["@com_github_stretchr_testify//require"],
1923
)

pkg/statistics/sort_utils.go renamed to pkg/util/generic/priority_queue.go

Lines changed: 23 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -12,65 +12,70 @@
1212
// See the License for the specific language governing permissions and
1313
// limitations under the License.
1414

15-
package statistics
15+
package generic
1616

1717
import "container/heap"
1818

19-
// TopNHeap implements a min-heap for maintaining top N items efficiently.
19+
// PriorityQueue implements a min-heap for maintaining the best N items efficiently.
2020
// It keeps the N items with the highest values according to the comparison function.
2121
// The root of the heap is always the smallest item, making it easy to remove when adding better items.
22-
type TopNHeap[T any] struct {
22+
type PriorityQueue[T any] struct {
2323
cmpFunc func(T, T) int
2424
items []T
2525
maxSize int
2626
}
2727

28-
// NewTopNHeap creates a new TopN heap with the specified maximum size and comparison function.
29-
func NewTopNHeap[T any](maxSize int, cmpFunc func(T, T) int) *TopNHeap[T] {
30-
return &TopNHeap[T]{
28+
// NewPriorityQueue creates a new priority queue with the specified maximum size and comparison function.
29+
func NewPriorityQueue[T any](maxSize int, cmpFunc func(T, T) int) *PriorityQueue[T] {
30+
return &PriorityQueue[T]{
3131
items: make([]T, 0, maxSize),
3232
maxSize: maxSize,
3333
cmpFunc: cmpFunc,
3434
}
3535
}
3636

3737
// Len returns the number of items in the heap.
38-
func (h *TopNHeap[T]) Len() int { return len(h.items) }
38+
func (h *PriorityQueue[T]) Len() int { return len(h.items) }
3939

40-
// Less compares two items in the heap. For TopN, we use a min-heap.
41-
func (h *TopNHeap[T]) Less(i, j int) bool {
40+
// Less compares two items in the heap. We use a min-heap for efficient priority queue operations.
41+
func (h *PriorityQueue[T]) Less(i, j int) bool {
4242
return h.cmpFunc(h.items[i], h.items[j]) < 0
4343
}
4444

4545
// Swap swaps two items in the heap.
46-
func (h *TopNHeap[T]) Swap(i, j int) {
46+
func (h *PriorityQueue[T]) Swap(i, j int) {
4747
h.items[i], h.items[j] = h.items[j], h.items[i]
4848
}
4949

5050
// Push adds an item to the heap (used by container/heap).
51-
func (h *TopNHeap[T]) Push(x any) {
51+
func (h *PriorityQueue[T]) Push(x any) {
5252
h.items = append(h.items, x.(T))
5353
}
5454

5555
// Pop removes and returns the smallest item from the heap (used by container/heap).
56-
func (h *TopNHeap[T]) Pop() any {
56+
func (h *PriorityQueue[T]) Pop() any {
5757
old := h.items
5858
n := len(old)
5959
item := old[n-1]
6060
h.items = old[0 : n-1]
6161
return item
6262
}
6363

64-
// Add adds an item to the TopN heap. If the heap is full and the new item
64+
// Add adds an item to the priority queue. If the queue is full and the new item
6565
// is better than the worst item, it replaces the worst item.
66-
func (h *TopNHeap[T]) Add(item T) {
66+
func (h *PriorityQueue[T]) Add(item T) {
67+
// handle zero capacity case
68+
if h.maxSize <= 0 {
69+
return
70+
}
71+
6772
if len(h.items) < h.maxSize {
68-
// heap not full, just add the item
73+
// queue not full, just add the item
6974
heap.Push(h, item)
7075
return
7176
}
7277

73-
// heap is full, check if new item is better than the worst (root of min-heap)
78+
// queue is full, check if new item is better than the worst (root of min-heap)
7479
if h.cmpFunc(item, h.items[0]) > 0 {
7580
// new item is better, replace the worst
7681
h.items[0] = item
@@ -79,12 +84,12 @@ func (h *TopNHeap[T]) Add(item T) {
7984
}
8085

8186
// ToSortedSlice returns all items in the heap as a sorted slice (best to worst).
82-
func (h *TopNHeap[T]) ToSortedSlice() []T {
87+
func (h *PriorityQueue[T]) ToSortedSlice() []T {
8388
if len(h.items) == 0 {
8489
return nil
8590
}
8691

87-
// copy items to avoid modifying the original heap
92+
// copy items to avoid modifying the original queue
8893
result := make([]T, len(h.items))
8994
copy(result, h.items)
9095

0 commit comments

Comments
 (0)