Skip to content

Commit

Permalink
refine code
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 committed Sep 3, 2018
1 parent 315c1fa commit 37902a2
Show file tree
Hide file tree
Showing 3 changed files with 34 additions and 25 deletions.
33 changes: 16 additions & 17 deletions executor/joiner.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,16 +158,18 @@ func (j *baseJoiner) makeShallowJoinRow(isRightJoin bool, inner, outer chunk.Row
j.shallowRow.ShallowCopyPartialRow(inner.Len(), outer)
}

func (j *baseJoiner) filter(input, output *chunk.Chunk) ([]bool, error) {
return expression.VectorizedFilter(j.ctx, j.conditions, chunk.NewIterator4Chunk(input), j.selected)
}

func (j *baseJoiner) BatchCopyJoinRowToChunk(outer chunk.Row, src, dst *chunk.Chunk) bool {
innerColOffset, outerColOffset := 0, src.NumCols()-outer.Len()
func (j *baseJoiner) filter(input, output *chunk.Chunk, outerColsLen int) (bool, error) {
var err error
j.selected, err = expression.VectorizedFilter(j.ctx, j.conditions, chunk.NewIterator4Chunk(input), j.selected)
if err != nil {
return false, errors.Trace(err)
}
// batch copy selected row to output chunk
innerColOffset, outerColOffset := 0, input.NumCols()-outerColsLen
if !j.outerIsRight {
innerColOffset, outerColOffset = outer.Len(), 0
innerColOffset, outerColOffset = outerColsLen, 0
}
return chunk.CopySelectedJoinRows(src, innerColOffset, outerColOffset, j.selected, dst)
return chunk.CopySelectedJoinRows(input, innerColOffset, outerColOffset, j.selected, output), nil
}

type semiJoiner struct {
Expand Down Expand Up @@ -327,7 +329,7 @@ type leftOuterJoiner struct {
}

// tryToMatch implements joiner interface.
func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) {
func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) {
if inners.Len() == 0 {
return false, nil
}
Expand All @@ -347,11 +349,10 @@ func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk
}

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

Expand All @@ -365,7 +366,7 @@ type rightOuterJoiner struct {
}

// tryToMatch implements joiner interface.
func (j *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) {
func (j *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) {
if inners.Len() == 0 {
return false, nil
}
Expand All @@ -385,11 +386,10 @@ func (j *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, ch
return true, nil
}

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

Expand All @@ -403,7 +403,7 @@ type innerJoiner struct {
}

// tryToMatch implements joiner interface.
func (j *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) {
func (j *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) {
if inners.Len() == 0 {
return false, nil
}
Expand All @@ -425,11 +425,10 @@ func (j *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *ch
}

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

}
Expand Down
10 changes: 5 additions & 5 deletions util/chunk/chunk_util.go
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2017 PingCAP, Inc.
// 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.
Expand Down Expand Up @@ -27,14 +27,14 @@ func CopySelectedJoinRows(src *Chunk, innerColOffset, outerColOffset int, select
return selectedRowNum > 0
}

// appendInnerRows appends multiple different rows to the chunk.
func appendInnerRows(innerColOffset, outerColOffset int, chkForJoin *Chunk, selected []bool, dst *Chunk) int {
// appendInnerRows appends different inner rows to the chunk.
func appendInnerRows(innerColOffset, outerColOffset int, src *Chunk, selected []bool, dst *Chunk) int {
oldLen := dst.columns[innerColOffset].length
var columns []*column
if innerColOffset == 0 {
columns = chkForJoin.columns[:outerColOffset]
columns = src.columns[:outerColOffset]
} else {
columns = chkForJoin.columns[innerColOffset:]
columns = src.columns[innerColOffset:]
}
for j, rowCol := range columns {
chkCol := dst.columns[innerColOffset+j]
Expand Down
16 changes: 13 additions & 3 deletions util/chunk/chunk_util_test.go
Original file line number Diff line number Diff line change
@@ -1,3 +1,16 @@
// 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 (
Expand All @@ -10,7 +23,6 @@ import (
func getChk() (*Chunk, *Chunk, []bool) {
numRows := 1024
srcChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0)
srcChk.Reset()
selected := make([]bool, numRows)
var row Row
for j := 0; j < numRows; j++ {
Expand All @@ -23,7 +35,6 @@ func getChk() (*Chunk, *Chunk, []bool) {
srcChk.AppendPartialRow(0, row)
}
dstChk := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0)

return srcChk, dstChk, selected
}

Expand All @@ -36,7 +47,6 @@ func TestBatchCopyJoinRowToChunk(t *testing.T) {
}
dstChk.AppendRow(srcChk.GetRow(i))
}

// batch copy
dstChk2 := newChunkWithInitCap(numRows, 0, 0, 8, 8, 16, 0)
CopySelectedJoinRows(srcChk, 0, 3, selected, dstChk2)
Expand Down

0 comments on commit 37902a2

Please sign in to comment.