forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathhash_table.go
686 lines (625 loc) · 24.3 KB
/
hash_table.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
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
// Copyright 2019 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package executor
import (
"fmt"
"hash"
"hash/fnv"
"sync/atomic"
"time"
"unsafe"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/bitmap"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/memory"
)
// hashContext keeps the needed hash context of a db table in hash join.
type hashContext struct {
// allTypes one-to-one correspondence with keyColIdx
allTypes []*types.FieldType
keyColIdx []int
naKeyColIdx []int
buf []byte
hashVals []hash.Hash64
hasNull []bool
naHasNull []bool
naColNullBitMap []*bitmap.ConcurrentBitmap
}
func (hc *hashContext) initHash(rows int) {
if hc.buf == nil {
hc.buf = make([]byte, 1)
}
if len(hc.hashVals) < rows {
hc.hasNull = make([]bool, rows)
hc.hashVals = make([]hash.Hash64, rows)
for i := 0; i < rows; i++ {
hc.hashVals[i] = fnv.New64()
}
} else {
for i := 0; i < rows; i++ {
hc.hasNull[i] = false
hc.hashVals[i].Reset()
}
}
if len(hc.naKeyColIdx) > 0 {
// isNAAJ
if len(hc.naColNullBitMap) < rows {
hc.naHasNull = make([]bool, rows)
hc.naColNullBitMap = make([]*bitmap.ConcurrentBitmap, rows)
for i := 0; i < rows; i++ {
hc.naColNullBitMap[i] = bitmap.NewConcurrentBitmap(len(hc.naKeyColIdx))
}
} else {
for i := 0; i < rows; i++ {
hc.naHasNull[i] = false
hc.naColNullBitMap[i].Reset(len(hc.naKeyColIdx))
}
}
}
}
type hashStatistic struct {
// NOTE: probeCollision may be accessed from multiple goroutines concurrently.
probeCollision int64
buildTableElapse time.Duration
}
func (s *hashStatistic) String() string {
return fmt.Sprintf("probe_collision:%v, build:%v", s.probeCollision, execdetails.FormatDuration(s.buildTableElapse))
}
type hashNANullBucket struct {
entries []*naEntry
}
// hashRowContainer handles the rows and the hash map of a table.
// NOTE: a hashRowContainer may be shallow copied by the invoker, define all the
// member attributes as pointer type to avoid unexpected problems.
type hashRowContainer struct {
sc *stmtctx.StatementContext
hCtx *hashContext
stat *hashStatistic
// hashTable stores the map of hashKey and RowPtr
hashTable baseHashTable
// hashNANullBucket stores the rows with any null value in NAAJ join key columns.
// After build process, NANUllBucket is read only here for multi probe worker.
hashNANullBucket *hashNANullBucket
rowContainer *chunk.RowContainer
memTracker *memory.Tracker
// chkBuf buffer the data reads from the disk if rowContainer is spilled.
chkBuf *chunk.Chunk
chkBufSizeForOneProbe int64
}
func newHashRowContainer(sCtx sessionctx.Context, hCtx *hashContext, allTypes []*types.FieldType) *hashRowContainer {
maxChunkSize := sCtx.GetSessionVars().MaxChunkSize
rc := chunk.NewRowContainer(allTypes, maxChunkSize)
c := &hashRowContainer{
sc: sCtx.GetSessionVars().StmtCtx,
hCtx: hCtx,
stat: new(hashStatistic),
hashTable: newConcurrentMapHashTable(),
rowContainer: rc,
memTracker: memory.NewTracker(memory.LabelForRowContainer, -1),
}
if isNAAJ := len(hCtx.naKeyColIdx) > 0; isNAAJ {
c.hashNANullBucket = &hashNANullBucket{}
}
rc.GetMemTracker().AttachTo(c.GetMemTracker())
return c
}
func (c *hashRowContainer) ShallowCopy() *hashRowContainer {
newHRC := *c
newHRC.rowContainer = c.rowContainer.ShallowCopyWithNewMutex()
// multi hashRowContainer ref to one single NA-NULL bucket slice.
// newHRC.hashNANullBucket = c.hashNANullBucket
return &newHRC
}
// GetMatchedRows get matched rows from probeRow. It can be called
// in multiple goroutines while each goroutine should keep its own
// h and buf.
func (c *hashRowContainer) GetMatchedRows(probeKey uint64, probeRow chunk.Row, hCtx *hashContext, matched []chunk.Row) ([]chunk.Row, error) {
matchedRows, _, err := c.GetMatchedRowsAndPtrs(probeKey, probeRow, hCtx, matched, nil, false)
return matchedRows, err
}
func (c *hashRowContainer) GetAllMatchedRows(probeHCtx *hashContext, probeSideRow chunk.Row,
probeKeyNullBits *bitmap.ConcurrentBitmap, matched []chunk.Row, needCheckBuildRowPos, needCheckProbeRowPos []int) ([]chunk.Row, error) {
// for NAAJ probe row with null, we should match them with all build rows.
var (
ok bool
err error
innerPtrs []chunk.RowPtr
)
c.hashTable.Iter(
func(_ uint64, e *entry) {
entryAddr := e
for entryAddr != nil {
innerPtrs = append(innerPtrs, entryAddr.ptr)
entryAddr = entryAddr.next
}
})
matched = matched[:0]
if len(innerPtrs) == 0 {
return matched, nil
}
// all built bucket rows come from hash table, their bitmap are all nil (doesn't contain any null). so
// we could only use the probe null bits to filter valid rows.
if probeKeyNullBits != nil && len(probeHCtx.naKeyColIdx) > 1 {
// if len(probeHCtx.naKeyColIdx)=1
// that means the NA-Join probe key is directly a (null) <-> (fetch all buckets), nothing to do.
// else like
// (null, 1, 2), we should use the not-null probe bit to filter rows. Only fetch rows like
// ( ? , 1, 2), that exactly with value as 1 and 2 in the second and third join key column.
needCheckProbeRowPos = needCheckProbeRowPos[:0]
needCheckBuildRowPos = needCheckBuildRowPos[:0]
keyColLen := len(c.hCtx.naKeyColIdx)
for i := 0; i < keyColLen; i++ {
// since all bucket is from hash table (Not Null), so the buildSideNullBits check is eliminated.
if probeKeyNullBits.UnsafeIsSet(i) {
continue
}
needCheckBuildRowPos = append(needCheckBuildRowPos, c.hCtx.naKeyColIdx[i])
needCheckProbeRowPos = append(needCheckProbeRowPos, probeHCtx.naKeyColIdx[i])
}
}
var mayMatchedRow chunk.Row
for _, ptr := range innerPtrs {
mayMatchedRow, c.chkBuf, err = c.rowContainer.GetRowAndAppendToChunk(ptr, c.chkBuf)
if err != nil {
return nil, err
}
if probeKeyNullBits != nil && len(probeHCtx.naKeyColIdx) > 1 {
// check the idxs-th value of the join columns.
ok, err = codec.EqualChunkRow(c.sc, mayMatchedRow, c.hCtx.allTypes, needCheckBuildRowPos, probeSideRow, probeHCtx.allTypes, needCheckProbeRowPos)
if err != nil {
return nil, err
}
if !ok {
continue
}
// once ok. just append the (maybe) valid build row for latter other conditions check if any.
}
matched = append(matched, mayMatchedRow)
}
return matched, nil
}
// signalCheckpointForJoinMask indicates the times of row probe that a signal detection will be triggered.
const signalCheckpointForJoinMask int = 1<<14 - 1
// rowSize is the size of Row.
const rowSize = int64(unsafe.Sizeof(chunk.Row{}))
// rowPtrSize is the size of RowPtr.
const rowPtrSize = int64(unsafe.Sizeof(chunk.RowPtr{}))
// GetMatchedRowsAndPtrs get matched rows and Ptrs from probeRow. It can be called
// in multiple goroutines while each goroutine should keep its own
// h and buf.
func (c *hashRowContainer) GetMatchedRowsAndPtrs(probeKey uint64, probeRow chunk.Row, hCtx *hashContext, matched []chunk.Row, matchedPtrs []chunk.RowPtr, needPtr bool) ([]chunk.Row, []chunk.RowPtr, error) {
var err error
innerPtrs := c.hashTable.Get(probeKey)
if len(innerPtrs) == 0 {
return nil, nil, err
}
matched = matched[:0]
var matchedRow chunk.Row
matchedPtrs = matchedPtrs[:0]
// Some variables used for memTracker.
var (
matchedDataSize = int64(cap(matched))*rowSize + int64(cap(matchedPtrs))*rowPtrSize
lastChunkBufPointer *chunk.Chunk = nil
memDelta int64 = 0
needTrackMemUsage = cap(innerPtrs) > signalCheckpointForJoinMask
)
c.chkBuf = nil
c.memTracker.Consume(-c.chkBufSizeForOneProbe)
if needTrackMemUsage {
c.memTracker.Consume(int64(cap(innerPtrs)) * rowPtrSize)
defer c.memTracker.Consume(-int64(cap(innerPtrs))*rowPtrSize + memDelta)
}
c.chkBufSizeForOneProbe = 0
for i, ptr := range innerPtrs {
matchedRow, c.chkBuf, err = c.rowContainer.GetRowAndAppendToChunk(ptr, c.chkBuf)
if err != nil {
return nil, nil, err
}
var ok bool
ok, err = c.matchJoinKey(matchedRow, probeRow, hCtx)
if err != nil {
return nil, nil, err
}
if needTrackMemUsage && c.chkBuf != lastChunkBufPointer && lastChunkBufPointer != nil {
lastChunkSize := lastChunkBufPointer.MemoryUsage()
c.chkBufSizeForOneProbe += lastChunkSize
memDelta += lastChunkSize
}
lastChunkBufPointer = c.chkBuf
if needTrackMemUsage && (i&signalCheckpointForJoinMask == signalCheckpointForJoinMask) {
// Trigger Consume for checking the OOM Action signal
memDelta += int64(cap(matched))*rowSize + int64(cap(matchedPtrs))*rowPtrSize - matchedDataSize
matchedDataSize = int64(cap(matched))*rowSize + int64(cap(matchedPtrs))*rowPtrSize
c.memTracker.Consume(memDelta + 1)
memDelta = 0
}
if !ok {
atomic.AddInt64(&c.stat.probeCollision, 1)
continue
}
matched = append(matched, matchedRow)
if needPtr {
matchedPtrs = append(matchedPtrs, ptr)
}
}
return matched, matchedPtrs, err
}
func (c *hashRowContainer) GetNullBucketRows(probeHCtx *hashContext, probeSideRow chunk.Row,
probeKeyNullBits *bitmap.ConcurrentBitmap, matched []chunk.Row, needCheckBuildRowPos, needCheckProbeRowPos []int) ([]chunk.Row, error) {
var (
ok bool
err error
mayMatchedRow chunk.Row
)
matched = matched[:0]
for _, nullEntry := range c.hashNANullBucket.entries {
mayMatchedRow, c.chkBuf, err = c.rowContainer.GetRowAndAppendToChunk(nullEntry.ptr, c.chkBuf)
if err != nil {
return nil, err
}
// since null bucket is a unified bucket. cases like below:
// case1: left side (probe side) has null
// left side key <1,null>, actually we can fetch all bucket <1, ?> and filter 1 at the first join key, once
// got a valid right row after other condition, then we can just return.
// case2: left side (probe side) don't have null
// left side key <1, 2>, actually we should fetch <1,null>, <null, 2>, <null, null> from the null bucket because
// case like <3,null> is obviously not matched with the probe key.
needCheckProbeRowPos = needCheckProbeRowPos[:0]
needCheckBuildRowPos = needCheckBuildRowPos[:0]
keyColLen := len(c.hCtx.naKeyColIdx)
if probeKeyNullBits != nil {
// when the probeKeyNullBits is not nil, it means the probe key has null values, where we should distinguish
// whether is empty set or not. In other words, we should fetch at least a valid from the null bucket here.
// for values at the same index of the join key in which they are both not null, the values should be exactly the same.
//
// step: probeKeyNullBits & buildKeyNullBits, for those bits with 0, we should check if both values are the same.
// we can just use the UnsafeIsSet here, because insert action of the build side has all finished.
//
// 1 0 1 0 means left join key : null ? null ?
// 1 0 0 0 means right join key : null ? ? ?
// ---------------------------------------------
// left & right: 1 0 1 0: just do the explicit column value check for whose bit is 0. (means no null from both side)
for i := 0; i < keyColLen; i++ {
if probeKeyNullBits.UnsafeIsSet(i) || nullEntry.nullBitMap.UnsafeIsSet(i) {
continue
}
needCheckBuildRowPos = append(needCheckBuildRowPos, c.hCtx.naKeyColIdx[i])
needCheckProbeRowPos = append(needCheckProbeRowPos, probeHCtx.naKeyColIdx[i])
}
// check the idxs-th value of the join columns.
ok, err = codec.EqualChunkRow(c.sc, mayMatchedRow, c.hCtx.allTypes, needCheckBuildRowPos, probeSideRow, probeHCtx.allTypes, needCheckProbeRowPos)
if err != nil {
return nil, err
}
if !ok {
continue
}
} else {
// when the probeKeyNullBits is nil, it means the probe key is not null. But in the process of matching the null bucket,
// we still need to do the non-null (explicit) value check.
//
// eg: the probe key is <1,2>, we only get <2, null> in the null bucket, even we can take the null as a wildcard symbol,
// the first value of this two tuple is obviously not a match. So we need filter it here.
for i := 0; i < keyColLen; i++ {
if nullEntry.nullBitMap.UnsafeIsSet(i) {
continue
}
needCheckBuildRowPos = append(needCheckBuildRowPos, c.hCtx.naKeyColIdx[i])
needCheckProbeRowPos = append(needCheckProbeRowPos, probeHCtx.naKeyColIdx[i])
}
// check the idxs-th value of the join columns.
ok, err = codec.EqualChunkRow(c.sc, mayMatchedRow, c.hCtx.allTypes, needCheckBuildRowPos, probeSideRow, probeHCtx.allTypes, needCheckProbeRowPos)
if err != nil {
return nil, err
}
if !ok {
continue
}
}
// once ok. just append the (maybe) valid build row for latter other conditions check if any.
matched = append(matched, mayMatchedRow)
}
return matched, err
}
// matchJoinKey checks if join keys of buildRow and probeRow are logically equal.
func (c *hashRowContainer) matchJoinKey(buildRow, probeRow chunk.Row, probeHCtx *hashContext) (ok bool, err error) {
return codec.EqualChunkRow(c.sc,
buildRow, c.hCtx.allTypes, c.hCtx.keyColIdx,
probeRow, probeHCtx.allTypes, probeHCtx.keyColIdx)
}
// alreadySpilledSafeForTest indicates that records have spilled out into disk. It's thread-safe.
// nolint: unused
func (c *hashRowContainer) alreadySpilledSafeForTest() bool {
return c.rowContainer.AlreadySpilledSafeForTest()
}
// PutChunk puts a chunk into hashRowContainer and build hash map. It's not thread-safe.
// key of hash table: hash value of key columns
// value of hash table: RowPtr of the corresponded row
func (c *hashRowContainer) PutChunk(chk *chunk.Chunk, ignoreNulls []bool) error {
return c.PutChunkSelected(chk, nil, ignoreNulls)
}
// PutChunkSelected selectively puts a chunk into hashRowContainer and build hash map. It's not thread-safe.
// key of hash table: hash value of key columns
// value of hash table: RowPtr of the corresponded row
func (c *hashRowContainer) PutChunkSelected(chk *chunk.Chunk, selected, ignoreNulls []bool) error {
start := time.Now()
defer func() { c.stat.buildTableElapse += time.Since(start) }()
chkIdx := uint32(c.rowContainer.NumChunks())
err := c.rowContainer.Add(chk)
if err != nil {
return err
}
numRows := chk.NumRows()
c.hCtx.initHash(numRows)
hCtx := c.hCtx
// By now, the combination of 1 and 2 can't take a run at same time.
// 1: write the row data of join key to hashVals. (normal EQ key should ignore the null values.) null-EQ for Except statement is an exception.
for keyIdx, colIdx := range c.hCtx.keyColIdx {
ignoreNull := len(ignoreNulls) > keyIdx && ignoreNulls[keyIdx]
err := codec.HashChunkSelected(c.sc, hCtx.hashVals, chk, hCtx.allTypes[keyIdx], colIdx, hCtx.buf, hCtx.hasNull, selected, ignoreNull)
if err != nil {
return errors.Trace(err)
}
}
// 2: write the row data of NA join key to hashVals. (NA EQ key should collect all rows including null value as one bucket.)
isNAAJ := len(c.hCtx.naKeyColIdx) > 0
hasNullMark := make([]bool, len(hCtx.hasNull))
for keyIdx, colIdx := range c.hCtx.naKeyColIdx {
// NAAJ won't ignore any null values, but collect them as one hash bucket.
err := codec.HashChunkSelected(c.sc, hCtx.hashVals, chk, hCtx.allTypes[keyIdx], colIdx, hCtx.buf, hCtx.hasNull, selected, false)
if err != nil {
return errors.Trace(err)
}
// todo: we can collect the bitmap in codec.HashChunkSelected to avoid loop here, but the params modification is quite big.
// after fetch one NA column, collect the null value to null bitmap for every row. (use hasNull flag to accelerate)
// eg: if a NA Join cols is (a, b, c), for every build row here we maintained a 3-bit map to mark which column are null for them.
for rowIdx := 0; rowIdx < numRows; rowIdx++ {
if hCtx.hasNull[rowIdx] {
hCtx.naColNullBitMap[rowIdx].UnsafeSet(keyIdx)
// clean and try fetch next NA join col.
hCtx.hasNull[rowIdx] = false
// just a mark variable for whether there is a null in at least one NA join column.
hasNullMark[rowIdx] = true
}
}
}
for i := 0; i < numRows; i++ {
if isNAAJ {
if selected != nil && !selected[i] {
continue
}
if hasNullMark[i] {
// collect the null rows to slice.
rowPtr := chunk.RowPtr{ChkIdx: chkIdx, RowIdx: uint32(i)}
// do not directly ref the null bits map here, because the bit map will be reset and reused in next batch of chunk data.
c.hashNANullBucket.entries = append(c.hashNANullBucket.entries, &naEntry{rowPtr, c.hCtx.naColNullBitMap[i].Clone()})
} else {
// insert the not-null rows to hash table.
key := c.hCtx.hashVals[i].Sum64()
rowPtr := chunk.RowPtr{ChkIdx: chkIdx, RowIdx: uint32(i)}
c.hashTable.Put(key, rowPtr)
}
} else {
if (selected != nil && !selected[i]) || c.hCtx.hasNull[i] {
continue
}
key := c.hCtx.hashVals[i].Sum64()
rowPtr := chunk.RowPtr{ChkIdx: chkIdx, RowIdx: uint32(i)}
c.hashTable.Put(key, rowPtr)
}
}
c.GetMemTracker().Consume(c.hashTable.GetAndCleanMemoryDelta())
return nil
}
// NumChunks returns the number of chunks in the rowContainer
func (c *hashRowContainer) NumChunks() int {
return c.rowContainer.NumChunks()
}
// NumRowsOfChunk returns the number of rows of a chunk
func (c *hashRowContainer) NumRowsOfChunk(chkID int) int {
return c.rowContainer.NumRowsOfChunk(chkID)
}
// GetChunk returns chkIdx th chunk of in memory records, only works if rowContainer is not spilled
func (c *hashRowContainer) GetChunk(chkIdx int) (*chunk.Chunk, error) {
return c.rowContainer.GetChunk(chkIdx)
}
// GetRow returns the row the ptr pointed to in the rowContainer
func (c *hashRowContainer) GetRow(ptr chunk.RowPtr) (chunk.Row, error) {
return c.rowContainer.GetRow(ptr)
}
// Len returns number of records in the hash table.
func (c *hashRowContainer) Len() uint64 {
return c.hashTable.Len()
}
func (c *hashRowContainer) Close() error {
defer c.memTracker.Detach()
c.chkBuf = nil
return c.rowContainer.Close()
}
// GetMemTracker returns the underlying memory usage tracker in hashRowContainer.
func (c *hashRowContainer) GetMemTracker() *memory.Tracker { return c.memTracker }
// GetDiskTracker returns the underlying disk usage tracker in hashRowContainer.
func (c *hashRowContainer) GetDiskTracker() *disk.Tracker { return c.rowContainer.GetDiskTracker() }
// ActionSpill returns a memory.ActionOnExceed for spilling over to disk.
func (c *hashRowContainer) ActionSpill() memory.ActionOnExceed {
return c.rowContainer.ActionSpill()
}
const (
initialEntrySliceLen = 64
maxEntrySliceLen = 8192
)
type entry struct {
ptr chunk.RowPtr
next *entry
}
type naEntry struct {
ptr chunk.RowPtr
nullBitMap *bitmap.ConcurrentBitmap
}
type entryStore struct {
slices [][]entry
cursor int
}
func newEntryStore() *entryStore {
es := new(entryStore)
es.slices = [][]entry{make([]entry, initialEntrySliceLen)}
es.cursor = 0
return es
}
func (es *entryStore) GetStore() (e *entry, memDelta int64) {
sliceIdx := uint32(len(es.slices) - 1)
slice := es.slices[sliceIdx]
if es.cursor >= cap(slice) {
size := cap(slice) * 2
if size >= maxEntrySliceLen {
size = maxEntrySliceLen
}
slice = make([]entry, size)
es.slices = append(es.slices, slice)
sliceIdx++
es.cursor = 0
memDelta = int64(unsafe.Sizeof(entry{})) * int64(size)
}
e = &es.slices[sliceIdx][es.cursor]
es.cursor++
return
}
type baseHashTable interface {
Put(hashKey uint64, rowPtr chunk.RowPtr)
Get(hashKey uint64) (rowPtrs []chunk.RowPtr)
Len() uint64
// GetAndCleanMemoryDelta gets and cleans the memDelta of the baseHashTable. Memory delta will be cleared after each fetch.
// It indicates the memory delta of the baseHashTable since the last calling GetAndCleanMemoryDelta().
GetAndCleanMemoryDelta() int64
Iter(func(uint64, *entry))
}
// TODO (fangzhuhe) remove unsafeHashTable later if it not used anymore
// unsafeHashTable stores multiple rowPtr of rows for a given key with minimum GC overhead.
// A given key can store multiple values.
// It is not thread-safe, should only be used in one goroutine.
type unsafeHashTable struct {
hashMap map[uint64]*entry
entryStore *entryStore
length uint64
bInMap int64 // indicate there are 2^bInMap buckets in hashMap
memDelta int64 // the memory delta of the unsafeHashTable since the last calling GetAndCleanMemoryDelta()
}
// newUnsafeHashTable creates a new unsafeHashTable. estCount means the estimated size of the hashMap.
// If unknown, set it to 0.
func newUnsafeHashTable(estCount int) *unsafeHashTable {
ht := new(unsafeHashTable)
ht.hashMap = make(map[uint64]*entry, estCount)
ht.entryStore = newEntryStore()
return ht
}
// Put puts the key/rowPtr pairs to the unsafeHashTable, multiple rowPtrs are stored in a list.
func (ht *unsafeHashTable) Put(hashKey uint64, rowPtr chunk.RowPtr) {
oldEntry := ht.hashMap[hashKey]
newEntry, memDelta := ht.entryStore.GetStore()
newEntry.ptr = rowPtr
newEntry.next = oldEntry
ht.hashMap[hashKey] = newEntry
if len(ht.hashMap) > (1<<ht.bInMap)*hack.LoadFactorNum/hack.LoadFactorDen {
memDelta += hack.DefBucketMemoryUsageForMapIntToPtr * (1 << ht.bInMap)
ht.bInMap++
}
ht.length++
ht.memDelta += memDelta
}
// Get gets the values of the "key" and appends them to "values".
func (ht *unsafeHashTable) Get(hashKey uint64) (rowPtrs []chunk.RowPtr) {
entryAddr := ht.hashMap[hashKey]
for entryAddr != nil {
rowPtrs = append(rowPtrs, entryAddr.ptr)
entryAddr = entryAddr.next
}
return
}
// Len returns the number of rowPtrs in the unsafeHashTable, the number of keys may be less than Len
// if the same key is put more than once.
func (ht *unsafeHashTable) Len() uint64 { return ht.length }
// GetAndCleanMemoryDelta gets and cleans the memDelta of the unsafeHashTable.
func (ht *unsafeHashTable) GetAndCleanMemoryDelta() int64 {
memDelta := ht.memDelta
ht.memDelta = 0
return memDelta
}
func (ht *unsafeHashTable) Iter(traverse func(key uint64, e *entry)) {
for k := range ht.hashMap {
entryAddr := ht.hashMap[k]
traverse(k, entryAddr)
}
}
// concurrentMapHashTable is a concurrent hash table built on concurrentMap
type concurrentMapHashTable struct {
hashMap concurrentMap
entryStore *entryStore
length uint64
memDelta int64 // the memory delta of the concurrentMapHashTable since the last calling GetAndCleanMemoryDelta()
}
// newConcurrentMapHashTable creates a concurrentMapHashTable
func newConcurrentMapHashTable() *concurrentMapHashTable {
ht := new(concurrentMapHashTable)
ht.hashMap = newConcurrentMap()
ht.entryStore = newEntryStore()
ht.length = 0
ht.memDelta = hack.DefBucketMemoryUsageForMapIntToPtr + int64(unsafe.Sizeof(entry{}))*initialEntrySliceLen
return ht
}
// Len return the number of rowPtrs in the concurrentMapHashTable
func (ht *concurrentMapHashTable) Len() uint64 {
return ht.length
}
// Put puts the key/rowPtr pairs to the concurrentMapHashTable, multiple rowPtrs are stored in a list.
func (ht *concurrentMapHashTable) Put(hashKey uint64, rowPtr chunk.RowPtr) {
newEntry, memDelta := ht.entryStore.GetStore()
newEntry.ptr = rowPtr
newEntry.next = nil
memDelta += ht.hashMap.Insert(hashKey, newEntry)
if memDelta != 0 {
atomic.AddInt64(&ht.memDelta, memDelta)
}
atomic.AddUint64(&ht.length, 1)
}
// Get gets the values of the "key" and appends them to "values".
func (ht *concurrentMapHashTable) Get(hashKey uint64) (rowPtrs []chunk.RowPtr) {
entryAddr, _ := ht.hashMap.Get(hashKey)
for entryAddr != nil {
rowPtrs = append(rowPtrs, entryAddr.ptr)
entryAddr = entryAddr.next
}
return
}
// Iter gets the every value of the hash table.
func (ht *concurrentMapHashTable) Iter(traverse func(key uint64, e *entry)) {
ht.hashMap.IterCb(traverse)
}
// GetAndCleanMemoryDelta gets and cleans the memDelta of the concurrentMapHashTable. Memory delta will be cleared after each fetch.
func (ht *concurrentMapHashTable) GetAndCleanMemoryDelta() int64 {
var memDelta int64
for {
memDelta = atomic.LoadInt64(&ht.memDelta)
if atomic.CompareAndSwapInt64(&ht.memDelta, memDelta, 0) {
break
}
}
return memDelta
}