-
Notifications
You must be signed in to change notification settings - Fork 4
exec: hash joiner #1
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
base: master
Are you sure you want to change the base?
Changes from 6 commits
c10e878
6559061
8924344
c910094
8815163
dd8089a
498d9d1
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 |
|---|---|---|
| @@ -0,0 +1,285 @@ | ||
| package exectoy | ||
|
|
||
| // todo(changangela) hash table should be able to rehash or determine bucket size dynamically | ||
| const hashTableBucketSize = 1 << 16 | ||
|
|
||
| type hashTableInt struct { | ||
| first []int | ||
| next []int | ||
|
|
||
| keys intColumn | ||
| values []intColumn | ||
|
|
||
| nOutCols int | ||
| count int | ||
| bucketSize int | ||
|
|
||
| // allocated represents the reserved number of keys that the hashTableInt | ||
| // can store in its current state | ||
| allocated int | ||
| } | ||
|
|
||
| func makeHashTableInt(bucketSize int, nOutCols int) *hashTableInt { | ||
| values := make([]intColumn, nOutCols) | ||
| for i := 0; i < nOutCols; i++ { | ||
| values[i] = make(intColumn, 1) | ||
| } | ||
|
|
||
| hashTable := &hashTableInt{ | ||
| first: make([]int, bucketSize), | ||
| next: make([]int, 1), | ||
| keys: make([]int, 1), | ||
| values: values, | ||
| nOutCols: nOutCols, | ||
| bucketSize: bucketSize, | ||
| allocated: 1, | ||
| } | ||
|
|
||
| return hashTable | ||
| } | ||
|
|
||
| func (hashTable *hashTableInt) hashInt(key int) int { | ||
| // todo(changangela) replace with a better hash function or use a library | ||
| return key % hashTable.bucketSize | ||
| } | ||
|
|
||
| func (hashTable *hashTableInt) grow(amount int) { | ||
| hashTable.next = append(hashTable.next, make([]int, amount)...) | ||
| hashTable.keys = append(hashTable.keys, make(intColumn, amount)...) | ||
|
Owner
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 may cause multiple allocations. Depending on whether you want fine-grained control over how much the slice will grow, I think the right way to do this is to allocate a new slice if the old's slice capacity is too small to fit the new amount, and copy the old slice into the new slice... I hear Go 1.11 is more optimal for this case (https://go-review.googlesource.com/c/go/+/109517) but unfortunately we're still on 1.10 for other reasons. If this is your bottleneck i'd consider changing it, otherwise I guess we can leave for now.
Author
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. Yeah that makes sense, the implementation's a little different with the |
||
|
|
||
| for i := 0; i < hashTable.nOutCols; i++ { | ||
| hashTable.values[i] = append(hashTable.values[i], make(intColumn, amount)...) | ||
| } | ||
| hashTable.allocated += amount | ||
| } | ||
|
|
||
| func (hashTable *hashTableInt) insert(hashkey int, key int) (id int) { | ||
| hashTable.count++ | ||
|
|
||
| id = hashTable.count | ||
| hashTable.next[id] = hashTable.first[hashkey] | ||
| hashTable.first[hashkey] = id | ||
| hashTable.keys[id] = key | ||
|
|
||
| return id | ||
| } | ||
|
|
||
| type hashJoinBuilder struct { | ||
| hashTable *hashTableInt | ||
|
|
||
| batchSize int | ||
|
|
||
| bucket []int | ||
| groupId []int | ||
| } | ||
|
|
||
| func makeHashJoinBuilder(batchSize int, hashTable *hashTableInt) *hashJoinBuilder { | ||
| return &hashJoinBuilder{ | ||
| hashTable: hashTable, | ||
| batchSize: batchSize, | ||
| bucket: make([]int, batchSize), | ||
| groupId: make([]int, batchSize), | ||
| } | ||
| } | ||
|
|
||
| func (builder *hashJoinBuilder) computeBucket(eqCol intColumn) { | ||
| for i := 0; i < builder.batchSize; i++ { | ||
| builder.bucket[i] = builder.hashTable.hashInt(eqCol[i]) | ||
| } | ||
| } | ||
|
|
||
| func (builder *hashJoinBuilder) insertBatch(eqCol intColumn) { | ||
| builder.hashTable.grow(builder.batchSize) | ||
|
|
||
| for i := 0; i < builder.batchSize; i++ { | ||
| builder.groupId[i] = builder.hashTable.insert(builder.bucket[i], eqCol[i]) | ||
| } | ||
| } | ||
|
|
||
| func (builder *hashJoinBuilder) spread(outCol intColumn, valIdx int) { | ||
| valCol := builder.hashTable.values[valIdx] | ||
|
|
||
| for i:= 0; i < builder.batchSize; i++ { | ||
| valCol[builder.groupId[i]] = outCol[i] | ||
| } | ||
| } | ||
|
|
||
| func (builder *hashJoinBuilder) start(flow dataFlow, eqColIdx int, outCols []int) { | ||
| eqCol := flow.b[eqColIdx].(intColumn)[:batchRowLen] | ||
|
|
||
| builder.computeBucket(eqCol) | ||
| builder.insertBatch(eqCol) | ||
|
|
||
| for idx, colIdx := range outCols { | ||
| outCol := flow.b[colIdx].(intColumn)[:batchRowLen] | ||
| builder.spread(outCol, idx) | ||
| } | ||
| } | ||
|
|
||
| type hashJoinProber struct { | ||
| hashTable *hashTableInt | ||
|
|
||
| batchSize int | ||
|
|
||
| bucket []int | ||
| groupId []int | ||
| toCheck []int | ||
| } | ||
|
|
||
| func makeHashJoinProber(batchSize int, hashTable *hashTableInt) *hashJoinProber { | ||
| return &hashJoinProber{ | ||
| hashTable: hashTable, | ||
| batchSize: batchSize, | ||
|
|
||
| bucket: make([]int, batchSize), | ||
| groupId: make([]int, batchSize), | ||
| toCheck: make([]int, batchSize), | ||
| } | ||
| } | ||
|
|
||
| func (prober *hashJoinProber) computeBucket(eqCol intColumn) { | ||
| for i := 0; i < prober.batchSize; i++ { | ||
| prober.bucket[i] = prober.hashTable.hashInt(eqCol[i]) | ||
| } | ||
| } | ||
|
|
||
| func (prober *hashJoinProber) lookupInitial() { | ||
| for i := 0; i < prober.batchSize; i++ { | ||
| prober.groupId[i] = prober.hashTable.first[prober.bucket[i]] | ||
| prober.toCheck[i] = i | ||
| } | ||
| } | ||
|
|
||
| func (prober *hashJoinProber) check(eqCol intColumn) int { | ||
| nToCheck := len(prober.toCheck) | ||
| differs := make([]int, 0) | ||
|
|
||
| for i := 0; i < nToCheck; i++ { | ||
| // id of 0 is reversed to represent end of next chain | ||
| if id := prober.groupId[prober.toCheck[i]]; id != 0 { | ||
| if prober.hashTable.keys[id] != eqCol[prober.toCheck[i]] { | ||
| differs = append(differs, prober.toCheck[i]) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| prober.toCheck = differs | ||
|
|
||
| return len(prober.toCheck) | ||
| } | ||
|
|
||
| func (prober *hashJoinProber) findNext(nToCheck int) { | ||
| for i := 0; i < nToCheck; i++ { | ||
| prober.groupId[prober.toCheck[i]] = prober.hashTable.next[prober.groupId[prober.toCheck[i]]] | ||
| } | ||
| } | ||
|
|
||
| func (prober *hashJoinProber) start(flow dataFlow, eqColIdx int, outCols []int, outFlow *dataFlow) { | ||
| eqCol := flow.b[eqColIdx].(intColumn)[:batchRowLen] | ||
|
|
||
| prober.computeBucket(eqCol) | ||
| prober.lookupInitial() | ||
|
|
||
| nToCheck := len(prober.toCheck) | ||
|
|
||
| for nToCheck > 0 { | ||
| nToCheck = prober.check(eqCol) | ||
| prober.findNext(nToCheck) | ||
| } | ||
|
|
||
| groupId := prober.groupId | ||
| results := make([]int, 0) | ||
| resultsIdx := make([]int, 0) | ||
|
|
||
| for i := 0; i < flow.n; i++ { | ||
| if groupId[i] != 0 { | ||
| results = append(results, groupId[i]) | ||
| resultsIdx = append(resultsIdx, i) | ||
| } | ||
| } | ||
|
|
||
| nBuildOutCols := prober.hashTable.nOutCols | ||
| nResults := len(results) | ||
|
|
||
| for j := 0; j < nBuildOutCols; j++ { | ||
| outCol := outFlow.b[j].(intColumn)[:batchRowLen] | ||
| valCol := prober.hashTable.values[j] | ||
|
|
||
| for i := 0; i < nResults; i++ { | ||
| outCol[i] = valCol[results[i]] | ||
| } | ||
| } | ||
|
|
||
| for j, colIdx := range outCols { | ||
| outCol := outFlow.b[j + nBuildOutCols].(intColumn)[:batchRowLen] | ||
| valCol := flow.b[colIdx].(intColumn)[:batchRowLen] | ||
|
|
||
| for i := 0; i < nResults; i++ { | ||
| outCol[i] = valCol[resultsIdx[i]] | ||
| } | ||
| } | ||
|
|
||
| outFlow.n = nResults | ||
| } | ||
|
|
||
| type hashJoinIntInt struct { | ||
| hashTable *hashTableInt | ||
|
|
||
| left ExecOp | ||
| right ExecOp | ||
|
|
||
| leftEqColIdx int | ||
| rightEqColIdx int | ||
|
|
||
| leftCols []int | ||
| rightCols []int | ||
|
|
||
| outFlow dataFlow | ||
| } | ||
|
|
||
| func (hashJoiner *hashJoinIntInt) Init() { | ||
| nOutputCols := len(hashJoiner.leftCols) + len(hashJoiner.rightCols) | ||
| if nOutputCols == 0 { | ||
| panic("no output cols") | ||
| } | ||
|
|
||
| hashJoiner.outFlow.b = make(batch, nOutputCols) | ||
| for i := range hashJoiner.outFlow.b { | ||
| hashJoiner.outFlow.b[i] = make(intColumn, batchRowLen) | ||
| } | ||
|
|
||
| // todo(changangela): we currently assume build relation has unique join | ||
| // column and is on the left | ||
|
|
||
| hashJoiner.hashTable = makeHashTableInt(hashTableBucketSize, len(hashJoiner.rightCols)) | ||
|
|
||
| hashJoiner.build() | ||
|
Owner
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 doesn't belong in |
||
| } | ||
|
|
||
| func (hashJoiner *hashJoinIntInt) Next() dataFlow { | ||
| flow := hashJoiner.right.Next() | ||
| batchSize := flow.n | ||
|
|
||
| if batchSize == 0 { | ||
| return flow | ||
| } | ||
|
|
||
| batchProber := makeHashJoinProber(batchSize, hashJoiner.hashTable) | ||
| batchProber.start(flow, hashJoiner.rightEqColIdx, hashJoiner.rightCols, &hashJoiner.outFlow) | ||
|
|
||
| return hashJoiner.outFlow | ||
| } | ||
|
|
||
| func (hashJoiner *hashJoinIntInt) build() { | ||
| for { | ||
| flow := hashJoiner.left.Next() | ||
| batchSize := flow.n | ||
|
|
||
| if batchSize == 0 { | ||
| return | ||
| } | ||
|
|
||
| batchBuilder := makeHashJoinBuilder(batchSize, hashJoiner.hashTable) | ||
| batchBuilder.start(flow, hashJoiner.leftEqColIdx, hashJoiner.leftCols) | ||
| } | ||
| } | ||
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.
I know the other code in this repo isn't well commented, but we'll need to add comments to all of these fields once we productionize it - otherwise, it'll be impossible for people to understand what's going on.
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.
You should describe the contract of
hashTableIntas well. How is it used? What does it do exactly? At least a few sentences would be helpful.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.
Specifically, what's first and next? How do they work? What's the overall structure of the hash table, what guarantees does it provide?
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.
Thanks for the review! I've started migrating this code out to cockroachdb and I promise there is better documentation in there 😆