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: reorg codes for hashtable in HashJoinExec #11937
Changes from 9 commits
6cf30e5
f980b13
8387cb3
63a0c3f
a086d7f
f0e04d6
0f66252
8a5087b
8def643
8ae2de4
ef97c5f
633143d
a3cafe8
b16e09b
a18a423
a5e1273
10b0a3e
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 | ||||
---|---|---|---|---|---|---|
|
@@ -14,31 +14,175 @@ | |||||
package executor | ||||||
|
||||||
import ( | ||||||
"hash" | ||||||
"hash/fnv" | ||||||
|
||||||
"github.com/pingcap/errors" | ||||||
"github.com/pingcap/tidb/sessionctx/stmtctx" | ||||||
"github.com/pingcap/tidb/types" | ||||||
"github.com/pingcap/tidb/util/chunk" | ||||||
"github.com/pingcap/tidb/util/codec" | ||||||
"github.com/pingcap/tidb/util/memory" | ||||||
) | ||||||
|
||||||
const maxEntrySliceLen = 8 * 1024 | ||||||
// hashRowContainer handles the rows and the hash map of a table. | ||||||
// TODO: support spilling out to disk when memory is limited. | ||||||
type hashRowContainer struct { | ||||||
records *chunk.List | ||||||
hashTable *rowHashMap | ||||||
|
||||||
sc *stmtctx.StatementContext | ||||||
allTypes []*types.FieldType | ||||||
keyColIdx []int | ||||||
h hash.Hash64 | ||||||
buf [1]byte | ||||||
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. how about:
Suggested change
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.
|
||||||
} | ||||||
|
||||||
func newHashRowContainer( | ||||||
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. This function contains too many parameters. How about not to use a function to create a |
||||||
sc *stmtctx.StatementContext, statCount int, | ||||||
allTypes []*types.FieldType, keyColIdx []int, initCap, maxChunkSize int) *hashRowContainer { | ||||||
|
||||||
c := &hashRowContainer{ | ||||||
hashTable: newRowHashMapWithStatCount(statCount), | ||||||
sc: sc, | ||||||
allTypes: allTypes, | ||||||
keyColIdx: keyColIdx, | ||||||
h: fnv.New64(), | ||||||
} | ||||||
c.records = chunk.NewList(allTypes, initCap, maxChunkSize) | ||||||
return c | ||||||
} | ||||||
|
||||||
func (c *hashRowContainer) GetMemTracker() *memory.Tracker { | ||||||
return c.records.GetMemTracker() | ||||||
} | ||||||
|
||||||
// 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(probeRow chunk.Row, joinKeysTypes []*types.FieldType, keyColIdx []int, h hash.Hash64, buf []byte) (matched []chunk.Row, hasNull bool, err error) { | ||||||
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. how about: func (c *hashRowContainer) GetMatchedRows(
probeRow chunk.Row,
allTypes []*types.FieldType,
keyColIdx []int,
h hash.Hash64,
buf []byte,
) (
matched []chunk.Row,
hasNull bool,
err error,
) { |
||||||
|
||||||
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. useless line |
||||||
var key uint64 | ||||||
hasNull, key, err = c.getJoinKeyFromChkRow(c.sc, probeRow, joinKeysTypes, keyColIdx, h, buf) | ||||||
if err != nil { | ||||||
return | ||||||
} | ||||||
if hasNull { | ||||||
return | ||||||
} | ||||||
innerPtrs := c.hashTable.Get(key) | ||||||
if len(innerPtrs) == 0 { | ||||||
hasNull = true | ||||||
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 we set 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. It means no inner rows matched by the outer row in 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. I've addressed this by removing the ambiguous |
||||||
return | ||||||
} | ||||||
matched = make([]chunk.Row, 0, len(innerPtrs)) | ||||||
for _, ptr := range innerPtrs { | ||||||
matchedRow := c.records.GetRow(ptr) | ||||||
var ok bool | ||||||
ok, err = c.matchJoinKey(matchedRow, probeRow, joinKeysTypes, keyColIdx) | ||||||
if err != nil { | ||||||
return | ||||||
} | ||||||
if !ok { | ||||||
continue | ||||||
} | ||||||
matched = append(matched, matchedRow) | ||||||
} | ||||||
if len(matched) == 0 { // TODO(fengliyuan): add test case | ||||||
hasNull = true | ||||||
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. ditto |
||||||
} | ||||||
return | ||||||
} | ||||||
|
||||||
// matchJoinKey checks if join keys of buildRow and probeRow are logically equal. | ||||||
func (c *hashRowContainer) matchJoinKey(buildRow, probeRow chunk.Row, probeAllTypes []*types.FieldType, probeColIdx []int) (ok bool, err error) { | ||||||
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. how about: func (c *hashRowContainer) matchJoinKey(
buildRow chunk.Row,
probeRow chunk.Row,
probeAllTypes []*types.FieldType,
probeColIdx []int,
) (ok bool, err error) { |
||||||
return codec.EqualChunkRow(c.sc, | ||||||
buildRow, c.allTypes, c.keyColIdx, | ||||||
probeRow, probeAllTypes, probeColIdx) | ||||||
} | ||||||
|
||||||
// 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) error { | ||||||
chkIdx := uint32(c.records.NumChunks()) | ||||||
c.records.Add(chk) | ||||||
var ( | ||||||
hasNull bool | ||||||
err error | ||||||
key uint64 | ||||||
) | ||||||
numRows := chk.NumRows() | ||||||
for j := 0; j < numRows; j++ { | ||||||
hasNull, key, err = c.getJoinKeyFromChkRow(c.sc, chk.GetRow(j), c.allTypes, c.keyColIdx, c.h, c.buf[:]) | ||||||
if err != nil { | ||||||
return errors.Trace(err) | ||||||
} | ||||||
if hasNull { | ||||||
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. Should we record the 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. I'll adjust it in further PR to involve |
||||||
continue | ||||||
} | ||||||
rowPtr := chunk.RowPtr{ChkIdx: chkIdx, RowIdx: uint32(j)} | ||||||
c.hashTable.Put(key, rowPtr) | ||||||
} | ||||||
return nil | ||||||
} | ||||||
|
||||||
// getJoinKeyFromChkRow fetches join keys from row and calculate the hash value. | ||||||
func (*hashRowContainer) getJoinKeyFromChkRow( | ||||||
sc *stmtctx.StatementContext, | ||||||
row chunk.Row, allTypes []*types.FieldType, keyColIdx []int, h hash.Hash64, buf []byte) (hasNull bool, key uint64, err error) { | ||||||
for _, i := range keyColIdx { | ||||||
if row.IsNull(i) { | ||||||
return true, 0, nil | ||||||
} | ||||||
} | ||||||
h.Reset() | ||||||
err = codec.HashChunkRow(sc, h, row, allTypes, keyColIdx, buf) | ||||||
return false, h.Sum64(), err | ||||||
} | ||||||
|
||||||
func (c hashRowContainer) Len() int { | ||||||
return c.hashTable.Len() | ||||||
} | ||||||
|
||||||
const ( | ||||||
initialEntrySliceLen = 64 | ||||||
maxEntrySliceLen = 8 * 1024 | ||||||
) | ||||||
|
||||||
type entry struct { | ||||||
ptr chunk.RowPtr | ||||||
next entryAddr | ||||||
} | ||||||
|
||||||
type entryStore struct { | ||||||
slices [][]entry | ||||||
sliceIdx uint32 | ||||||
sliceLen uint32 | ||||||
slices [][]entry | ||||||
} | ||||||
|
||||||
func (es *entryStore) init() { | ||||||
es.slices = [][]entry{make([]entry, 0, initialEntrySliceLen)} | ||||||
// Reserve the first empty entry, so entryAddr{} can represent nullEntryAddr. | ||||||
reserved := es.put(entry{}) | ||||||
if reserved != nullEntryAddr { | ||||||
panic("entryStore: first entry is not nullEntryAddr") | ||||||
} | ||||||
} | ||||||
|
||||||
func (es *entryStore) put(e entry) entryAddr { | ||||||
if es.sliceLen == maxEntrySliceLen { | ||||||
es.slices = append(es.slices, make([]entry, 0, maxEntrySliceLen)) | ||||||
es.sliceLen = 0 | ||||||
es.sliceIdx++ | ||||||
sliceIdx := uint32(len(es.slices) - 1) | ||||||
slice := es.slices[sliceIdx] | ||||||
if len(slice) == cap(slice) { | ||||||
// TODO: add test here. | ||||||
size := cap(slice) * 2 | ||||||
if size >= maxEntrySliceLen { | ||||||
size = maxEntrySliceLen | ||||||
} | ||||||
slice = make([]entry, 0, size) | ||||||
es.slices = append(es.slices, slice) | ||||||
sliceIdx++ | ||||||
} | ||||||
addr := entryAddr{sliceIdx: es.sliceIdx, offset: es.sliceLen} | ||||||
es.slices[es.sliceIdx] = append(es.slices[es.sliceIdx], e) | ||||||
es.sliceLen++ | ||||||
addr := entryAddr{sliceIdx: sliceIdx, offset: uint32(len(slice))} | ||||||
es.slices[sliceIdx] = append(slice, e) | ||||||
return addr | ||||||
} | ||||||
|
||||||
|
@@ -56,23 +200,25 @@ var nullEntryAddr = entryAddr{} | |||||
// rowHashMap 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. | ||||||
// TODO(fengliyuan): add unit test for this. | ||||||
type rowHashMap struct { | ||||||
entryStore entryStore | ||||||
hashTable map[uint64]entryAddr | ||||||
length int | ||||||
} | ||||||
|
||||||
// newRowHashMap creates a new rowHashMap. | ||||||
func newRowHashMap() *rowHashMap { | ||||||
func newRowHashMapWithStatCount(statCount int) *rowHashMap { | ||||||
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. Could you explain what does 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.
|
||||||
m := new(rowHashMap) | ||||||
// TODO(fengliyuan): initialize the size of map from the estimated row count for better performance. | ||||||
m.hashTable = make(map[uint64]entryAddr) | ||||||
m.entryStore.slices = [][]entry{make([]entry, 0, 64)} | ||||||
// Reserve the first empty entry, so entryAddr{} can represent nullEntryAddr. | ||||||
m.entryStore.put(entry{}) | ||||||
m.hashTable = make(map[uint64]entryAddr, statCount) | ||||||
m.entryStore.init() | ||||||
return m | ||||||
} | ||||||
|
||||||
func newRowHashMap() *rowHashMap { | ||||||
return newRowHashMapWithStatCount(0) | ||||||
} | ||||||
|
||||||
// Put puts the key/rowPtr pairs to the rowHashMap, multiple rowPtrs are stored in a list. | ||||||
func (m *rowHashMap) Put(hashKey uint64, rowPtr chunk.RowPtr) { | ||||||
oldEntryAddr := m.hashTable[hashKey] | ||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
// 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, | ||
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
package executor | ||
|
||
import ( | ||
. "github.com/pingcap/check" | ||
"github.com/pingcap/tidb/util/chunk" | ||
) | ||
|
||
func (s *pkgTestSuite) TestRowHashMap(c *C) { | ||
m := newRowHashMap() | ||
m.Put(1, chunk.RowPtr{ChkIdx: 1, RowIdx: 1}) | ||
c.Check(m.Get(1), DeepEquals, []chunk.RowPtr{{ChkIdx: 1, RowIdx: 1}}) | ||
|
||
rawData := map[uint64][]chunk.RowPtr{} | ||
for i := uint64(0); i < 10; i++ { | ||
for j := uint64(0); j < initialEntrySliceLen*i; j++ { | ||
rawData[i] = append(rawData[i], chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)}) | ||
} | ||
} | ||
m = newRowHashMap() | ||
// put all rawData into m vertically | ||
for j := uint64(0); j < initialEntrySliceLen*9; j++ { | ||
for i := 9; i >= 0; i-- { | ||
i := uint64(i) | ||
if !(j < initialEntrySliceLen*i) { | ||
break | ||
} | ||
m.Put(i, rawData[i][j]) | ||
} | ||
} | ||
// check | ||
totalCount := 0 | ||
for i := uint64(0); i < 10; i++ { | ||
totalCount += len(rawData[i]) | ||
c.Check(m.Get(i), DeepEquals, rawData[i]) | ||
} | ||
c.Check(m.Len(), Equals, totalCount) | ||
} |
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.
s/innerStatsCount/innerEstCount/
?est
is short forestimated