-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathexternal_sort_test.go
353 lines (340 loc) · 13.3 KB
/
external_sort_test.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
// Copyright 2020 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 (
"context"
"fmt"
"testing"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/colcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/colexec/typeconv"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils/colcontainerutils"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/marusama/semaphore"
"github.com/stretchr/testify/require"
)
func TestExternalSort(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
defer evalCtx.Stop(ctx)
flowCtx := &execinfra.FlowCtx{
EvalCtx: &evalCtx,
Cfg: &execinfra.ServerConfig{
Settings: st,
},
}
queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */)
defer cleanup()
var (
memAccounts []*mon.BoundAccount
memMonitors []*mon.BytesMonitor
)
const maxNumberPartitions = 3
// Test the case in which the default memory is used as well as the case in
// which the joiner spills to disk.
for _, spillForced := range []bool{false, true} {
flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced
if spillForced {
// In order to increase test coverage of recursive merging, we have the
// lowest possible memory limit - this will force creating partitions
// consisting of a single batch.
flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = 1
} else {
flowCtx.Cfg.TestingKnobs.MemoryLimitBytes = 0
}
for _, tcs := range [][]sortTestCase{sortAllTestCases, topKSortTestCases, sortChunksTestCases} {
for _, tc := range tcs {
t.Run(fmt.Sprintf("spillForced=%t/%s", spillForced, tc.description), func(t *testing.T) {
// Unfortunately, there is currently no better way to check that a
// sorter does not have leftover file descriptors other than appending
// each semaphore used to this slice on construction. This is because
// some tests don't fully drain the input, making intercepting the
// sorter.Close() method not a useful option, since it is impossible
// to check between an expected case where more than 0 FDs are open
// (e.g. in verifySelAndNullResets, where the sorter is not fully
// drained so Close must be called explicitly) and an unexpected one.
// These cases happen during normal execution when a limit is
// satisfied, but flows will call Close explicitly on Cleanup.
// TODO(asubiotto): Not implemented yet, currently we rely on the
// flow tracking open FDs and releasing any leftovers.
var semsToCheck []semaphore.Semaphore
runTests(
t,
[]tuples{tc.tuples},
tc.expected,
orderedVerifier,
func(input []Operator) (Operator, error) {
// A sorter should never exceed maxNumberPartitions+1, even during
// repartitioning. A panic will happen if a sorter requests more
// than this number of file descriptors.
sem := NewTestingSemaphore(maxNumberPartitions + 1)
// If a limit is satisfied before the sorter is drained of all its
// tuples, the sorter will not close its partitioner. During a
// flow this will happen in Cleanup, since there is no way to tell
// an operator that Next won't be called again.
if tc.k == 0 || int(tc.k) >= len(tc.tuples) {
semsToCheck = append(semsToCheck, sem)
}
sorter, accounts, monitors, err := createDiskBackedSorter(
ctx, flowCtx, input, tc.logTypes, tc.ordCols, tc.matchLen, tc.k, func() {},
maxNumberPartitions, false /* delegateFDAcquisition */, queueCfg, sem,
)
memAccounts = append(memAccounts, accounts...)
memMonitors = append(memMonitors, monitors...)
return sorter, err
})
for i, sem := range semsToCheck {
require.Equal(t, 0, sem.GetCount(), "sem still reports open FDs at index %d", i)
}
})
}
}
}
for _, account := range memAccounts {
account.Close(ctx)
}
for _, monitor := range memMonitors {
monitor.Stop(ctx)
}
}
func TestExternalSortRandomized(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
defer evalCtx.Stop(ctx)
flowCtx := &execinfra.FlowCtx{
EvalCtx: &evalCtx,
Cfg: &execinfra.ServerConfig{
Settings: st,
},
}
rng, _ := randutil.NewPseudoRand()
nTups := coldata.BatchSize()*4 + 1
maxCols := 2
// TODO(yuzefovich): randomize types as well.
logTypes := make([]types.T, maxCols)
for i := range logTypes {
logTypes[i] = *types.Int
}
queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */)
defer cleanup()
var (
memAccounts []*mon.BoundAccount
memMonitors []*mon.BytesMonitor
)
const maxNumberPartitions = 3
// Interesting disk spilling scenarios:
// 1) The sorter is forced to spill to disk as soon as possible.
// 2) The memory limit is dynamically set to repartition twice, this will also
// allow the in-memory sorter to spool several batches before hitting the
// memory limit.
colTyps, err := typeconv.FromColumnTypes(logTypes)
require.NoError(t, err)
// memoryToSort is the total amount of memory that will be sorted in this
// test.
memoryToSort := (nTups / coldata.BatchSize()) * estimateBatchSizeBytes(colTyps, coldata.BatchSize())
// partitionSize will be the memory limit passed in to tests with a memory
// limit. With a maximum number of partitions of 2 this will result in
// repartitioning twice.
partitionSize := int64(memoryToSort / 4)
for _, tk := range []execinfra.TestingKnobs{{ForceDiskSpill: true}, {MemoryLimitBytes: partitionSize}} {
flowCtx.Cfg.TestingKnobs = tk
for nCols := 1; nCols <= maxCols; nCols++ {
for nOrderingCols := 1; nOrderingCols <= nCols; nOrderingCols++ {
namePrefix := "MemoryLimit=" + humanizeutil.IBytes(tk.MemoryLimitBytes)
if tk.ForceDiskSpill {
namePrefix = "ForceDiskSpill=true"
}
delegateFDAcquisition := rng.Float64() < 0.5
name := fmt.Sprintf("%s/nCols=%d/nOrderingCols=%d/delegateFDAcquisition=%t", namePrefix, nCols, nOrderingCols, delegateFDAcquisition)
t.Run(name, func(t *testing.T) {
// Unfortunately, there is currently no better way to check that a
// sorter does not have leftover file descriptors other than appending
// each semaphore used to this slice on construction. This is because
// some tests don't fully drain the input, making intercepting the
// sorter.Close() method not a useful option, since it is impossible
// to check between an expected case where more than 0 FDs are open
// (e.g. in verifySelAndNullResets, where the sorter is not fully
// drained so Close must be called explicitly) and an unexpected one.
// These cases happen during normal execution when a limit is
// satisfied, but flows will call Close explicitly on Cleanup.
// TODO(asubiotto): Not implemented yet, currently we rely on the
// flow tracking open FDs and releasing any leftovers.
var semsToCheck []semaphore.Semaphore
tups, expected, ordCols := generateRandomDataForTestSort(rng, nTups, nCols, nOrderingCols)
runTests(
t,
[]tuples{tups},
expected,
orderedVerifier,
func(input []Operator) (Operator, error) {
sem := NewTestingSemaphore(maxNumberPartitions + 1)
semsToCheck = append(semsToCheck, sem)
sorter, accounts, monitors, err := createDiskBackedSorter(
ctx, flowCtx, input, logTypes[:nCols], ordCols,
0 /* matchLen */, 0 /* k */, func() {},
maxNumberPartitions, delegateFDAcquisition, queueCfg, sem)
memAccounts = append(memAccounts, accounts...)
memMonitors = append(memMonitors, monitors...)
return sorter, err
})
for i, sem := range semsToCheck {
require.Equal(t, 0, sem.GetCount(), "sem still reports open FDs at index %d", i)
}
})
}
}
}
for _, account := range memAccounts {
account.Close(ctx)
}
for _, monitor := range memMonitors {
monitor.Stop(ctx)
}
}
func BenchmarkExternalSort(b *testing.B) {
defer leaktest.AfterTest(b)()
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
evalCtx := tree.MakeTestingEvalContext(st)
defer evalCtx.Stop(ctx)
flowCtx := &execinfra.FlowCtx{
EvalCtx: &evalCtx,
Cfg: &execinfra.ServerConfig{
Settings: st,
},
}
rng, _ := randutil.NewPseudoRand()
var (
memAccounts []*mon.BoundAccount
memMonitors []*mon.BytesMonitor
)
queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, false /* inMem */)
defer cleanup()
for _, nBatches := range []int{1 << 1, 1 << 4, 1 << 8} {
for _, nCols := range []int{1, 2, 4} {
for _, spillForced := range []bool{false, true} {
flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced
name := fmt.Sprintf("rows=%d/cols=%d/spilled=%t", nBatches*coldata.BatchSize(), nCols, spillForced)
b.Run(name, func(b *testing.B) {
// 8 (bytes / int64) * nBatches (number of batches) * coldata.BatchSize() (rows /
// batch) * nCols (number of columns / row).
b.SetBytes(int64(8 * nBatches * coldata.BatchSize() * nCols))
logTypes := make([]types.T, nCols)
for i := range logTypes {
logTypes[i] = *types.Int
}
physTypes, err := typeconv.FromColumnTypes(logTypes)
require.NoError(b, err)
batch := testAllocator.NewMemBatch(physTypes)
batch.SetLength(coldata.BatchSize())
ordCols := make([]execinfrapb.Ordering_Column, nCols)
for i := range ordCols {
ordCols[i].ColIdx = uint32(i)
ordCols[i].Direction = execinfrapb.Ordering_Column_Direction(rng.Int() % 2)
col := batch.ColVec(i).Int64()
for j := 0; j < coldata.BatchSize(); j++ {
col[j] = rng.Int63() % int64((i*1024)+1)
}
}
b.ResetTimer()
for n := 0; n < b.N; n++ {
source := newFiniteBatchSource(batch, nBatches)
var spilled bool
// TODO(yuzefovich): do not specify maxNumberPartitions (let the
// external sorter figure out that number itself) once we pass in
// filled-in disk queue config.
sorter, accounts, monitors, err := createDiskBackedSorter(
ctx, flowCtx, []Operator{source}, logTypes, ordCols,
0 /* matchLen */, 0 /* k */, func() { spilled = true },
64 /* maxNumberPartitions */, false /* delegateFDAcquisitions */, queueCfg, &TestingSemaphore{},
)
memAccounts = append(memAccounts, accounts...)
memMonitors = append(memMonitors, monitors...)
if err != nil {
b.Fatal(err)
}
sorter.Init()
for out := sorter.Next(ctx); out.Length() != 0; out = sorter.Next(ctx) {
}
require.Equal(b, spillForced, spilled, fmt.Sprintf(
"expected: spilled=%t\tactual: spilled=%t", spillForced, spilled,
))
}
})
}
}
}
for _, account := range memAccounts {
account.Close(ctx)
}
for _, monitor := range memMonitors {
monitor.Stop(ctx)
}
}
// createDiskBackedSorter is a helper function that instantiates a disk-backed
// sort operator. The desired memory limit must have been already set on
// flowCtx. It returns an operator and an error as well as memory monitors and
// memory accounts that will need to be closed once the caller is done with the
// operator.
func createDiskBackedSorter(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
input []Operator,
logTypes []types.T,
ordCols []execinfrapb.Ordering_Column,
matchLen int,
k uint16,
spillingCallbackFn func(),
maxNumberPartitions int,
delegateFDAcquisitions bool,
diskQueueCfg colcontainer.DiskQueueCfg,
testingSemaphore semaphore.Semaphore,
) (Operator, []*mon.BoundAccount, []*mon.BytesMonitor, error) {
sorterSpec := &execinfrapb.SorterSpec{
OutputOrdering: execinfrapb.Ordering{Columns: ordCols},
OrderingMatchLen: uint32(matchLen),
}
spec := &execinfrapb.ProcessorSpec{
Input: []execinfrapb.InputSyncSpec{{ColumnTypes: logTypes}},
Core: execinfrapb.ProcessorCoreUnion{
Sorter: sorterSpec,
},
Post: execinfrapb.PostProcessSpec{
Limit: uint64(k),
},
}
args := NewColOperatorArgs{
Spec: spec,
Inputs: input,
StreamingMemAccount: testMemAcc,
DiskQueueCfg: diskQueueCfg,
FDSemaphore: testingSemaphore,
}
// External sorter relies on different memory accounts to
// understand when to start a new partition, so we will not use
// the streaming memory account.
args.TestingKnobs.SpillingCallbackFn = spillingCallbackFn
args.TestingKnobs.NumForcedRepartitions = maxNumberPartitions
args.TestingKnobs.DelegateFDAcquisitions = delegateFDAcquisitions
result, err := NewColOperator(ctx, flowCtx, args)
return result.Op, result.BufferingOpMemAccounts, result.BufferingOpMemMonitors, err
}