-
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 batch copy to inner join, left and right outer join. #7493
Changes from 15 commits
0cc4ab9
2592034
265dd58
39cde9c
7b19c1c
e312fa4
4022038
9b6ce0f
315c1fa
37902a2
c4556b4
cefae06
9fe55b3
91366cd
d3f8e85
026be7f
50a698d
28018f9
2d04676
e47fb7c
3208738
cd6eda1
cedcd0e
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,106 @@ | ||
// Copyright 2018 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 chunk | ||
|
||
// CopySelectedJoinRows copies the selected joined rows from the source Chunk | ||
// to the destination Chunk. | ||
// Return true if at least one joined row was selected. | ||
// | ||
// NOTE: All the outer rows in the source Chunk should be the same. | ||
func CopySelectedJoinRows(src *Chunk, innerColOffset, outerColOffset int, selected []bool, dst *Chunk) bool { | ||
if src.NumRows() == 0 { | ||
return false | ||
} | ||
|
||
numSelected := copySelectedInnerRows(innerColOffset, outerColOffset, src, selected, dst) | ||
copyOuterRows(innerColOffset, outerColOffset, src, numSelected, dst) | ||
dst.numVirtualRows += numSelected | ||
return numSelected > 0 | ||
} | ||
|
||
// copySelectedInnerRows copies the selected inner rows from the source Chunk | ||
// to the destination Chunk. | ||
func copySelectedInnerRows(innerColOffset, outerColOffset int, src *Chunk, selected []bool, dst *Chunk) int { | ||
oldLen := dst.columns[innerColOffset].length | ||
var srcCols []*column | ||
if innerColOffset == 0 { | ||
srcCols = src.columns[:outerColOffset] | ||
} else { | ||
srcCols = src.columns[innerColOffset:] | ||
} | ||
for j, srcCol := range srcCols { | ||
dstCol := dst.columns[innerColOffset+j] | ||
if srcCol.isFixed() { | ||
for i := 0; i < len(selected); i++ { | ||
if !selected[i] { | ||
continue | ||
} | ||
dstCol.appendNullBitmap(!srcCol.isNull(i)) | ||
dstCol.length++ | ||
|
||
elemLen := len(srcCol.elemBuf) | ||
offset := i * elemLen | ||
dstCol.data = append(dstCol.data, srcCol.data[offset:offset+elemLen]...) | ||
} | ||
} else { | ||
for i := 0; i < len(selected); i++ { | ||
if !selected[i] { | ||
continue | ||
} | ||
dstCol.appendNullBitmap(!srcCol.isNull(i)) | ||
dstCol.length++ | ||
|
||
start, end := srcCol.offsets[i], srcCol.offsets[i+1] | ||
dstCol.data = append(dstCol.data, srcCol.data[start:end]...) | ||
dstCol.offsets = append(dstCol.offsets, int32(len(dstCol.data))) | ||
} | ||
} | ||
} | ||
return dst.columns[innerColOffset].length - oldLen | ||
} | ||
|
||
// copyOuterRows copies the continuous 'numRows' outer rows in the source Chunk | ||
// to the destination Chunk. | ||
func copyOuterRows(innerColOffset, outerColOffset int, src *Chunk, numRows int, dst *Chunk) { | ||
if numRows <= 0 { | ||
return | ||
} | ||
row := src.GetRow(0) | ||
var srcCols []*column | ||
if innerColOffset == 0 { | ||
srcCols = src.columns[outerColOffset:] | ||
} else { | ||
srcCols = src.columns[:innerColOffset] | ||
} | ||
for i, srcCol := range srcCols { | ||
dstCol := dst.columns[outerColOffset+i] | ||
dstCol.appendMultiSameNullBitmap(!srcCol.isNull(row.idx), numRows) | ||
dstCol.length += numRows | ||
if srcCol.isFixed() { | ||
elemLen := len(srcCol.elemBuf) | ||
start := row.idx * elemLen | ||
end := start + numRows*elemLen | ||
dstCol.data = append(dstCol.data, srcCol.data[start:end]...) | ||
} else { | ||
start, end := srcCol.offsets[row.idx], srcCol.offsets[row.idx+numRows] | ||
dstCol.data = append(dstCol.data, srcCol.data[start:end]...) | ||
offsets := dstCol.offsets | ||
elemLen := srcCol.offsets[row.idx+1] - srcCol.offsets[row.idx] | ||
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. Will it be possible that 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. In the normal case, it will not out of range. 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. Will it be possible that 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. if 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. ok |
||
for j := 0; j < numRows; j++ { | ||
offsets = append(offsets, int32(offsets[len(offsets)-1]+elemLen)) | ||
} | ||
dstCol.offsets = offsets | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
// Copyright 2018 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 chunk | ||
|
||
import ( | ||
"reflect" | ||
"testing" | ||
|
||
"github.com/pingcap/tidb/types" | ||
) | ||
|
||
func getChk() (*Chunk, *Chunk, []bool) { | ||
numRows := 1024 | ||
srcChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0) | ||
selected := make([]bool, numRows) | ||
var row Row | ||
for j := 0; j < numRows; j++ { | ||
if j%7 == 0 { | ||
row = MutRowFromValues("abc", "abcdefg", nil, 123, types.ZeroDatetime, "abcdefg").ToRow() | ||
} else { | ||
row = MutRowFromValues("abc", "abcdefg", j, 123, types.ZeroDatetime, "abcdefg").ToRow() | ||
selected[j] = true | ||
} | ||
srcChk.AppendPartialRow(0, row) | ||
} | ||
dstChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0) | ||
return srcChk, dstChk, selected | ||
} | ||
|
||
func TestCopySelectedJoinRows(t *testing.T) { | ||
srcChk, dstChk, selected := getChk() | ||
numRows := srcChk.NumRows() | ||
for i := 0; i < numRows; i++ { | ||
if !selected[i] { | ||
continue | ||
} | ||
dstChk.AppendRow(srcChk.GetRow(i)) | ||
} | ||
// batch copy | ||
dstChk2 := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0) | ||
CopySelectedJoinRows(srcChk, 0, 3, selected, dstChk2) | ||
|
||
if !reflect.DeepEqual(dstChk, dstChk2) { | ||
t.Fatal() | ||
} | ||
} | ||
|
||
func BenchmarkCopySelectedJoinRows(b *testing.B) { | ||
b.ReportAllocs() | ||
srcChk, dstChk, selected := getChk() | ||
b.ResetTimer() | ||
for i := 0; i < b.N; i++ { | ||
dstChk.Reset() | ||
CopySelectedJoinRows(srcChk, 0, 3, selected, dstChk) | ||
} | ||
} | ||
|
||
func BenchmarkAppendSelectedRow(b *testing.B) { | ||
b.ReportAllocs() | ||
srcChk, dstChk, selected := getChk() | ||
numRows := srcChk.NumRows() | ||
b.ResetTimer() | ||
for i := 0; i < b.N; i++ { | ||
dstChk.Reset() | ||
for j := 0; j < numRows; j++ { | ||
if !selected[j] { | ||
continue | ||
} | ||
dstChk.AppendRow(srcChk.GetRow(j)) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -72,19 +72,45 @@ func (c *column) isNull(rowIdx int) bool { | |
return nullByte&(1<<(uint(rowIdx)&7)) == 0 | ||
} | ||
|
||
func (c *column) appendNullBitmap(on bool) { | ||
func (c *column) appendNullBitmap(notNull bool) { | ||
idx := c.length >> 3 | ||
if idx >= len(c.nullBitmap) { | ||
c.nullBitmap = append(c.nullBitmap, 0) | ||
} | ||
if on { | ||
if notNull { | ||
pos := uint(c.length) & 7 | ||
c.nullBitmap[idx] |= byte(1 << pos) | ||
} else { | ||
c.nullCount++ | ||
} | ||
} | ||
|
||
// appendMultiSameNullBitmap appends multiple same bit value to `nullBitMap`. | ||
// notNull mean not not null. | ||
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. s/ mean/ means |
||
// num mean appends `num` bit value to `nullBitMap`. | ||
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. num means the number of bits that should be appended. |
||
func (c *column) appendMultiSameNullBitmap(notNull bool, num int) { | ||
numNewBytes := ((c.length + num + 7) >> 3) - len(c.nullBitmap) | ||
b := byte(0) | ||
if notNull { | ||
b = 0xff | ||
} | ||
for i := 0; i < numNewBytes; i++ { | ||
c.nullBitmap = append(c.nullBitmap, b) | ||
} | ||
if !notNull { | ||
c.nullCount += num | ||
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. we also need to set all the existing bits to zero even if |
||
return | ||
} | ||
// 1. Set all the higher 8-'numOldBits' bits in the last old byte to 1. | ||
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.
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. done. 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. wait, I ask @CaitinChen 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. 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. @crazycs520 I prefer "the remaining bits" |
||
numOldBits := uint(c.length % 8) | ||
bitMask := byte(^((1 << numOldBits) - 1)) | ||
c.nullBitmap[c.length/8] |= bitMask | ||
// 2. Set all the higher 'numRedundantBits' bits in the last new byte to 0. | ||
numRedundantBits := uint(len(c.nullBitmap)*8 - c.length - num) | ||
bitMask = byte(1<<(8-numRedundantBits)) - 1 | ||
c.nullBitmap[len(c.nullBitmap)-1] &= bitMask | ||
} | ||
|
||
func (c *column) appendNull() { | ||
c.appendNullBitmap(false) | ||
if c.isFixed() { | ||
|
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 the return value.