-
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 8 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 |
---|---|---|
|
@@ -232,6 +232,77 @@ func (c *Chunk) AppendPartialRow(colIdx int, row Row) { | |
} | ||
} | ||
|
||
// BatchCopyJoinRowToChunk uses for join to batch copy inner rows and outer row to chunk. | ||
func (c *Chunk) BatchCopyJoinRowToChunk(isRight bool, chkForJoin *Chunk, outer Row, selected []bool) bool { | ||
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. Can this function be simplified to func CopySelectedRows(src *Chunk, selected []bool, dst *Chunk) 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. Maybe we cann't ... 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. No, |
||
if chkForJoin.NumRows() == 0 { | ||
return false | ||
} | ||
|
||
innerIdx, outerIdx := 0, chkForJoin.NumCols()-outer.Len() | ||
if !isRight { | ||
innerIdx, outerIdx = outer.Len(), 0 | ||
} | ||
rowNum := appendPartialRows(innerIdx, outerIdx, chkForJoin, c, selected) | ||
appendPartialSameRows(outerIdx, outer, rowNum, c) | ||
c.numVirtualRows += rowNum | ||
return rowNum > 0 | ||
} | ||
|
||
// appendPartialRows appends multiple different rows to the chunk. | ||
func appendPartialRows(colIdx, outerIdx int, chkForJoin, chk *Chunk, selected []bool) int { | ||
oldLen := chk.columns[colIdx].length | ||
var columns []*column | ||
if colIdx == 0 { | ||
columns = chkForJoin.columns[:outerIdx] | ||
} else { | ||
columns = chkForJoin.columns[colIdx:] | ||
} | ||
for j, rowCol := range columns { | ||
chkCol := chk.columns[colIdx+j] | ||
for i := 0; i < len(selected); i++ { | ||
if !selected[i] { | ||
continue | ||
} | ||
chkCol.appendNullBitmap(!rowCol.isNull(i)) | ||
chkCol.length++ | ||
|
||
if rowCol.isFixed() { | ||
elemLen := len(rowCol.elemBuf) | ||
offset := i * elemLen | ||
chkCol.data = append(chkCol.data, rowCol.data[offset:offset+elemLen]...) | ||
} else { | ||
start, end := rowCol.offsets[i], rowCol.offsets[i+1] | ||
chkCol.data = append(chkCol.data, rowCol.data[start:end]...) | ||
chkCol.offsets = append(chkCol.offsets, int32(len(chkCol.data))) | ||
} | ||
} | ||
} | ||
return chk.columns[colIdx].length - oldLen | ||
} | ||
|
||
// appendPartialSameRows appends same row to the chunk with `rowNum` times. | ||
func appendPartialSameRows(colIdx int, row Row, rowNum int, c *Chunk) { | ||
for i, rowCol := range row.c.columns { | ||
chkCol := c.columns[colIdx+i] | ||
chkCol.appendMultiSameNullBitmap(!rowCol.isNull(row.idx), rowNum) | ||
chkCol.length += rowNum | ||
if rowCol.isFixed() { | ||
elemLen := len(rowCol.elemBuf) | ||
start := row.idx * elemLen | ||
end := start + elemLen | ||
for j := 0; j < rowNum; j++ { | ||
chkCol.data = append(chkCol.data, rowCol.data[start:end]...) | ||
} | ||
} else { | ||
start, end := rowCol.offsets[row.idx], rowCol.offsets[row.idx+1] | ||
for j := 0; j < rowNum; j++ { | ||
chkCol.data = append(chkCol.data, rowCol.data[start:end]...) | ||
chkCol.offsets = append(chkCol.offsets, int32(len(chkCol.data))) | ||
} | ||
} | ||
} | ||
} | ||
|
||
// Append appends rows in [begin, end) in another Chunk to a Chunk. | ||
func (c *Chunk) Append(other *Chunk, begin, end int) { | ||
for colID, src := range other.columns { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -85,6 +85,34 @@ func (c *column) appendNullBitmap(on bool) { | |
} | ||
} | ||
|
||
func (c *column) appendMultiSameNullBitmap(on bool, num int) { | ||
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/ on/ notNull ? 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 see 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 prefer to both use 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 function and its parameters. |
||
l := ((c.length + num - 1) >> 3) - len(c.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. it should be: l := ((c.length + num + 7) >> 3) - len(c.nullBitmap) how about:
|
||
for i := 0; i <= l; i++ { | ||
c.nullBitmap = append(c.nullBitmap, 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. Maybe it's much clear and easier to understand if we change the copy strategy to:
|
||
} | ||
if on { | ||
idx := c.length >> 3 | ||
pos := uint(c.length) & 7 | ||
for num > 0 { | ||
if pos == 0 && num > 8 { | ||
c.nullBitmap[idx] = 0xff | ||
idx++ | ||
num = num - 8 | ||
} else { | ||
c.nullBitmap[idx] |= byte(1 << pos) | ||
pos++ | ||
num-- | ||
if pos == 8 { | ||
pos = 0 | ||
idx++ | ||
} | ||
} | ||
} | ||
} else { | ||
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 |
||
} | ||
} | ||
|
||
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.
Can we make
j.selected
as a return value for this function? The function caller can directly use the returned[]bool
, no need to dig into this function to know that the result is stored inj.selected
.