-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathsorttopk.go
384 lines (355 loc) · 12 KB
/
sorttopk.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
// Copyright 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package colexec
import (
"container/heap"
"context"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)
const (
topKVecIdx = 0
inputVecIdx = 1
)
// NewTopKSorter returns a new sort operator, which sorts its input on the
// columns given in orderingCols and returns the first K rows. The inputTypes
// must correspond 1-1 with the columns in the input operator. If matchLen is
// non-zero, then the input tuples must be sorted on first matchLen columns.
func NewTopKSorter(
allocator *colmem.Allocator,
input colexecop.Operator,
inputTypes []*types.T,
orderingCols []execinfrapb.Ordering_Column,
matchLen int,
k uint64,
maxOutputBatchMemSize int64,
) (colexecop.ResettableOperator, error) {
base := &topKSorter{
allocator: allocator,
OneInputNode: colexecop.NewOneInputNode(input),
inputTypes: inputTypes,
orderingCols: orderingCols,
k: k,
maxOutputBatchMemSize: maxOutputBatchMemSize,
}
partialOrderCols := make([]uint32, matchLen)
for i := range partialOrderCols {
partialOrderCols[i] = orderingCols[i].ColIdx
}
var err error
base.distincterInput = &colexecop.FeedOperator{}
base.distincter, base.distinctOutput, err = colexecbase.OrderedDistinctColsToOperators(
base.distincterInput, partialOrderCols, inputTypes, false, /* nullsAreDistinct */
)
if err != nil {
return base, err
}
return base, nil
}
var _ colexecop.BufferingInMemoryOperator = &topKSorter{}
var _ colexecop.Resetter = &topKSorter{}
// topKSortState represents the state of the sort operator.
type topKSortState int
const (
// topKSortSpooling is the initial state of the operator, where it spools
// its input.
topKSortSpooling topKSortState = iota
// topKSortEmitting is the second state of the operator, indicating that
// each call to Next will return another batch of the sorted data.
topKSortEmitting
// topKSortDone is the final state of the operator, where it always returns
// a zero batch.
topKSortDone
)
type topKSorter struct {
colexecop.OneInputNode
colexecop.InitHelper
allocator *colmem.Allocator
orderingCols []execinfrapb.Ordering_Column
partialOrderingCols []execinfrapb.Ordering_Column
inputTypes []*types.T
k uint64
// state is the current state of the sort.
state topKSortState
// inputBatch is the last read batch from the input.
inputBatch coldata.Batch
// firstUnprocessedTupleIdx indicates the index of the first tuple in
// inputBatch that hasn't been processed yet.
firstUnprocessedTupleIdx int
// comparators stores one comparator per ordering column.
comparators []vecComparator
// topK stores the top K rows. It is not sorted internally.
topK *colexecutils.AppendOnlyBufferedBatch
// heap is a max heap which stores indices into topK.
heap []int
// sel is a selection vector which specifies an ordering on topK.
sel []int
// emitted is the count of rows which have been emitted so far.
emitted int
output coldata.Batch
maxOutputBatchMemSize int64
// distincter is an operator that groups an input batch by its partially
// ordered column values.
distincterInput *colexecop.FeedOperator
distincter colexecop.Operator
distinctOutput []bool
exportedFromTopK int
exportedFromBatch int
windowedBatch coldata.Batch
}
func (t *topKSorter) Init(ctx context.Context) {
if !t.InitHelper.Init(ctx) {
return
}
t.Input.Init(t.Ctx)
t.topK = colexecutils.NewAppendOnlyBufferedBatch(t.allocator, t.inputTypes, nil /* colsToStore */)
t.comparators = make([]vecComparator, len(t.inputTypes))
for i, typ := range t.inputTypes {
t.comparators[i] = GetVecComparator(typ, 2)
}
// TODO(yuzefovich): switch to calling this method on allocator. This will
// require plumbing unlimited allocator to work correctly in tests with
// memory limit of 1.
t.windowedBatch = coldata.NewMemBatchNoCols(t.inputTypes, coldata.BatchSize())
}
func (t *topKSorter) Next() coldata.Batch {
for {
switch t.state {
case topKSortSpooling:
t.spool()
t.state = topKSortEmitting
case topKSortEmitting:
output := t.emit()
if output.Length() == 0 {
t.state = topKSortDone
continue
}
return output
case topKSortDone:
return coldata.ZeroBatch
default:
colexecerror.InternalError(errors.AssertionFailedf("invalid sort state %v", t.state))
// This code is unreachable, but the compiler cannot infer that.
return nil
}
}
}
func (t *topKSorter) Reset(ctx context.Context) {
if r, ok := t.Input.(colexecop.Resetter); ok {
r.Reset(ctx)
}
t.state = topKSortSpooling
t.firstUnprocessedTupleIdx = 0
t.topK.ResetInternalBatch()
t.emitted = 0
}
// spool reads in the entire input, always storing the top K rows it has seen so
// far in o.topK. This is done by maintaining a max heap of indices into o.topK.
// Whenever we encounter a row which is smaller than the max row in the heap,
// we replace the max with that row.
//
// After all the input has been read, we pop everything off the heap to
// determine the final output ordering. This is used in emit() to output the rows
// in sorted order.
func (t *topKSorter) spool() {
t.distincter.Init(t.Ctx)
t.distincter.(colexecop.Resetter).Reset(t.Ctx)
// Fill up t.topK by spooling up to K rows from the input.
// We don't need to check for distinct groups until after we have filled
// t.topK.
t.inputBatch = t.Input.Next()
t.distincterInput.SetBatch(t.inputBatch)
t.distincter.Next()
remainingRows := t.k
for remainingRows > 0 && t.inputBatch.Length() > 0 {
fromLength := t.inputBatch.Length()
if remainingRows < uint64(t.inputBatch.Length()) {
// t.topK will be full after this batch.
fromLength = int(remainingRows)
}
t.firstUnprocessedTupleIdx = fromLength
t.topK.AppendTuples(t.inputBatch, 0 /* startIdx */, fromLength)
remainingRows -= uint64(fromLength)
if fromLength == t.inputBatch.Length() {
t.inputBatch = t.Input.Next()
t.distincterInput.SetBatch(t.inputBatch)
t.distincter.Next()
t.firstUnprocessedTupleIdx = 0
}
}
t.updateComparators(topKVecIdx, t.topK)
// Initialize the heap.
if cap(t.heap) < t.topK.Length() {
t.heap = make([]int, t.topK.Length())
} else {
t.heap = t.heap[:t.topK.Length()]
}
for i := range t.heap {
t.heap[i] = i
}
heap.Init(t)
// Read the remainder of the input. Whenever a row is less than the heap max,
// swap it in. When we find the end of the group, we can finish reading the
// input.
groupDone := false
for t.inputBatch.Length() > 0{
t.updateComparators(inputVecIdx, t.inputBatch)
sel := t.inputBatch.Selection()
t.allocator.PerformOperation(
t.topK.ColVecs(),
func() {
for i := t.firstUnprocessedTupleIdx; i < t.inputBatch.Length(); i++ {
idx := i
if sel != nil {
idx = sel[i]
}
// If this is a distinct group, we have already found the top K input,
// so we can stop comparing the rest of this and subsequent batches.
if t.distinctOutput[idx] {
groupDone = true
return
}
maxIdx := t.heap[0]
if t.compareRow(inputVecIdx, topKVecIdx, idx, maxIdx) < 0 {
for j := range t.inputTypes {
t.comparators[j].set(inputVecIdx, topKVecIdx, idx, maxIdx)
}
heap.Fix(t, 0)
}
}
t.firstUnprocessedTupleIdx = t.inputBatch.Length()
},
)
if groupDone {
break
}
t.inputBatch = t.Input.Next()
t.distincterInput.SetBatch(t.inputBatch)
t.distincter.Next()
t.firstUnprocessedTupleIdx = 0
}
// t.topK now contains the top K rows unsorted. Create a selection vector
// which specifies the rows in sorted order by popping everything off the
// heap. Note that it's a max heap so we need to fill the selection vector in
// reverse.
t.sel = make([]int, t.topK.Length())
for i := 0; i < t.topK.Length(); i++ {
t.sel[len(t.sel)-i-1] = heap.Pop(t).(int)
}
}
func (t *topKSorter) emit() coldata.Batch {
toEmit := t.topK.Length() - t.emitted
if toEmit == 0 {
// We're done.
return coldata.ZeroBatch
}
t.output, _ = t.allocator.ResetMaybeReallocate(t.inputTypes, t.output, toEmit, t.maxOutputBatchMemSize)
if toEmit > t.output.Capacity() {
toEmit = t.output.Capacity()
}
for i := range t.inputTypes {
vec := t.output.ColVec(i)
// At this point, we have already fully sorted the input. It is ok to do
// this Copy outside of the allocator - the work has been done, but
// theoretically it is possible to hit the limit here (mainly with
// variable-sized types like Bytes). Nonetheless, for performance reasons
// it would be sad to fallback to disk at this point.
vec.Copy(
coldata.SliceArgs{
Src: t.topK.ColVec(i),
Sel: t.sel,
SrcStartIdx: t.emitted,
SrcEndIdx: t.emitted + toEmit,
},
)
}
t.output.SetLength(toEmit)
t.emitted += toEmit
return t.output
}
func (t *topKSorter) compareRow(vecIdx1, vecIdx2 int, rowIdx1, rowIdx2 int) int {
for i := range t.orderingCols {
info := t.orderingCols[i]
res := t.comparators[info.ColIdx].compare(vecIdx1, vecIdx2, rowIdx1, rowIdx2)
if res != 0 {
switch d := info.Direction; d {
case execinfrapb.Ordering_Column_ASC:
return res
case execinfrapb.Ordering_Column_DESC:
return -res
default:
colexecerror.InternalError(errors.AssertionFailedf("unexpected direction value %d", d))
}
}
}
return 0
}
func (t *topKSorter) updateComparators(vecIdx int, batch coldata.Batch) {
for i := range t.inputTypes {
t.comparators[i].setVec(vecIdx, batch.ColVec(i))
}
}
func (t *topKSorter) ExportBuffered(colexecop.Operator) coldata.Batch {
topKLen := t.topK.Length()
// First, we check whether we have exported all tuples from the topK vector.
if t.exportedFromTopK < topKLen {
newExportedFromTopK := t.exportedFromTopK + coldata.BatchSize()
if newExportedFromTopK > topKLen {
newExportedFromTopK = topKLen
}
for i := range t.inputTypes {
window := t.topK.ColVec(i).Window(t.exportedFromTopK, newExportedFromTopK)
t.windowedBatch.ReplaceCol(window, i)
}
t.windowedBatch.SetSelection(false)
t.windowedBatch.SetLength(newExportedFromTopK - t.exportedFromTopK)
t.exportedFromTopK = newExportedFromTopK
return t.windowedBatch
}
// Next, we check whether we have exported all tuples from the last read
// batch.
if t.inputBatch != nil && t.firstUnprocessedTupleIdx+t.exportedFromBatch < t.inputBatch.Length() {
colexecutils.MakeWindowIntoBatch(
t.windowedBatch, t.inputBatch, t.firstUnprocessedTupleIdx, t.inputBatch.Length(), t.inputTypes,
)
t.exportedFromBatch = t.windowedBatch.Length()
return t.windowedBatch
}
return coldata.ZeroBatch
}
// Len is part of heap.Interface and is only meant to be used internally.
func (t *topKSorter) Len() int {
return len(t.heap)
}
// Less is part of heap.Interface and is only meant to be used internally.
func (t *topKSorter) Less(i, j int) bool {
return t.compareRow(topKVecIdx, topKVecIdx, t.heap[i], t.heap[j]) > 0
}
// Swap is part of heap.Interface and is only meant to be used internally.
func (t *topKSorter) Swap(i, j int) {
t.heap[i], t.heap[j] = t.heap[j], t.heap[i]
}
// Push is part of heap.Interface and is only meant to be used internally.
func (t *topKSorter) Push(x interface{}) {
t.heap = append(t.heap, x.(int))
}
// Pop is part of heap.Interface and is only meant to be used internally.
func (t *topKSorter) Pop() interface{} {
x := t.heap[len(t.heap)-1]
t.heap = t.heap[:len(t.heap)-1]
return x
}