-
Notifications
You must be signed in to change notification settings - Fork 5.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
executor: add some memory tracker in HashJoin #33918
Changes from 3 commits
6aee861
25a1a8c
8468511
34e838d
e2a0552
33551d5
7d358db
57dfb7a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,6 +20,7 @@ import ( | |
"hash/fnv" | ||
"sync/atomic" | ||
"time" | ||
"unsafe" | ||
|
||
"github.com/pingcap/errors" | ||
"github.com/pingcap/tidb/sessionctx" | ||
|
@@ -29,6 +30,7 @@ import ( | |
"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" | ||
) | ||
|
||
|
@@ -186,6 +188,7 @@ func (c *hashRowContainer) PutChunkSelected(chk *chunk.Chunk, selected, ignoreNu | |
rowPtr := chunk.RowPtr{ChkIdx: chkIdx, RowIdx: uint32(i)} | ||
c.hashTable.Put(key, rowPtr) | ||
} | ||
c.GetMemTracker().Consume(c.hashTable.GetMemoryDelta()) | ||
return nil | ||
} | ||
|
||
|
@@ -251,7 +254,7 @@ func newEntryStore() *entryStore { | |
return es | ||
} | ||
|
||
func (es *entryStore) GetStore() (e *entry) { | ||
func (es *entryStore) GetStore() (e *entry, memDelta int64) { | ||
sliceIdx := uint32(len(es.slices) - 1) | ||
slice := es.slices[sliceIdx] | ||
if es.cursor >= cap(slice) { | ||
|
@@ -263,6 +266,7 @@ func (es *entryStore) GetStore() (e *entry) { | |
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++ | ||
|
@@ -273,6 +277,7 @@ type baseHashTable interface { | |
Put(hashKey uint64, rowPtr chunk.RowPtr) | ||
Get(hashKey uint64) (rowPtrs []chunk.RowPtr) | ||
Len() uint64 | ||
GetMemoryDelta() int64 | ||
} | ||
|
||
// TODO (fangzhuhe) remove unsafeHashTable later if it not used anymore | ||
|
@@ -283,6 +288,9 @@ type unsafeHashTable struct { | |
hashMap map[uint64]*entry | ||
entryStore *entryStore | ||
length uint64 | ||
|
||
bInMap int64 // indicate there are 2^bInMap buckets in hashMap | ||
memDelta int64 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. add a comment for this |
||
} | ||
|
||
// newUnsafeHashTable creates a new unsafeHashTable. estCount means the estimated size of the hashMap. | ||
|
@@ -297,11 +305,16 @@ func newUnsafeHashTable(estCount int) *unsafeHashTable { | |
// 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 := ht.entryStore.GetStore() | ||
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". | ||
|
@@ -318,11 +331,19 @@ func (ht *unsafeHashTable) Get(hashKey uint64) (rowPtrs []chunk.RowPtr) { | |
// if the same key is put more than once. | ||
func (ht *unsafeHashTable) Len() uint64 { return ht.length } | ||
|
||
// GetMemoryDelta gets the memDelta of the concurrentMapHashTable. | ||
func (ht *unsafeHashTable) GetMemoryDelta() int64 { | ||
memDelta := ht.memDelta | ||
ht.memDelta = 0 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why set this to 0? |
||
return memDelta | ||
} | ||
|
||
// concurrentMapHashTable is a concurrent hash table built on concurrentMap | ||
type concurrentMapHashTable struct { | ||
hashMap concurrentMap | ||
entryStore *entryStore | ||
length uint64 | ||
memDelta int64 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a comment for this |
||
} | ||
|
||
// newConcurrentMapHashTable creates a concurrentMapHashTable | ||
|
@@ -331,6 +352,7 @@ func newConcurrentMapHashTable() *concurrentMapHashTable { | |
ht.hashMap = newConcurrentMap() | ||
ht.entryStore = newEntryStore() | ||
ht.length = 0 | ||
ht.memDelta = hack.DefBucketMemoryUsageForMapIntToPtr + int64(unsafe.Sizeof(entry{}))*initialEntrySliceLen | ||
return ht | ||
} | ||
|
||
|
@@ -341,10 +363,13 @@ func (ht *concurrentMapHashTable) Len() uint64 { | |
|
||
// 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 := ht.entryStore.GetStore() | ||
newEntry, memDelta := ht.entryStore.GetStore() | ||
newEntry.ptr = rowPtr | ||
newEntry.next = nil | ||
ht.hashMap.Insert(hashKey, newEntry) | ||
memDelta += ht.hashMap.Insert(hashKey, newEntry) | ||
if memDelta != 0 { | ||
atomic.AddInt64(&ht.memDelta, memDelta) | ||
} | ||
atomic.AddUint64(&ht.length, 1) | ||
} | ||
|
||
|
@@ -357,3 +382,15 @@ func (ht *concurrentMapHashTable) Get(hashKey uint64) (rowPtrs []chunk.RowPtr) { | |
} | ||
return | ||
} | ||
|
||
// GetMemoryDelta gets the memDelta of the concurrentMapHashTable. Memory delta will be cleared after each fetch. | ||
func (ht *concurrentMapHashTable) GetMemoryDelta() int64 { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. GetAndCleanMemoryDelta |
||
var memDelta int64 | ||
for { | ||
memDelta = atomic.LoadInt64(&ht.memDelta) | ||
if atomic.CompareAndSwapInt64(&ht.memDelta, memDelta, 0) { | ||
break | ||
} | ||
} | ||
return memDelta | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add a comment for this func