-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
Copy pathprojection.go
468 lines (413 loc) · 14.4 KB
/
projection.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
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package executor
import (
"context"
"fmt"
"sync"
"sync/atomic"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"go.uber.org/zap"
)
// This file contains the implementation of the physical Projection Operator:
// https://en.wikipedia.org/wiki/Projection_(relational_algebra)
//
// NOTE:
// 1. The number of "projectionWorker" is controlled by the global session
// variable "tidb_projection_concurrency".
// 2. Unparallel version is used when one of the following situations occurs:
// a. "tidb_projection_concurrency" is set to 0.
// b. The estimated input size is smaller than "tidb_max_chunk_size".
// c. This projection can not be executed vectorially.
type projectionInput struct {
chk *chunk.Chunk
targetWorker *projectionWorker
}
type projectionOutput struct {
chk *chunk.Chunk
done chan error
}
// ProjectionExec implements the physical Projection Operator:
// https://en.wikipedia.org/wiki/Projection_(relational_algebra)
type ProjectionExec struct {
baseExecutor
evaluatorSuit *expression.EvaluatorSuite
finishCh chan struct{}
outputCh chan *projectionOutput
fetcher projectionInputFetcher
numWorkers int64
workers []*projectionWorker
childResult *chunk.Chunk
// parentReqRows indicates how many rows the parent executor is
// requiring. It is set when parallelExecute() is called and used by the
// concurrent projectionInputFetcher.
//
// NOTE: It should be protected by atomic operations.
parentReqRows int64
memTracker *memory.Tracker
wg sync.WaitGroup
calculateNoDelay bool
prepared bool
}
// Open implements the Executor Open interface.
func (e *ProjectionExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return err
}
return e.open(ctx)
}
func (e *ProjectionExec) open(ctx context.Context) error {
e.prepared = false
e.parentReqRows = int64(e.maxChunkSize)
e.memTracker = memory.NewTracker(e.id, -1)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
// For now a Projection can not be executed vectorially only because it
// contains "SetVar" or "GetVar" functions, in this scenario this
// Projection can not be executed parallelly.
if e.numWorkers > 0 && !e.evaluatorSuit.Vectorizable() {
e.numWorkers = 0
}
if e.isUnparallelExec() {
e.childResult = newFirstChunk(e.children[0])
e.memTracker.Consume(e.childResult.MemoryUsage())
}
return nil
}
// Next implements the Executor Next interface.
//
// Here we explain the execution flow of the parallel projection implementation.
// There are 3 main components:
// 1. "projectionInputFetcher": Fetch input "Chunk" from child.
// 2. "projectionWorker": Do the projection work.
// 3. "ProjectionExec.Next": Return result to parent.
//
// 1. "projectionInputFetcher" gets its input and output resources from its
// "inputCh" and "outputCh" channel, once the input and output resources are
// abtained, it fetches child's result into "input.chk" and:
// a. Dispatches this input to the worker specified in "input.targetWorker"
// b. Dispatches this output to the main thread: "ProjectionExec.Next"
// c. Dispatches this output to the worker specified in "input.targetWorker"
// It is finished and exited once:
// a. There is no more input from child.
// b. "ProjectionExec" close the "globalFinishCh"
//
// 2. "projectionWorker" gets its input and output resources from its
// "inputCh" and "outputCh" channel, once the input and output resources are
// abtained, it calculates the projection result use "input.chk" as the input
// and "output.chk" as the output, once the calculation is done, it:
// a. Sends "nil" or error to "output.done" to mark this input is finished.
// b. Returns the "input" resource to "projectionInputFetcher.inputCh"
// They are finished and exited once:
// a. "ProjectionExec" closes the "globalFinishCh"
//
// 3. "ProjectionExec.Next" gets its output resources from its "outputCh" channel.
// After receiving an output from "outputCh", it should wait to receive a "nil"
// or error from "output.done" channel. Once a "nil" or error is received:
// a. Returns this output to its parent
// b. Returns the "output" resource to "projectionInputFetcher.outputCh"
//
// +-----------+----------------------+--------------------------+
// | | | |
// | +--------+---------+ +--------+---------+ +--------+---------+
// | | projectionWorker | + projectionWorker | ... + projectionWorker |
// | +------------------+ +------------------+ +------------------+
// | ^ ^ ^ ^ ^ ^
// | | | | | | |
// | inputCh outputCh inputCh outputCh inputCh outputCh
// | ^ ^ ^ ^ ^ ^
// | | | | | | |
// | | |
// | | +----------------->outputCh
// | | | |
// | | | v
// | +-------+-------+--------+ +---------------------+
// | | projectionInputFetcher | | ProjectionExec.Next |
// | +------------------------+ +---------+-----------+
// | ^ ^ |
// | | | |
// | inputCh outputCh |
// | ^ ^ |
// | | | |
// +------------------------------+ +----------------------+
//
func (e *ProjectionExec) Next(ctx context.Context, req *chunk.Chunk) error {
req.GrowAndReset(e.maxChunkSize)
if e.isUnparallelExec() {
return e.unParallelExecute(ctx, req)
}
return e.parallelExecute(ctx, req)
}
func (e *ProjectionExec) isUnparallelExec() bool {
return e.numWorkers <= 0
}
func (e *ProjectionExec) unParallelExecute(ctx context.Context, chk *chunk.Chunk) error {
// transmit the requiredRows
e.childResult.SetRequiredRows(chk.RequiredRows(), e.maxChunkSize)
mSize := e.childResult.MemoryUsage()
err := Next(ctx, e.children[0], e.childResult)
e.memTracker.Consume(e.childResult.MemoryUsage() - mSize)
if err != nil {
return err
}
if e.childResult.NumRows() == 0 {
return nil
}
err = e.evaluatorSuit.Run(e.ctx, e.childResult, chk)
return err
}
func (e *ProjectionExec) parallelExecute(ctx context.Context, chk *chunk.Chunk) error {
atomic.StoreInt64(&e.parentReqRows, int64(chk.RequiredRows()))
if !e.prepared {
e.prepare(ctx)
e.prepared = true
}
output, ok := <-e.outputCh
if !ok {
return nil
}
err := <-output.done
if err != nil {
return err
}
mSize := output.chk.MemoryUsage()
chk.SwapColumns(output.chk)
e.memTracker.Consume(output.chk.MemoryUsage() - mSize)
e.fetcher.outputCh <- output
return nil
}
func (e *ProjectionExec) prepare(ctx context.Context) {
e.finishCh = make(chan struct{})
e.outputCh = make(chan *projectionOutput, e.numWorkers)
// Initialize projectionInputFetcher.
e.fetcher = projectionInputFetcher{
proj: e,
child: e.children[0],
globalFinishCh: e.finishCh,
globalOutputCh: e.outputCh,
inputCh: make(chan *projectionInput, e.numWorkers),
outputCh: make(chan *projectionOutput, e.numWorkers),
}
// Initialize projectionWorker.
e.workers = make([]*projectionWorker, 0, e.numWorkers)
for i := int64(0); i < e.numWorkers; i++ {
e.workers = append(e.workers, &projectionWorker{
proj: e,
sctx: e.ctx,
evaluatorSuit: e.evaluatorSuit,
globalFinishCh: e.finishCh,
inputGiveBackCh: e.fetcher.inputCh,
inputCh: make(chan *projectionInput, 1),
outputCh: make(chan *projectionOutput, 1),
})
inputChk := newFirstChunk(e.children[0])
e.memTracker.Consume(inputChk.MemoryUsage())
e.fetcher.inputCh <- &projectionInput{
chk: inputChk,
targetWorker: e.workers[i],
}
outputChk := newFirstChunk(e)
e.memTracker.Consume(outputChk.MemoryUsage())
e.fetcher.outputCh <- &projectionOutput{
chk: outputChk,
done: make(chan error, 1),
}
}
e.wg.Add(1)
go e.fetcher.run(ctx)
for i := range e.workers {
e.wg.Add(1)
go e.workers[i].run(ctx)
}
}
func (e *ProjectionExec) drainInputCh(ch chan *projectionInput) {
close(ch)
for item := range ch {
if item.chk != nil {
e.memTracker.Consume(-item.chk.MemoryUsage())
}
}
}
func (e *ProjectionExec) drainOutputCh(ch chan *projectionOutput) {
close(ch)
for item := range ch {
if item.chk != nil {
e.memTracker.Consume(-item.chk.MemoryUsage())
}
}
}
// Close implements the Executor Close interface.
func (e *ProjectionExec) Close() error {
if e.isUnparallelExec() {
e.memTracker.Consume(-e.childResult.MemoryUsage())
e.childResult = nil
}
if e.prepared {
close(e.finishCh)
e.wg.Wait() // Wait for fetcher and workers to finish and exit.
// clear fetcher
e.drainInputCh(e.fetcher.inputCh)
e.drainOutputCh(e.fetcher.outputCh)
// clear workers
for _, w := range e.workers {
e.drainInputCh(w.inputCh)
e.drainOutputCh(w.outputCh)
}
}
if e.runtimeStats != nil {
if e.isUnparallelExec() {
e.runtimeStats.SetConcurrencyInfo("Concurrency", 0)
} else {
e.runtimeStats.SetConcurrencyInfo("Concurrency", int(e.numWorkers))
}
}
return e.baseExecutor.Close()
}
type projectionInputFetcher struct {
proj *ProjectionExec
child Executor
globalFinishCh <-chan struct{}
globalOutputCh chan<- *projectionOutput
wg sync.WaitGroup
inputCh chan *projectionInput
outputCh chan *projectionOutput
}
// run gets projectionInputFetcher's input and output resources from its
// "inputCh" and "outputCh" channel, once the input and output resources are
// abtained, it fetches child's result into "input.chk" and:
// a. Dispatches this input to the worker specified in "input.targetWorker"
// b. Dispatches this output to the main thread: "ProjectionExec.Next"
// c. Dispatches this output to the worker specified in "input.targetWorker"
//
// It is finished and exited once:
// a. There is no more input from child.
// b. "ProjectionExec" close the "globalFinishCh"
func (f *projectionInputFetcher) run(ctx context.Context) {
var output *projectionOutput
defer func() {
if r := recover(); r != nil {
recoveryProjection(output, r)
}
close(f.globalOutputCh)
f.proj.wg.Done()
}()
for {
input := readProjectionInput(f.inputCh, f.globalFinishCh)
if input == nil {
return
}
targetWorker := input.targetWorker
output = readProjectionOutput(f.outputCh, f.globalFinishCh)
if output == nil {
f.proj.memTracker.Consume(-input.chk.MemoryUsage())
return
}
f.globalOutputCh <- output
requiredRows := atomic.LoadInt64(&f.proj.parentReqRows)
input.chk.SetRequiredRows(int(requiredRows), f.proj.maxChunkSize)
mSize := input.chk.MemoryUsage()
err := Next(ctx, f.child, input.chk)
f.proj.memTracker.Consume(input.chk.MemoryUsage() - mSize)
if err != nil || input.chk.NumRows() == 0 {
output.done <- err
f.proj.memTracker.Consume(-input.chk.MemoryUsage())
return
}
targetWorker.inputCh <- input
targetWorker.outputCh <- output
}
}
type projectionWorker struct {
proj *ProjectionExec
sctx sessionctx.Context
evaluatorSuit *expression.EvaluatorSuite
globalFinishCh <-chan struct{}
inputGiveBackCh chan<- *projectionInput
// channel "input" and "output" is :
// a. initialized by "ProjectionExec.prepare"
// b. written by "projectionInputFetcher.run"
// c. read by "projectionWorker.run"
inputCh chan *projectionInput
outputCh chan *projectionOutput
}
// run gets projectionWorker's input and output resources from its
// "inputCh" and "outputCh" channel, once the input and output resources are
// abtained, it calculate the projection result use "input.chk" as the input
// and "output.chk" as the output, once the calculation is done, it:
// a. Sends "nil" or error to "output.done" to mark this input is finished.
// b. Returns the "input" resource to "projectionInputFetcher.inputCh".
//
// It is finished and exited once:
// a. "ProjectionExec" closes the "globalFinishCh".
func (w *projectionWorker) run(ctx context.Context) {
var output *projectionOutput
defer func() {
if r := recover(); r != nil {
recoveryProjection(output, r)
}
w.proj.wg.Done()
}()
for {
input := readProjectionInput(w.inputCh, w.globalFinishCh)
if input == nil {
return
}
output = readProjectionOutput(w.outputCh, w.globalFinishCh)
if output == nil {
return
}
mSize := output.chk.MemoryUsage()
// TODO: trace memory used by the evaluatorSuit including all temporal buffers it uses
err := w.evaluatorSuit.Run(w.sctx, input.chk, output.chk)
w.proj.memTracker.Consume(output.chk.MemoryUsage() - mSize)
output.done <- err
if err != nil {
return
}
w.inputGiveBackCh <- input
}
}
func recoveryProjection(output *projectionOutput, r interface{}) {
if output != nil {
output.done <- errors.Errorf("%v", r)
}
buf := util.GetStack()
logutil.BgLogger().Error("projection executor panicked", zap.String("error", fmt.Sprintf("%v", r)), zap.String("stack", string(buf)))
}
func readProjectionInput(inputCh <-chan *projectionInput, finishCh <-chan struct{}) *projectionInput {
select {
case <-finishCh:
return nil
case input, ok := <-inputCh:
if !ok {
return nil
}
return input
}
}
func readProjectionOutput(outputCh <-chan *projectionOutput, finishCh <-chan struct{}) *projectionOutput {
select {
case <-finishCh:
return nil
case output, ok := <-outputCh:
if !ok {
return nil
}
return output
}
}