Skip to content
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

Merged
merged 23 commits into from
Sep 5, 2018
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
0cc4ab9
batch copy init
crazycs520 Aug 25, 2018
2592034
add comment
crazycs520 Aug 28, 2018
265dd58
add batch copy test and benchmark
crazycs520 Aug 28, 2018
39cde9c
refine code
crazycs520 Aug 28, 2018
7b19c1c
fix bugs
crazycs520 Aug 28, 2018
e312fa4
Merge branch 'master' of https://github.com/pingcap/tidb into only-ba…
crazycs520 Aug 28, 2018
4022038
Merge branch 'master' of https://github.com/pingcap/tidb into only-ba…
crazycs520 Aug 29, 2018
9b6ce0f
Merge branch 'master' of https://github.com/pingcap/tidb into only-ba…
crazycs520 Sep 2, 2018
315c1fa
refactor code and comment
crazycs520 Sep 3, 2018
37902a2
refine code
crazycs520 Sep 3, 2018
c4556b4
address comment
crazycs520 Sep 3, 2018
cefae06
refactor code and comment
crazycs520 Sep 3, 2018
9fe55b3
refactor code and comment
crazycs520 Sep 4, 2018
91366cd
address comment
crazycs520 Sep 4, 2018
d3f8e85
address comment
crazycs520 Sep 4, 2018
026be7f
Merge branch 'master' of https://github.com/pingcap/tidb into only-ba…
crazycs520 Sep 4, 2018
50a698d
address comment
crazycs520 Sep 4, 2018
28018f9
address comment
crazycs520 Sep 4, 2018
2d04676
Merge branch 'master' into only-batch-copy
zz-jason Sep 4, 2018
e47fb7c
Merge branch 'master' into only-batch-copy
zz-jason Sep 5, 2018
3208738
refine comments
crazycs520 Sep 5, 2018
cd6eda1
Merge branch 'master' of https://github.com/pingcap/tidb into only-ba…
crazycs520 Sep 5, 2018
cedcd0e
Merge branch 'only-batch-copy' of https://github.com/crazycs520/tidb …
crazycs520 Sep 5, 2018
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
39 changes: 22 additions & 17 deletions executor/joiner.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,19 +158,12 @@ func (j *baseJoiner) makeShallowJoinRow(isRightJoin bool, inner, outer chunk.Row
j.shallowRow.ShallowCopyPartialRow(inner.Len(), outer)
}

func (j *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) {
func (j *baseJoiner) filter(input, output *chunk.Chunk) (err error) {
Copy link
Member

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 in j.selected.

j.selected, err = expression.VectorizedFilter(j.ctx, j.conditions, chunk.NewIterator4Chunk(input), j.selected)
if err != nil {
return false, errors.Trace(err)
}
for i := 0; i < len(j.selected); i++ {
if !j.selected[i] {
continue
}
matched = true
output.AppendRow(input.GetRow(i))
return errors.Trace(err)
}
return matched, nil
return nil
}

type semiJoiner struct {
Expand Down Expand Up @@ -350,8 +343,12 @@ func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk
}

// reach here, chkForJoin is j.chk
matched, err := j.filter(chkForJoin, chk)
return matched, errors.Trace(err)
err := j.filter(chkForJoin, chk)
if err != nil {
return false, errors.Trace(err)
}
matched := chk.BatchCopyJoinRowToChunk(false, chkForJoin, outer, j.selected)
return matched, nil
}

func (j *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) {
Expand Down Expand Up @@ -384,9 +381,12 @@ func (j *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, ch
return true, nil
}

// reach here, chkForJoin is j.chk
matched, err := j.filter(chkForJoin, chk)
return matched, errors.Trace(err)
err := j.filter(chkForJoin, chk)
if err != nil {
return false, errors.Trace(err)
}
matched := chk.BatchCopyJoinRowToChunk(true, chkForJoin, outer, j.selected)
return matched, nil
}

func (j *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) {
Expand Down Expand Up @@ -421,8 +421,13 @@ func (j *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *ch
}

// reach here, chkForJoin is j.chk
matched, err := j.filter(chkForJoin, chk)
return matched, errors.Trace(err)
err := j.filter(chkForJoin, chk)
if err != nil {
return false, errors.Trace(err)
}
matched := chk.BatchCopyJoinRowToChunk(j.outerIsRight, chkForJoin, outer, j.selected)
return matched, nil

}

func (j *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) {
Expand Down
71 changes: 71 additions & 0 deletions util/chunk/chunk.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Copy link
Member

Choose a reason for hiding this comment

The 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)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we cann't ...
There is a special optimizer for copy outer row.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, chkForJoin shares the same schema with c

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 {
Expand Down
65 changes: 65 additions & 0 deletions util/chunk/chunk_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"bytes"
"fmt"
"math"
"reflect"
"strconv"
"testing"
"time"
Expand Down Expand Up @@ -657,6 +658,70 @@ func BenchmarkChunkMemoryUsage(b *testing.B) {
}
}

func getChk() (*Chunk, *Chunk, Row, int, []bool) {
numRows := 1024
srcChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16)
srcChk.Reset()
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).ToRow()
} else {
row = MutRowFromValues("abc", "abcdefg", j, 123, types.ZeroDatetime).ToRow()
}
srcChk.AppendPartialRow(0, row)
selected[j] = true
}
dstChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16)
outerRow := MutRowFromValues(123, types.ZeroDatetime).ToRow()

return srcChk, dstChk, outerRow, numRows, selected
}

func TestBatchCopyJoinRowToChunk(t *testing.T) {
srcChk, dstChk, outerRow, numRows, selected := getChk()
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)
dstChk2.BatchCopyJoinRowToChunk(true, srcChk, outerRow, selected)

if !reflect.DeepEqual(dstChk, dstChk2) {
t.Fatal()
}
}

func BenchmarkChunkBatchCopyJoinRow(b *testing.B) {
b.ReportAllocs()
srcChk, dstChk, outerRow, _, selected := getChk()
b.ResetTimer()
for i := 0; i < b.N; i++ {
dstChk.Reset()
dstChk.BatchCopyJoinRowToChunk(false, srcChk, outerRow, selected)
}
}

func BenchmarkChunkAppendRow(b *testing.B) {
b.ReportAllocs()
srcChk, dstChk, _, numRows, selected := getChk()
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))
}
}
}

type seqNumberGenerateExec struct {
seq int
genCountSize int
Expand Down
28 changes: 28 additions & 0 deletions util/chunk/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,34 @@ func (c *column) appendNullBitmap(on bool) {
}
}

func (c *column) appendMultiSameNullBitmap(on bool, num int) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ on/ notNull ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see appendNullBitmap also use on, so, both use on or notNull ?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I prefer to both use on.

Copy link
Contributor

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 function and its parameters.

l := ((c.length + num - 1) >> 3) - len(c.nullBitmap)
Copy link
Member

Choose a reason for hiding this comment

The 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:

s/l/numNewBytes/

for i := 0; i <= l; i++ {
c.nullBitmap = append(c.nullBitmap, 0)
Copy link
Member

Choose a reason for hiding this comment

The 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:

  1. set all the higher x bits of c.nullBitmap[len(c.nullBitmap)-1] to 0 or 1 according to the value of on.
  2. memset the new bytes to 0xFF or 0x00 according to the value of on.

}
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
Copy link
Member

@zz-jason zz-jason Sep 3, 2018

Choose a reason for hiding this comment

The 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 on is set to false, because this Chunk maybe is truncated, and the null bitmap is not reset in that scenario.

}
}

func (c *column) appendNull() {
c.appendNullBitmap(false)
if c.isFixed() {
Expand Down