forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
join.go
657 lines (598 loc) · 18.4 KB
/
join.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
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
// Copyright 2016 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 (
"runtime"
"sync"
"sync/atomic"
"unsafe"
"github.com/juju/errors"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/mvmap"
log "github.com/sirupsen/logrus"
"golang.org/x/net/context"
)
var (
_ Executor = &HashJoinExec{}
_ Executor = &NestedLoopApplyExec{}
)
// HashJoinExec implements the hash join algorithm.
type HashJoinExec struct {
baseExecutor
outerExec Executor
innerExec Executor
outerFilter expression.CNFExprs
outerKeys []*expression.Column
innerKeys []*expression.Column
prepared bool
concurrency uint // concurrency is number of concurrent channels and join workers.
hashTable *mvmap.MVMap
hashJoinBuffers []*hashJoinBuffer
workerWaitGroup sync.WaitGroup // workerWaitGroup is for sync multiple join workers.
finished atomic.Value
closeCh chan struct{} // closeCh add a lock for closing executor.
joinType plan.JoinType
innerIdx int
// We build individual resultGenerator for each join worker when use chunk-based execution,
// to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected.
resultGenerators []joinResultGenerator
outerKeyColIdx []int
innerKeyColIdx []int
innerResult *chunk.List
outerChkResourceCh chan *outerChkResource
outerResultChs []chan *chunk.Chunk
joinChkResourceCh []chan *chunk.Chunk
joinResultCh chan *hashjoinWorkerResult
hashTableValBufs [][][]byte
memTracker *memory.Tracker // track memory usage.
}
// outerChkResource stores the result of the join outer fetch worker,
// `dest` is for Chunk reuse: after join workers process the outer chunk which is read from `dest`,
// they'll store the used chunk as `chk`, and then the outer fetch worker will put new data into `chk` and write `chk` into dest.
type outerChkResource struct {
chk *chunk.Chunk
dest chan<- *chunk.Chunk
}
// hashjoinWorkerResult stores the result of join workers,
// `src` is for Chunk reuse: the main goroutine will get the join result chunk `chk`,
// and push `chk` into `src` after processing, join worker goroutines get the empty chunk from `src`
// and push new data into this chunk.
type hashjoinWorkerResult struct {
chk *chunk.Chunk
err error
src chan<- *chunk.Chunk
}
type hashJoinBuffer struct {
data []types.Datum
bytes []byte
}
// Close implements the Executor Close interface.
func (e *HashJoinExec) Close() error {
close(e.closeCh)
e.finished.Store(true)
if e.prepared {
if e.joinResultCh != nil {
for range e.joinResultCh {
}
}
if e.outerChkResourceCh != nil {
close(e.outerChkResourceCh)
for range e.outerChkResourceCh {
}
}
for i := range e.outerResultChs {
for range e.outerResultChs[i] {
}
}
for i := range e.joinChkResourceCh {
close(e.joinChkResourceCh[i])
for range e.joinChkResourceCh[i] {
}
}
e.outerChkResourceCh = nil
e.joinChkResourceCh = nil
}
e.memTracker.Detach()
e.memTracker = nil
err := e.baseExecutor.Close()
return errors.Trace(err)
}
// Open implements the Executor Open interface.
func (e *HashJoinExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return errors.Trace(err)
}
e.prepared = false
e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaHashJoin)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
e.hashTableValBufs = make([][][]byte, e.concurrency)
e.hashJoinBuffers = make([]*hashJoinBuffer, 0, e.concurrency)
for i := uint(0); i < e.concurrency; i++ {
buffer := &hashJoinBuffer{
data: make([]types.Datum, len(e.outerKeys)),
bytes: make([]byte, 0, 10000),
}
e.hashJoinBuffers = append(e.hashJoinBuffers, buffer)
}
e.closeCh = make(chan struct{})
e.finished.Store(false)
e.workerWaitGroup = sync.WaitGroup{}
return nil
}
func (e *HashJoinExec) getJoinKeyFromChkRow(isOuterKey bool, row chunk.Row, keyBuf []byte) (hasNull bool, _ []byte, err error) {
var keyColIdx []int
var allTypes []*types.FieldType
if isOuterKey {
keyColIdx = e.outerKeyColIdx
allTypes = e.outerExec.retTypes()
} else {
keyColIdx = e.innerKeyColIdx
allTypes = e.innerExec.retTypes()
}
for _, i := range keyColIdx {
if row.IsNull(i) {
return true, keyBuf, nil
}
}
keyBuf = keyBuf[:0]
keyBuf, err = codec.HashChunkRow(e.ctx.GetSessionVars().StmtCtx, keyBuf, row, allTypes, keyColIdx)
if err != nil {
err = errors.Trace(err)
}
return false, keyBuf, err
}
// fetchOuterChunks get chunks from fetches chunks from the big table in a background goroutine
// and sends the chunks to multiple channels which will be read by multiple join workers.
func (e *HashJoinExec) fetchOuterChunks(ctx context.Context) {
defer func() {
for i := range e.outerResultChs {
close(e.outerResultChs[i])
}
if r := recover(); r != nil {
buf := make([]byte, 4096)
stackSize := runtime.Stack(buf, false)
buf = buf[:stackSize]
log.Errorf("hash join outer fetcher panic stack is:\n%s", buf)
e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)}
}
e.workerWaitGroup.Done()
}()
for {
if e.finished.Load().(bool) {
return
}
var outerResource *outerChkResource
ok := true
select {
case <-e.closeCh:
return
case outerResource, ok = <-e.outerChkResourceCh:
if !ok {
return
}
}
outerResult := outerResource.chk
err := e.outerExec.Next(ctx, outerResult)
if err != nil {
e.joinResultCh <- &hashjoinWorkerResult{
err: errors.Trace(err),
}
return
}
if outerResult.NumRows() == 0 {
return
}
outerResource.dest <- outerResult
}
}
// fetchInnerRows fetches all rows from inner executor,
// and append them to e.innerResult.
func (e *HashJoinExec) fetchInnerRows(ctx context.Context) (err error) {
e.innerResult = chunk.NewList(e.innerExec.retTypes(), e.maxChunkSize)
e.innerResult.GetMemTracker().AttachTo(e.memTracker)
e.innerResult.GetMemTracker().SetLabel("innerResult")
for {
chk := e.children[e.innerIdx].newChunk()
err = e.innerExec.Next(ctx, chk)
if err != nil || chk.NumRows() == 0 {
return errors.Trace(err)
}
e.innerResult.Add(chk)
}
}
func (e *HashJoinExec) initializeForProbe() {
// e.outerResultChs is for transmitting the chunks which store the data of outerExec,
// it'll be written by outer worker goroutine, and read by join workers.
e.outerResultChs = make([]chan *chunk.Chunk, e.concurrency)
for i := uint(0); i < e.concurrency; i++ {
e.outerResultChs[i] = make(chan *chunk.Chunk, 1)
}
// e.outerChkResourceCh is for transmitting the used outerExec chunks from join workers to outerExec worker.
e.outerChkResourceCh = make(chan *outerChkResource, e.concurrency)
for i := uint(0); i < e.concurrency; i++ {
e.outerChkResourceCh <- &outerChkResource{
chk: e.outerExec.newChunk(),
dest: e.outerResultChs[i],
}
}
// e.joinChkResourceCh is for transmitting the reused join result chunks
// from the main thread to join worker goroutines.
e.joinChkResourceCh = make([]chan *chunk.Chunk, e.concurrency)
for i := uint(0); i < e.concurrency; i++ {
e.joinChkResourceCh[i] = make(chan *chunk.Chunk, 1)
e.joinChkResourceCh[i] <- e.newChunk()
}
// e.joinResultCh is for transmitting the join result chunks to the main thread.
e.joinResultCh = make(chan *hashjoinWorkerResult, e.concurrency+1)
e.outerKeyColIdx = make([]int, len(e.outerKeys))
for i := range e.outerKeys {
e.outerKeyColIdx[i] = e.outerKeys[i].Index
}
}
func (e *HashJoinExec) fetchOuterAndProbeHashTable(ctx context.Context) {
if e.hashTable.Len() == 0 && e.joinType == plan.InnerJoin {
return
}
e.initializeForProbe()
e.workerWaitGroup.Add(1)
go e.fetchOuterChunks(ctx)
// Start e.concurrency join workers to probe hash table and join inner and outer rows.
for i := uint(0); i < e.concurrency; i++ {
e.workerWaitGroup.Add(1)
go e.runJoinWorker(i)
}
go e.waitJoinWorkersAndCloseResultChan()
}
func (e *HashJoinExec) waitJoinWorkersAndCloseResultChan() {
e.workerWaitGroup.Wait()
close(e.joinResultCh)
}
func (e *HashJoinExec) runJoinWorker(workerID uint) {
defer func() {
if r := recover(); r != nil {
buf := make([]byte, 4096)
stackSize := runtime.Stack(buf, false)
buf = buf[:stackSize]
log.Errorf("hash join worker %v panic stack is:\n%s", workerID, buf)
e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)}
}
e.workerWaitGroup.Done()
}()
var (
outerResult *chunk.Chunk
selected = make([]bool, 0, chunk.InitialCapacity)
)
ok, joinResult := e.getNewJoinResult(workerID)
if !ok {
return
}
// Read and filter outerResult, and join the outerResult with the inner rows.
emptyOuterResult := &outerChkResource{
dest: e.outerResultChs[workerID],
}
for ok := true; ok; {
if e.finished.Load().(bool) {
break
}
select {
case <-e.closeCh:
return
case outerResult, ok = <-e.outerResultChs[workerID]:
}
if !ok {
break
}
ok, joinResult = e.join2Chunk(workerID, outerResult, joinResult, selected)
if !ok {
break
}
outerResult.Reset()
emptyOuterResult.chk = outerResult
e.outerChkResourceCh <- emptyOuterResult
}
if joinResult == nil {
return
} else if joinResult.err != nil || (joinResult.chk != nil && joinResult.chk.NumRows() > 0) {
e.joinResultCh <- joinResult
}
}
func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.Row,
joinResult *hashjoinWorkerResult) (bool, *hashjoinWorkerResult) {
buffer := e.hashJoinBuffers[workerID]
hasNull, joinKey, err := e.getJoinKeyFromChkRow(true, outerRow, buffer.bytes)
if err != nil {
joinResult.err = errors.Trace(err)
return false, joinResult
}
if hasNull {
e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk)
return true, joinResult
}
e.hashTableValBufs[workerID] = e.hashTable.Get(joinKey, e.hashTableValBufs[workerID][:0])
innerPtrs := e.hashTableValBufs[workerID]
if len(innerPtrs) == 0 {
e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk)
return true, joinResult
}
innerRows := make([]chunk.Row, 0, len(innerPtrs))
for _, b := range innerPtrs {
ptr := *(*chunk.RowPtr)(unsafe.Pointer(&b[0]))
matchedInner := e.innerResult.GetRow(ptr)
innerRows = append(innerRows, matchedInner)
}
iter := chunk.NewIterator4Slice(innerRows)
hasMatch := false
for iter.Begin(); iter.Current() != iter.End(); {
matched, err := e.resultGenerators[workerID].tryToMatch(outerRow, iter, joinResult.chk)
if err != nil {
joinResult.err = errors.Trace(err)
return false, joinResult
}
hasMatch = hasMatch || matched
if joinResult.chk.NumRows() == e.maxChunkSize {
ok := true
e.joinResultCh <- joinResult
ok, joinResult = e.getNewJoinResult(workerID)
if !ok {
return false, joinResult
}
}
}
if !hasMatch {
e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk)
}
return true, joinResult
}
func (e *HashJoinExec) getNewJoinResult(workerID uint) (bool, *hashjoinWorkerResult) {
joinResult := &hashjoinWorkerResult{
src: e.joinChkResourceCh[workerID],
}
ok := true
select {
case <-e.closeCh:
ok = false
case joinResult.chk, ok = <-e.joinChkResourceCh[workerID]:
}
return ok, joinResult
}
func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResult *hashjoinWorkerResult,
selected []bool) (ok bool, _ *hashjoinWorkerResult) {
var err error
selected, err = expression.VectorizedFilter(e.ctx, e.outerFilter, chunk.NewIterator4Chunk(outerChk), selected)
if err != nil {
joinResult.err = errors.Trace(err)
return false, joinResult
}
for i := range selected {
if !selected[i] { // process unmatched outer rows
e.resultGenerators[workerID].onMissMatch(outerChk.GetRow(i), joinResult.chk)
} else { // process matched outer rows
ok, joinResult = e.joinMatchedOuterRow2Chunk(workerID, outerChk.GetRow(i), joinResult)
if !ok {
return false, joinResult
}
}
if joinResult.chk.NumRows() == e.maxChunkSize {
e.joinResultCh <- joinResult
ok, joinResult = e.getNewJoinResult(workerID)
if !ok {
return false, joinResult
}
}
}
return true, joinResult
}
// Next implements the Executor Next interface.
// hash join constructs the result following these steps:
// step 1. fetch data from inner child and build a hash table;
// step 2. fetch data from outer child in a background goroutine and probe the hash table in multiple join workers.
func (e *HashJoinExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) {
if !e.prepared {
if err = e.fetchInnerRows(ctx); err != nil {
return errors.Trace(err)
}
if err = e.buildHashTableForList(); err != nil {
return errors.Trace(err)
}
e.fetchOuterAndProbeHashTable(ctx)
e.prepared = true
}
chk.Reset()
if e.joinResultCh == nil {
return nil
}
result, ok := <-e.joinResultCh
if !ok {
return nil
}
if result.err != nil {
e.finished.Store(true)
return errors.Trace(result.err)
}
chk.SwapColumns(result.chk)
result.src <- result.chk
return nil
}
// buildHashTableForList builds hash table from `list`.
// key of hash table: hash value of key columns
// value of hash table: RowPtr of the corresponded row
func (e *HashJoinExec) buildHashTableForList() error {
e.hashTable = mvmap.NewMVMap()
e.innerKeyColIdx = make([]int, len(e.innerKeys))
for i := range e.innerKeys {
e.innerKeyColIdx[i] = e.innerKeys[i].Index
}
var (
hasNull bool
err error
keyBuf = make([]byte, 0, 64)
valBuf = make([]byte, 8)
)
for i := 0; i < e.innerResult.NumChunks(); i++ {
chk := e.innerResult.GetChunk(i)
for j := 0; j < chk.NumRows(); j++ {
hasNull, keyBuf, err = e.getJoinKeyFromChkRow(false, chk.GetRow(j), keyBuf)
if err != nil {
return errors.Trace(err)
}
if hasNull {
continue
}
rowPtr := chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)}
*(*chunk.RowPtr)(unsafe.Pointer(&valBuf[0])) = rowPtr
e.hashTable.Put(keyBuf, valBuf)
}
}
return nil
}
// NestedLoopApplyExec is the executor for apply.
type NestedLoopApplyExec struct {
baseExecutor
innerRows []chunk.Row
cursor int
innerExec Executor
outerExec Executor
innerFilter expression.CNFExprs
outerFilter expression.CNFExprs
outer bool
resultGenerator joinResultGenerator
outerSchema []*expression.CorrelatedColumn
outerChunk *chunk.Chunk
outerChunkCursor int
outerSelected []bool
innerList *chunk.List
innerChunk *chunk.Chunk
innerSelected []bool
innerIter chunk.Iterator
outerRow *chunk.Row
hasMatch bool
memTracker *memory.Tracker // track memory usage.
}
// Close implements the Executor interface.
func (e *NestedLoopApplyExec) Close() error {
e.innerRows = nil
e.memTracker.Detach()
e.memTracker = nil
return errors.Trace(e.outerExec.Close())
}
// Open implements the Executor interface.
func (e *NestedLoopApplyExec) Open(ctx context.Context) error {
err := e.outerExec.Open(ctx)
if err != nil {
return errors.Trace(err)
}
e.cursor = 0
e.innerRows = e.innerRows[:0]
e.outerChunk = e.outerExec.newChunk()
e.innerChunk = e.innerExec.newChunk()
e.innerList = chunk.NewList(e.innerExec.retTypes(), e.maxChunkSize)
e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaNestedLoopApply)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
e.innerList.GetMemTracker().SetLabel("innerList")
e.innerList.GetMemTracker().AttachTo(e.memTracker)
return nil
}
func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *chunk.Chunk) (*chunk.Row, error) {
outerIter := chunk.NewIterator4Chunk(e.outerChunk)
for {
if e.outerChunkCursor >= e.outerChunk.NumRows() {
err := e.outerExec.Next(ctx, e.outerChunk)
if err != nil {
return nil, errors.Trace(err)
}
if e.outerChunk.NumRows() == 0 {
return nil, nil
}
e.outerSelected, err = expression.VectorizedFilter(e.ctx, e.outerFilter, outerIter, e.outerSelected)
if err != nil {
return nil, errors.Trace(err)
}
e.outerChunkCursor = 0
}
outerRow := e.outerChunk.GetRow(e.outerChunkCursor)
selected := e.outerSelected[e.outerChunkCursor]
e.outerChunkCursor++
if selected {
return &outerRow, nil
} else if e.outer {
e.resultGenerator.onMissMatch(outerRow, chk)
if chk.NumRows() == e.maxChunkSize {
return nil, nil
}
}
}
}
// fetchAllInners reads all data from the inner table and stores them in a List.
func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error {
err := e.innerExec.Open(ctx)
defer terror.Call(e.innerExec.Close)
if err != nil {
return errors.Trace(err)
}
e.innerList.Reset()
innerIter := chunk.NewIterator4Chunk(e.innerChunk)
for {
err := e.innerExec.Next(ctx, e.innerChunk)
if err != nil {
return errors.Trace(err)
}
if e.innerChunk.NumRows() == 0 {
return nil
}
e.innerSelected, err = expression.VectorizedFilter(e.ctx, e.innerFilter, innerIter, e.innerSelected)
if err != nil {
return errors.Trace(err)
}
for row := innerIter.Begin(); row != innerIter.End(); row = innerIter.Next() {
if e.innerSelected[row.Idx()] {
e.innerList.AppendRow(row)
}
}
}
}
// Next implements the Executor interface.
func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) {
chk.Reset()
for {
if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() {
if e.outerRow != nil && !e.hasMatch {
e.resultGenerator.onMissMatch(*e.outerRow, chk)
}
e.outerRow, err = e.fetchSelectedOuterRow(ctx, chk)
if e.outerRow == nil || err != nil {
return errors.Trace(err)
}
e.hasMatch = false
for _, col := range e.outerSchema {
*col.Data = e.outerRow.GetDatum(col.Index, col.RetType)
}
err = e.fetchAllInners(ctx)
if err != nil {
return errors.Trace(err)
}
e.innerIter = chunk.NewIterator4List(e.innerList)
e.innerIter.Begin()
}
matched, err := e.resultGenerator.tryToMatch(*e.outerRow, e.innerIter, chk)
e.hasMatch = e.hasMatch || matched
if err != nil || chk.NumRows() == e.maxChunkSize {
return errors.Trace(err)
}
}
}