From 0207e9159aabd89b0d8d5d0a2a082b2399061ce8 Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Mon, 6 Aug 2018 18:58:42 +0800 Subject: [PATCH 01/10] introduce onMissMatch --- executor/index_lookup_join.go | 22 +-- executor/join.go | 44 +++--- executor/join_result_generators.go | 224 ++++++++++++++++------------- executor/merge_join.go | 14 +- 4 files changed, 170 insertions(+), 134 deletions(-) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 29024d1c2e622..a68d69aa66ec4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -90,8 +90,9 @@ type lookUpJoinTask struct { lookupMap *mvmap.MVMap matchedInners []chunk.Row - doneCh chan error - cursor int + doneCh chan error + cursor int + hasMatch bool memTracker *memory.Tracker // track memory usage. } @@ -205,16 +206,19 @@ func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { } outerRow := task.outerResult.GetRow(task.cursor) - if e.innerIter.Len() == 0 { - err = e.resultGenerator.emit(outerRow, nil, chk) - } else if e.innerIter.Current() != e.innerIter.End() { - err = e.resultGenerator.emit(outerRow, e.innerIter, chk) - } - if err != nil { - return errors.Trace(err) + if e.innerIter.Current() != e.innerIter.End() { + matched, err := e.resultGenerator.tryToMatch(outerRow, e.innerIter, chk) + if err != nil { + return errors.Trace(err) + } + task.hasMatch = task.hasMatch || matched } if e.innerIter.Current() == e.innerIter.End() { + if !task.hasMatch { + e.resultGenerator.onMissMatch(outerRow, chk) + } task.cursor++ + task.hasMatch = false } if chk.NumRows() == e.maxChunkSize { return nil diff --git a/executor/join.go b/executor/join.go index 9b10a1aefe61f..f7e8eba8c8e57 100644 --- a/executor/join.go +++ b/executor/join.go @@ -385,20 +385,14 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R return false, joinResult } if hasNull { - err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - } - return err == nil, joinResult + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + return true, joinResult } e.hashTableValBufs[workerID] = e.hashTable.Get(joinKey, e.hashTableValBufs[workerID][:0]) innerPtrs := e.hashTableValBufs[workerID] if len(innerPtrs) == 0 { - err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - } - return err == nil, joinResult + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + return true, joinResult } innerRows := make([]chunk.Row, 0, len(innerPtrs)) for _, b := range innerPtrs { @@ -407,12 +401,15 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R innerRows = append(innerRows, matchedInner) } iter := chunk.NewIterator4Slice(innerRows) + hasMatch := false for iter.Begin(); iter.Current() != iter.End(); { - err = e.resultGenerators[workerID].emit(outerRow, iter, joinResult.chk) + matched, err := e.resultGenerators[workerID].tryToMatch(outerRow, iter, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) return false, joinResult } + hasMatch = hasMatch || matched + if joinResult.chk.NumRows() == e.maxChunkSize { ok := true e.joinResultCh <- joinResult @@ -422,6 +419,9 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R } } } + if !hasMatch { + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + } return true, joinResult } @@ -448,11 +448,7 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu } for i := range selected { if !selected[i] { // process unmatched outer rows - err = e.resultGenerators[workerID].emit(outerChk.GetRow(i), nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - return false, joinResult - } + e.resultGenerators[workerID].onMissMatch(outerChk.GetRow(i), joinResult.chk) } else { // process matched outer rows ok, joinResult = e.joinMatchedOuterRow2Chunk(workerID, outerChk.GetRow(i), joinResult) if !ok { @@ -578,6 +574,7 @@ type NestedLoopApplyExec struct { innerSelected []bool innerIter chunk.Iterator outerRow *chunk.Row + hasMatch bool memTracker *memory.Tracker // track memory usage. } @@ -635,9 +632,9 @@ func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *ch if selected { return &outerRow, nil } else if e.outer { - err := e.resultGenerator.emit(outerRow, nil, chk) - if err != nil || chk.NumRows() == e.maxChunkSize { - return nil, errors.Trace(err) + e.resultGenerator.onMissMatch(outerRow, chk) + if chk.NumRows() == e.maxChunkSize { + return nil, nil } } } @@ -678,10 +675,15 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e chk.Reset() for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { + if e.outerRow != nil && !e.hasMatch { + e.resultGenerator.onMissMatch(*e.outerRow, chk) + } e.outerRow, err = e.fetchSelectedOuterRow(ctx, chk) if e.outerRow == nil || err != nil { return errors.Trace(err) } + e.hasMatch = false + for _, col := range e.outerSchema { *col.Data = e.outerRow.GetDatum(col.Index, col.RetType) } @@ -693,7 +695,9 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e e.innerIter.Begin() } - err = e.resultGenerator.emit(*e.outerRow, e.innerIter, chk) + matched, err := e.resultGenerator.tryToMatch(*e.outerRow, e.innerIter, chk) + e.hasMatch = e.hasMatch || matched + if err != nil || chk.NumRows() == e.maxChunkSize { return errors.Trace(err) } diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 759561ee7ecd1..526a533291e3c 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -32,9 +32,23 @@ var ( _ joinResultGenerator = &innerJoinResultGenerator{} ) -// joinResultGenerator is used to generate join results according the join type, see every implementor for detailed information. +// joinResultGenerator is used to generate join results according the join +// type, see every implementor for detailed information. +// +// A typical instruction flow is: +// ```go +// hasMatch := false +// for innerIter.Current() != innerIter.End() { +// matched, err := g.tryToMatch(outer, innerIter, chk) +// // handle err +// hasMatch = hasMatch || matched +// } +// if !hasMatch { +// g.onMissMatch(outer) +// } +// ``` type joinResultGenerator interface { - // emit tries to join an outer row with a batch of inner rows. + // tryToMatch tries to join an outer row with a batch of inner rows. // When inners == nil or inners.Len() == 0, it means that the outer row can not be joined with any inner row: // 1. SemiJoin: unmatched outer row is ignored. // 2. AntiSemiJoin: unmatched outer row is appended to the result buffer. @@ -46,7 +60,9 @@ type joinResultGenerator interface { // When inners.Len != 0 but all the joined rows are filtered, this means that the outer row is unmatched and the above action is tacked as well. // Otherwise, the outer row is matched and some joined rows is appended to the `chk`. // The size of `chk` is MaxChunkSize at most. - emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error + tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) + + onMissMatch(outer chunk.Row, chk *chunk.Chunk) } func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, @@ -133,15 +149,15 @@ type semiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *semiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } defer inners.ReachEnd() if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) - return nil + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { @@ -153,29 +169,32 @@ func (outputer *semiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iter } selected, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } if selected { chk.AppendRow(outer) - return nil + return true, nil } } - return nil + return false, nil +} + +func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + return } type antiSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - chk.AppendRow(outer) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } defer inners.ReachEnd() if len(outputer.conditions) == 0 { - return nil + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { @@ -188,33 +207,40 @@ func (outputer *antiSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk. matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } if matched { - return nil + return true, nil } } + return false, nil +} + +func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) - return nil + return } type leftOuterSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (hasMatch bool, err error) { + if inners.Len() == 0 { + return false, nil } - - defer inners.ReachEnd() - if len(outputer.conditions) == 0 { + defer func() { + if !(err == nil && hasMatch) { + return + } + inners.ReachEnd() chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) - return nil + }() + + if len(outputer.conditions) == 0 { + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { @@ -222,147 +248,143 @@ func (outputer *leftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners c outputer.makeJoinRowToChunk(outputer.chk, outer, inner) matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } if matched { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - return nil + return true, nil } } + return false, nil +} + +func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) - return nil + return } type antiLeftOuterSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (hasMatch bool, err error) { + if inners.Len() == 0 { + return false, nil } - - defer inners.ReachEnd() - // outer row can be joined with an inner row. - if len(outputer.conditions) == 0 { + defer func() { + if !(err == nil && hasMatch) { + return + } + inners.ReachEnd() chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) - return nil + }() + + if len(outputer.conditions) == 0 { + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } - // outer row can be joined with an inner row. if matched { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - return nil + return true, nil } } - // outer row can not be joined with any inner row. + return false, nil +} + +func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) - return nil + return } type leftOuterJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } outputer.chk.Reset() chkForJoin := outputer.chk if len(outputer.conditions) == 0 { chkForJoin = chk } + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk matched, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() - if !matched { - // outer row can not be joined with any inner row. - chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) - } - return nil + return matched, errors.Trace(err) +} + +func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendPartialRow(outer.Len(), outputer.defaultInner) + return } type rightOuterJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } outputer.chk.Reset() chkForJoin := outputer.chk if len(outputer.conditions) == 0 { chkForJoin = chk } + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk matched, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() - // outer row can not be joined with any inner row. - if !matched { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) - } - return nil + return matched, errors.Trace(err) +} + +func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outputer.defaultInner) + chk.AppendPartialRow(outputer.defaultInner.Len(), outer) + return } type innerJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } outputer.chk.Reset() chkForJoin := outputer.chk @@ -378,14 +400,14 @@ func (outputer *innerJoinResultGenerator) emit(outer chunk.Row, inners chunk.Ite } } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk - _, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() + matched, err := outputer.filter(chkForJoin, chk) + return matched, errors.Trace(err) +} - return nil +func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + return } diff --git a/executor/merge_join.go b/executor/merge_join.go index d2373603e2924..1d23b19ef2274 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -58,8 +58,9 @@ type mergeJoinOuterTable struct { chk *chunk.Chunk selected []bool - iter *chunk.Iterator4Chunk - row chunk.Row + iter *chunk.Iterator4Chunk + row chunk.Row + hasMatch bool } // mergeJoinInnerTable represents the inner table of merge join. @@ -298,12 +299,13 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM } if cmpResult < 0 { - err = e.resultGenerator.emit(e.outerTable.row, nil, chk) + e.resultGenerator.onMissMatch(e.outerTable.row, chk) if err != nil { return false, errors.Trace(err) } e.outerTable.row = e.outerTable.iter.Next() + e.outerTable.hasMatch = false if chk.NumRows() == e.maxChunkSize { return true, nil @@ -311,12 +313,16 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM continue } - err = e.resultGenerator.emit(e.outerTable.row, e.innerIter4Row, chk) + matched, err := e.resultGenerator.tryToMatch(e.outerTable.row, e.innerIter4Row, chk) if err != nil { return false, errors.Trace(err) } + e.outerTable.hasMatch = e.outerTable.hasMatch || matched if e.innerIter4Row.Current() == e.innerIter4Row.End() { + if !e.outerTable.hasMatch { + e.resultGenerator.onMissMatch(e.outerTable.row, chk) + } e.outerTable.row = e.outerTable.iter.Next() e.innerIter4Row.Begin() } From 6b74d452d6feda7cd8c89a02f420d61a0dcd9f88 Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Mon, 6 Aug 2018 20:23:52 +0800 Subject: [PATCH 02/10] rename some variables --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 346 +++++++++++++++-------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 190 insertions(+), 178 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index cf5c37e58944c..a07a8d55fb1e5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]joinResultGenerator, e.concurrency) + e.resultGenerators = make([]recordJoiner, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a68d69aa66ec4..45634b4848c17 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator joinResultGenerator + resultGenerator recordJoiner indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index f7e8eba8c8e57..8161667dbce4a 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. - resultGenerators []joinResultGenerator + // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. + resultGenerators []recordJoiner outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator joinResultGenerator + resultGenerator recordJoiner outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 526a533291e3c..4f0030922086d 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,52 +23,66 @@ import ( ) var ( - _ joinResultGenerator = &semiJoinResultGenerator{} - _ joinResultGenerator = &antiSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &antiLeftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterJoinResultGenerator{} - _ joinResultGenerator = &rightOuterJoinResultGenerator{} - _ joinResultGenerator = &innerJoinResultGenerator{} + _ recordJoiner = &semiJoiner{} + _ recordJoiner = &antiSemiJoiner{} + _ recordJoiner = &leftOuterSemiJoiner{} + _ recordJoiner = &antiLeftOuterSemiJoiner{} + _ recordJoiner = &leftOuterJoiner{} + _ recordJoiner = &rightOuterJoiner{} + _ recordJoiner = &innerJoiner{} ) -// joinResultGenerator is used to generate join results according the join -// type, see every implementor for detailed information. -// +// recordJoiner is used to generate join results according the join type. // A typical instruction flow is: -// ```go -// hasMatch := false -// for innerIter.Current() != innerIter.End() { -// matched, err := g.tryToMatch(outer, innerIter, chk) -// // handle err -// hasMatch = hasMatch || matched -// } -// if !hasMatch { -// g.onMissMatch(outer) -// } -// ``` -type joinResultGenerator interface { - // tryToMatch tries to join an outer row with a batch of inner rows. - // When inners == nil or inners.Len() == 0, it means that the outer row can not be joined with any inner row: - // 1. SemiJoin: unmatched outer row is ignored. - // 2. AntiSemiJoin: unmatched outer row is appended to the result buffer. - // 3. LeftOuterSemiJoin: unmatched outer row is appended with 0 and appended to the result buffer. - // 4. AntiLeftOuterSemiJoin: unmatched outer row is appended with 1 and appended to the result buffer. - // 5. LeftOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. - // 6. RightOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. - // 7. InnerJoin: unmatched outer row is ignored. - // When inners.Len != 0 but all the joined rows are filtered, this means that the outer row is unmatched and the above action is tacked as well. - // Otherwise, the outer row is matched and some joined rows is appended to the `chk`. - // The size of `chk` is MaxChunkSize at most. +// +// hasMatch := false +// for innerIter.Current() != innerIter.End() { +// matched, err := g.tryToMatch(outer, innerIter, chk) +// // handle err +// hasMatch = hasMatch || matched +// } +// if !hasMatch { +// g.onMissMatch(outer) +// } +// +// NOTE: This interface is **not** thread-safe. +type recordJoiner interface { + // tryToMatch tries to join an outer row with a batch of inner rows. When + // 'inners.Len != 0' but all the joined rows are filtered, the outer row is + // considered unmatched. Otherwise, the outer row is matched and some joined + // rows is appended to `chk`. The size of `chk` is limited to MaxChunkSize. + // + // NOTE: Callers need to call this function multiple times to consume all + // the inner rows for an outer row, and dicide whether the outer row can be + // matched with at lease one inner row. tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) + // onMissMatch operates on the unmatched outer row according to the join + // type. An outer row can be considered miss matched if: + // 1. it can not pass the filter on the outer table side. + // 2. there does not exist an inner row with the same join key. + // 3. all the joined rows can not pass the filter on the join result. + // + // On these conditions, the caller calls this function to handle the + // unmatched outer rows according to the current join type: + // 1. 'SemiJoin': ignores the unmatched outer row. + // 2. 'AntiSemiJoin': appends the unmatched outer row to the result buffer. + // 3. 'LeftOuterSemiJoin': concates the unmatched outer row with 0 and + // appended it to the result buffer. + // 4. 'AntiLeftOuterSemiJoin': concates the unmatched outer row with 0 and + // appended to the result buffer. + // 5. 'LeftOuterJoin': concates the unmatched outer row with a row of NULLs + // and appended to the result buffer. + // 6. 'RightOuterJoin': concates the unmatched outer row with a row of NULLs + // and appended to the result buffer. + // 7. 'InnerJoin': ignores the unmatched outer row. onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, +func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { - base := baseJoinResultGenerator{ + lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { + base := baseJoiner{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -88,26 +102,24 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiJoinResultGenerator{base} + return &semiJoiner{base} case plan.AntiSemiJoin: - return &antiSemiJoinResultGenerator{base} + return &antiSemiJoiner{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoinResultGenerator{base} + return &leftOuterSemiJoiner{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoinResultGenerator{base} + return &antiLeftOuterSemiJoiner{base} case plan.LeftOuterJoin: - return &leftOuterJoinResultGenerator{base} + return &leftOuterJoiner{base} case plan.RightOuterJoin: - return &rightOuterJoinResultGenerator{base} + return &rightOuterJoiner{base} case plan.InnerJoin: - return &innerJoinResultGenerator{base} + return &innerJoiner{base} } - panic("unsupported join type in func newJoinResultGenerator()") + panic("unsupported join type in func newRecordJoiner()") } -// baseJoinResultGenerator is not thread-safe, -// so we should build individual generator for every join goroutine. -type baseJoinResultGenerator struct { +type baseJoiner struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -117,26 +129,26 @@ type baseJoinResultGenerator struct { maxChunkSize int } -func (outputer *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - outputer.defaultInner = mutableRow.ToRow() + joiner.defaultInner = mutableRow.ToRow() } -func (outputer *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (outputer *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { - outputer.selected, err = expression.VectorizedFilter(outputer.ctx, outputer.conditions, chunk.NewIterator4Chunk(input), outputer.selected) +func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { + joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(outputer.selected); i++ { - if !outputer.selected[i] { + for i := 0; i < len(joiner.selected); i++ { + if !joiner.selected[i] { continue } matched = true @@ -145,269 +157,269 @@ func (outputer *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (mat return matched, nil } -type semiJoinResultGenerator struct { - baseJoinResultGenerator +type semiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } - defer inners.ReachEnd() - if len(outputer.conditions) == 0 { + + defer func() { + if !(err == nil && matched) { + return + } + // here we handle the matched outer. chk.AppendPartialRow(0, outer) + inners.ReachEnd() + }() + + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(outputer.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - selected, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if selected { - chk.AppendRow(outer) - return true, nil + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + if err != nil || matched { + return true, errors.Trace(err) } } return false, nil } -func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiJoinResultGenerator struct { - baseJoinResultGenerator +type antiSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } + defer inners.ReachEnd() - if len(outputer.conditions) == 0 { + + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(outputer.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - return true, nil + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + if err != nil || matched { + return true, errors.Trace(err) } } return false, nil } -func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator +type leftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (hasMatch bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } + defer func() { - if !(err == nil && hasMatch) { + if !(err == nil && matched) { return } - inners.ReachEnd() + // here we handle the matched outer. chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) + inners.ReachEnd() }() - if len(outputer.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - return true, nil + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + if err != nil || matched { + return true, errors.Trace(err) } } return false, nil } -func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator +type antiLeftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (hasMatch bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } + defer func() { - if !(err == nil && hasMatch) { + if !(err == nil && matched) { return } - inners.ReachEnd() + // here we handle the matched outer. chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) + inners.ReachEnd() }() - if len(outputer.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - return true, nil + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + if err != nil || matched { + return true, errors.Trace(err) } } - return false, nil } -func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterJoinResultGenerator struct { - baseJoinResultGenerator +type leftOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { + + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputer.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(outputer.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) + chk.AppendPartialRow(outer.Len(), joiner.defaultInner) return } -type rightOuterJoinResultGenerator struct { - baseJoinResultGenerator +type rightOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { + + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputer.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(outputer.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) +func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, joiner.defaultInner) + chk.AppendPartialRow(joiner.defaultInner.Len(), outer) return } -type innerJoinResultGenerator struct { - baseJoinResultGenerator +type innerJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), outputer.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(chkForJoin, inner, outer) + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - outputer.makeJoinRowToChunk(chkForJoin, outer, inner) + joiner.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(outputer.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index 1d23b19ef2274..f63498582850f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator joinResultGenerator + resultGenerator recordJoiner prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 40d4c5670291e..c631a28257a58 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, + generator := newRecordJoiner(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From 59a2dea7c6ade53c0ed9eb1fb6e39ffb7f2914cb Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Mon, 6 Aug 2018 20:30:15 +0800 Subject: [PATCH 03/10] add test --- executor/join_test.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/executor/join_test.go b/executor/join_test.go index 79b068246c9ea..c8a9b8cad5c58 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -851,6 +851,19 @@ func (s *testSuite) TestIndexLookupJoin(c *C) { `1.01`, `2.02`, )) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a bigint, b bigint, unique key idx1(a, b));`) + tk.MustExec(`insert into t values(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6);`) + tk.MustExec(`set @@tidb_max_chunk_size = 2;`) + tk.MustQuery(`select /*+ TIDB_INLJ(t1) */ * from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b + 4;`).Check(testkit.Rows( + `1 1 `, + `1 2 `, + `1 3 `, + `1 4 `, + `1 5 1 1`, + `1 6 1 2`, + )) } func (s *testSuite) TestMergejoinOrder(c *C) { From a2dda9e8578391a54aeda240fa8309d2c54fc71c Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Tue, 7 Aug 2018 16:53:51 +0800 Subject: [PATCH 04/10] rename bac --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 222 ++++++++++++++--------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 122 insertions(+), 122 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index a07a8d55fb1e5..cf5c37e58944c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]recordJoiner, e.concurrency) + e.resultGenerators = make([]joinResultGenerator, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 45634b4848c17..a68d69aa66ec4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator recordJoiner + resultGenerator joinResultGenerator indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index 8161667dbce4a..f7e8eba8c8e57 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. - resultGenerators []recordJoiner + // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. + resultGenerators []joinResultGenerator outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator recordJoiner + resultGenerator joinResultGenerator outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 4f0030922086d..1c083175ad345 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,16 +23,16 @@ import ( ) var ( - _ recordJoiner = &semiJoiner{} - _ recordJoiner = &antiSemiJoiner{} - _ recordJoiner = &leftOuterSemiJoiner{} - _ recordJoiner = &antiLeftOuterSemiJoiner{} - _ recordJoiner = &leftOuterJoiner{} - _ recordJoiner = &rightOuterJoiner{} - _ recordJoiner = &innerJoiner{} + _ joinResultGenerator = &semiResultGenerator{} + _ joinResultGenerator = &antiSemiResultGenerator{} + _ joinResultGenerator = &leftOuterSemiResultGenerator{} + _ joinResultGenerator = &antiLeftOuterSemiResultGenerator{} + _ joinResultGenerator = &leftOuterResultGenerator{} + _ joinResultGenerator = &rightOuterResultGenerator{} + _ joinResultGenerator = &innerResultGenerator{} ) -// recordJoiner is used to generate join results according the join type. +// joinResultGenerator is used to generate join results according the join type. // A typical instruction flow is: // // hasMatch := false @@ -46,7 +46,7 @@ var ( // } // // NOTE: This interface is **not** thread-safe. -type recordJoiner interface { +type joinResultGenerator interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined @@ -79,10 +79,10 @@ type recordJoiner interface { onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, +func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { - base := baseJoiner{ + lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { + base := baseResultGenerator{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -102,24 +102,24 @@ func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiJoiner{base} + return &semiResultGenerator{base} case plan.AntiSemiJoin: - return &antiSemiJoiner{base} + return &antiSemiResultGenerator{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoiner{base} + return &leftOuterSemiResultGenerator{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoiner{base} + return &antiLeftOuterSemiResultGenerator{base} case plan.LeftOuterJoin: - return &leftOuterJoiner{base} + return &leftOuterResultGenerator{base} case plan.RightOuterJoin: - return &rightOuterJoiner{base} + return &rightOuterResultGenerator{base} case plan.InnerJoin: - return &innerJoiner{base} + return &innerResultGenerator{base} } - panic("unsupported join type in func newRecordJoiner()") + panic("unsupported join type in func newJoinResultGenerator()") } -type baseJoiner struct { +type baseResultGenerator struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -129,26 +129,26 @@ type baseJoiner struct { maxChunkSize int } -func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (outputor *baseResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - joiner.defaultInner = mutableRow.ToRow() + outputor.defaultInner = mutableRow.ToRow() } -func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (outputor *baseResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { - joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) +func (outputor *baseResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { + outputor.selected, err = expression.VectorizedFilter(outputor.ctx, outputor.conditions, chunk.NewIterator4Chunk(input), outputor.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(joiner.selected); i++ { - if !joiner.selected[i] { + for i := 0; i < len(outputor.selected); i++ { + if !outputor.selected[i] { continue } matched = true @@ -157,11 +157,11 @@ func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err return matched, nil } -type semiJoiner struct { - baseJoiner +type semiResultGenerator struct { + baseResultGenerator } -func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +func (outputor *semiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -175,18 +175,18 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputor.chk.Reset() + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(outputor.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -194,35 +194,35 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk return false, nil } -func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *semiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiJoiner struct { - baseJoiner +type antiSemiResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *antiSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } defer inners.ReachEnd() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputor.chk.Reset() + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(outputor.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -230,17 +230,17 @@ func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, return false, nil } -func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *antiSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiJoiner struct { - baseJoiner +type leftOuterSemiResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *leftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -255,14 +255,14 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputor.chk.Reset() + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -270,18 +270,18 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter return false, nil } -func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *leftOuterSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiJoiner struct { - baseJoiner +type antiLeftOuterSemiResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *antiLeftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -296,14 +296,14 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputor.chk.Reset() + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + matched, err := expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -311,115 +311,115 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. return false, nil } -func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *antiLeftOuterSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterJoiner struct { - baseJoiner +type leftOuterResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *leftOuterResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputor.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + outputor.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *leftOuterResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), joiner.defaultInner) + chk.AppendPartialRow(outer.Len(), outputor.defaultInner) return } -type rightOuterJoiner struct { - baseJoiner +type rightOuterResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *rightOuterResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputor.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + outputor.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, joiner.defaultInner) - chk.AppendPartialRow(joiner.defaultInner.Len(), outer) +func (outputor *rightOuterResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outputor.defaultInner) + chk.AppendPartialRow(outputor.defaultInner.Len(), outer) return } -type innerJoiner struct { - baseJoiner +type innerResultGenerator struct { + baseResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *innerResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), outputor.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(chkForJoin, inner, outer) + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - joiner.makeJoinRowToChunk(chkForJoin, outer, inner) + outputor.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *innerResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index f63498582850f..1d23b19ef2274 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator recordJoiner + resultGenerator joinResultGenerator prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index c631a28257a58..40d4c5670291e 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newRecordJoiner(sctx, plan.InnerJoin, false, + generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From 63581665d5dae38cc156bee3c9cb3edfb1d51010 Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Tue, 7 Aug 2018 16:56:09 +0800 Subject: [PATCH 05/10] Revert "rename bac" This reverts commit a2dda9e8578391a54aeda240fa8309d2c54fc71c. --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 222 ++++++++++++++--------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 122 insertions(+), 122 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index cf5c37e58944c..a07a8d55fb1e5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]joinResultGenerator, e.concurrency) + e.resultGenerators = make([]recordJoiner, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a68d69aa66ec4..45634b4848c17 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator joinResultGenerator + resultGenerator recordJoiner indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index f7e8eba8c8e57..8161667dbce4a 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. - resultGenerators []joinResultGenerator + // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. + resultGenerators []recordJoiner outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator joinResultGenerator + resultGenerator recordJoiner outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 1c083175ad345..4f0030922086d 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,16 +23,16 @@ import ( ) var ( - _ joinResultGenerator = &semiResultGenerator{} - _ joinResultGenerator = &antiSemiResultGenerator{} - _ joinResultGenerator = &leftOuterSemiResultGenerator{} - _ joinResultGenerator = &antiLeftOuterSemiResultGenerator{} - _ joinResultGenerator = &leftOuterResultGenerator{} - _ joinResultGenerator = &rightOuterResultGenerator{} - _ joinResultGenerator = &innerResultGenerator{} + _ recordJoiner = &semiJoiner{} + _ recordJoiner = &antiSemiJoiner{} + _ recordJoiner = &leftOuterSemiJoiner{} + _ recordJoiner = &antiLeftOuterSemiJoiner{} + _ recordJoiner = &leftOuterJoiner{} + _ recordJoiner = &rightOuterJoiner{} + _ recordJoiner = &innerJoiner{} ) -// joinResultGenerator is used to generate join results according the join type. +// recordJoiner is used to generate join results according the join type. // A typical instruction flow is: // // hasMatch := false @@ -46,7 +46,7 @@ var ( // } // // NOTE: This interface is **not** thread-safe. -type joinResultGenerator interface { +type recordJoiner interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined @@ -79,10 +79,10 @@ type joinResultGenerator interface { onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, +func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { - base := baseResultGenerator{ + lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { + base := baseJoiner{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -102,24 +102,24 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiResultGenerator{base} + return &semiJoiner{base} case plan.AntiSemiJoin: - return &antiSemiResultGenerator{base} + return &antiSemiJoiner{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiResultGenerator{base} + return &leftOuterSemiJoiner{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiResultGenerator{base} + return &antiLeftOuterSemiJoiner{base} case plan.LeftOuterJoin: - return &leftOuterResultGenerator{base} + return &leftOuterJoiner{base} case plan.RightOuterJoin: - return &rightOuterResultGenerator{base} + return &rightOuterJoiner{base} case plan.InnerJoin: - return &innerResultGenerator{base} + return &innerJoiner{base} } - panic("unsupported join type in func newJoinResultGenerator()") + panic("unsupported join type in func newRecordJoiner()") } -type baseResultGenerator struct { +type baseJoiner struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -129,26 +129,26 @@ type baseResultGenerator struct { maxChunkSize int } -func (outputor *baseResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - outputor.defaultInner = mutableRow.ToRow() + joiner.defaultInner = mutableRow.ToRow() } -func (outputor *baseResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (outputor *baseResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { - outputor.selected, err = expression.VectorizedFilter(outputor.ctx, outputor.conditions, chunk.NewIterator4Chunk(input), outputor.selected) +func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { + joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(outputor.selected); i++ { - if !outputor.selected[i] { + for i := 0; i < len(joiner.selected); i++ { + if !joiner.selected[i] { continue } matched = true @@ -157,11 +157,11 @@ func (outputor *baseResultGenerator) filter(input, output *chunk.Chunk) (matched return matched, nil } -type semiResultGenerator struct { - baseResultGenerator +type semiJoiner struct { + baseJoiner } -func (outputor *semiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -175,18 +175,18 @@ func (outputor *semiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.It inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(outputor.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -194,35 +194,35 @@ func (outputor *semiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.It return false, nil } -func (outputor *semiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiResultGenerator struct { - baseResultGenerator +type antiSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *antiSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } defer inners.ReachEnd() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(outputor.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -230,17 +230,17 @@ func (outputor *antiSemiResultGenerator) tryToMatch(outer chunk.Row, inners chun return false, nil } -func (outputor *antiSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiResultGenerator struct { - baseResultGenerator +type leftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *leftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -255,14 +255,14 @@ func (outputor *leftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -270,18 +270,18 @@ func (outputor *leftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners return false, nil } -func (outputor *leftOuterSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiResultGenerator struct { - baseResultGenerator +type antiLeftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *antiLeftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -296,14 +296,14 @@ func (outputor *antiLeftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, in inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) - matched, err := expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -311,115 +311,115 @@ func (outputor *antiLeftOuterSemiResultGenerator) tryToMatch(outer chunk.Row, in return false, nil } -func (outputor *antiLeftOuterSemiResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterResultGenerator struct { - baseResultGenerator +type leftOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *leftOuterResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputor.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputor.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *leftOuterResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputor.defaultInner) + chk.AppendPartialRow(outer.Len(), joiner.defaultInner) return } -type rightOuterResultGenerator struct { - baseResultGenerator +type rightOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *rightOuterResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputor.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputor.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *rightOuterResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outputor.defaultInner) - chk.AppendPartialRow(outputor.defaultInner.Len(), outer) +func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, joiner.defaultInner) + chk.AppendPartialRow(joiner.defaultInner.Len(), outer) return } -type innerResultGenerator struct { - baseResultGenerator +type innerJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *innerResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), outputor.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(chkForJoin, inner, outer) + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - outputor.makeJoinRowToChunk(chkForJoin, outer, inner) + joiner.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *innerResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index 1d23b19ef2274..f63498582850f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator joinResultGenerator + resultGenerator recordJoiner prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 40d4c5670291e..c631a28257a58 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, + generator := newRecordJoiner(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From a2db5b89e3d54e149e450dfb8e634ff440fd5acb Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Tue, 7 Aug 2018 16:56:56 +0800 Subject: [PATCH 06/10] rename back, again --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 222 ++++++++++++++--------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 122 insertions(+), 122 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index a07a8d55fb1e5..cf5c37e58944c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]recordJoiner, e.concurrency) + e.resultGenerators = make([]joinResultGenerator, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 45634b4848c17..a68d69aa66ec4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator recordJoiner + resultGenerator joinResultGenerator indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index 8161667dbce4a..f7e8eba8c8e57 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. - resultGenerators []recordJoiner + // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. + resultGenerators []joinResultGenerator outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator recordJoiner + resultGenerator joinResultGenerator outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 4f0030922086d..fd41fd00debfd 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,16 +23,16 @@ import ( ) var ( - _ recordJoiner = &semiJoiner{} - _ recordJoiner = &antiSemiJoiner{} - _ recordJoiner = &leftOuterSemiJoiner{} - _ recordJoiner = &antiLeftOuterSemiJoiner{} - _ recordJoiner = &leftOuterJoiner{} - _ recordJoiner = &rightOuterJoiner{} - _ recordJoiner = &innerJoiner{} + _ joinResultGenerator = &semiJoinResultGenerator{} + _ joinResultGenerator = &antiSemiJoinResultGenerator{} + _ joinResultGenerator = &leftOuterSemiJoinResultGenerator{} + _ joinResultGenerator = &antiLeftOuterSemiJoinResultGenerator{} + _ joinResultGenerator = &leftOuterJoinResultGenerator{} + _ joinResultGenerator = &rightOuterJoinResultGenerator{} + _ joinResultGenerator = &innerJoinResultGenerator{} ) -// recordJoiner is used to generate join results according the join type. +// joinResultGenerator is used to generate join results according the join type. // A typical instruction flow is: // // hasMatch := false @@ -46,7 +46,7 @@ var ( // } // // NOTE: This interface is **not** thread-safe. -type recordJoiner interface { +type joinResultGenerator interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined @@ -79,10 +79,10 @@ type recordJoiner interface { onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, +func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { - base := baseJoiner{ + lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { + base := baseJoinResultGenerator{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -102,24 +102,24 @@ func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiJoiner{base} + return &semiJoinResultGenerator{base} case plan.AntiSemiJoin: - return &antiSemiJoiner{base} + return &antiSemiJoinResultGenerator{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoiner{base} + return &leftOuterSemiJoinResultGenerator{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoiner{base} + return &antiLeftOuterSemiJoinResultGenerator{base} case plan.LeftOuterJoin: - return &leftOuterJoiner{base} + return &leftOuterJoinResultGenerator{base} case plan.RightOuterJoin: - return &rightOuterJoiner{base} + return &rightOuterJoinResultGenerator{base} case plan.InnerJoin: - return &innerJoiner{base} + return &innerJoinResultGenerator{base} } - panic("unsupported join type in func newRecordJoiner()") + panic("unsupported join type in func newJoinResultGenerator()") } -type baseJoiner struct { +type baseJoinResultGenerator struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -129,26 +129,26 @@ type baseJoiner struct { maxChunkSize int } -func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (outputor *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - joiner.defaultInner = mutableRow.ToRow() + outputor.defaultInner = mutableRow.ToRow() } -func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (outputor *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { - joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) +func (outputor *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { + outputor.selected, err = expression.VectorizedFilter(outputor.ctx, outputor.conditions, chunk.NewIterator4Chunk(input), outputor.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(joiner.selected); i++ { - if !joiner.selected[i] { + for i := 0; i < len(outputor.selected); i++ { + if !outputor.selected[i] { continue } matched = true @@ -157,11 +157,11 @@ func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err return matched, nil } -type semiJoiner struct { - baseJoiner +type semiJoinResultGenerator struct { + baseJoinResultGenerator } -func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +func (outputor *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -175,18 +175,18 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputor.chk.Reset() + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(outputor.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -194,35 +194,35 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk return false, nil } -func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiJoiner struct { - baseJoiner +type antiSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } defer inners.ReachEnd() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputor.chk.Reset() + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(outputor.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -230,17 +230,17 @@ func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, return false, nil } -func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiJoiner struct { - baseJoiner +type leftOuterSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -255,14 +255,14 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputor.chk.Reset() + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -270,18 +270,18 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter return false, nil } -func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiJoiner struct { - baseJoiner +type antiLeftOuterSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -296,14 +296,14 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputor.chk.Reset() + outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + matched, err := expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -311,115 +311,115 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. return false, nil } -func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterJoiner struct { - baseJoiner +type leftOuterJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputor.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + outputor.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), joiner.defaultInner) + chk.AppendPartialRow(outer.Len(), outputor.defaultInner) return } -type rightOuterJoiner struct { - baseJoiner +type rightOuterJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputor.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + outputor.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, joiner.defaultInner) - chk.AppendPartialRow(joiner.defaultInner.Len(), outer) +func (outputor *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outputor.defaultInner) + chk.AppendPartialRow(outputor.defaultInner.Len(), outer) return } -type innerJoiner struct { - baseJoiner +type innerJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputor *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputor.chk.Reset() + chkForJoin := outputor.chk + if len(outputor.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), outputor.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(chkForJoin, inner, outer) + if outputor.outerIsRight { + outputor.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - joiner.makeJoinRowToChunk(chkForJoin, outer, inner) + outputor.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(joiner.conditions) == 0 { + if len(outputor.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputor.chk + matched, err := outputor.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputor *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index f63498582850f..1d23b19ef2274 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator recordJoiner + resultGenerator joinResultGenerator prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index c631a28257a58..40d4c5670291e 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newRecordJoiner(sctx, plan.InnerJoin, false, + generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From 62321250bcbe89326fbed3839d15770b95accd6d Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Tue, 7 Aug 2018 16:58:11 +0800 Subject: [PATCH 07/10] Revert "rename back, again" This reverts commit a2db5b89e3d54e149e450dfb8e634ff440fd5acb. --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 222 ++++++++++++++--------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 122 insertions(+), 122 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index cf5c37e58944c..a07a8d55fb1e5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]joinResultGenerator, e.concurrency) + e.resultGenerators = make([]recordJoiner, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a68d69aa66ec4..45634b4848c17 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator joinResultGenerator + resultGenerator recordJoiner indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index f7e8eba8c8e57..8161667dbce4a 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. - resultGenerators []joinResultGenerator + // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. + resultGenerators []recordJoiner outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator joinResultGenerator + resultGenerator recordJoiner outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index fd41fd00debfd..4f0030922086d 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,16 +23,16 @@ import ( ) var ( - _ joinResultGenerator = &semiJoinResultGenerator{} - _ joinResultGenerator = &antiSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &antiLeftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterJoinResultGenerator{} - _ joinResultGenerator = &rightOuterJoinResultGenerator{} - _ joinResultGenerator = &innerJoinResultGenerator{} + _ recordJoiner = &semiJoiner{} + _ recordJoiner = &antiSemiJoiner{} + _ recordJoiner = &leftOuterSemiJoiner{} + _ recordJoiner = &antiLeftOuterSemiJoiner{} + _ recordJoiner = &leftOuterJoiner{} + _ recordJoiner = &rightOuterJoiner{} + _ recordJoiner = &innerJoiner{} ) -// joinResultGenerator is used to generate join results according the join type. +// recordJoiner is used to generate join results according the join type. // A typical instruction flow is: // // hasMatch := false @@ -46,7 +46,7 @@ var ( // } // // NOTE: This interface is **not** thread-safe. -type joinResultGenerator interface { +type recordJoiner interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined @@ -79,10 +79,10 @@ type joinResultGenerator interface { onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, +func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { - base := baseJoinResultGenerator{ + lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { + base := baseJoiner{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -102,24 +102,24 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiJoinResultGenerator{base} + return &semiJoiner{base} case plan.AntiSemiJoin: - return &antiSemiJoinResultGenerator{base} + return &antiSemiJoiner{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoinResultGenerator{base} + return &leftOuterSemiJoiner{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoinResultGenerator{base} + return &antiLeftOuterSemiJoiner{base} case plan.LeftOuterJoin: - return &leftOuterJoinResultGenerator{base} + return &leftOuterJoiner{base} case plan.RightOuterJoin: - return &rightOuterJoinResultGenerator{base} + return &rightOuterJoiner{base} case plan.InnerJoin: - return &innerJoinResultGenerator{base} + return &innerJoiner{base} } - panic("unsupported join type in func newJoinResultGenerator()") + panic("unsupported join type in func newRecordJoiner()") } -type baseJoinResultGenerator struct { +type baseJoiner struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -129,26 +129,26 @@ type baseJoinResultGenerator struct { maxChunkSize int } -func (outputor *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - outputor.defaultInner = mutableRow.ToRow() + joiner.defaultInner = mutableRow.ToRow() } -func (outputor *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (outputor *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { - outputor.selected, err = expression.VectorizedFilter(outputor.ctx, outputor.conditions, chunk.NewIterator4Chunk(input), outputor.selected) +func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { + joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(outputor.selected); i++ { - if !outputor.selected[i] { + for i := 0; i < len(joiner.selected); i++ { + if !joiner.selected[i] { continue } matched = true @@ -157,11 +157,11 @@ func (outputor *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (mat return matched, nil } -type semiJoinResultGenerator struct { - baseJoinResultGenerator +type semiJoiner struct { + baseJoiner } -func (outputor *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -175,18 +175,18 @@ func (outputor *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chun inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(outputor.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -194,35 +194,35 @@ func (outputor *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chun return false, nil } -func (outputor *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiJoinResultGenerator struct { - baseJoinResultGenerator +type antiSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } defer inners.ReachEnd() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(outputor.chk, inner, outer) + joiner.chk.Reset() + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(joiner.chk, inner, outer) } else { - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) } - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -230,17 +230,17 @@ func (outputor *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners return false, nil } -func (outputor *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator +type leftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -255,14 +255,14 @@ func (outputor *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, in inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) - matched, err = expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -270,18 +270,18 @@ func (outputor *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, in return false, nil } -func (outputor *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator +type antiLeftOuterSemiJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements recordJoiner interface. +func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -296,14 +296,14 @@ func (outputor *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row inners.ReachEnd() }() - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputor.chk.Reset() - outputor.makeJoinRowToChunk(outputor.chk, outer, inner) - matched, err := expression.EvalBool(outputor.ctx, outputor.conditions, outputor.chk.GetRow(0)) + joiner.chk.Reset() + joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -311,115 +311,115 @@ func (outputor *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row return false, nil } -func (outputor *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterJoinResultGenerator struct { - baseJoinResultGenerator +type leftOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputor.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputor.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputor.defaultInner) + chk.AppendPartialRow(outer.Len(), joiner.defaultInner) return } -type rightOuterJoinResultGenerator struct { - baseJoinResultGenerator +type rightOuterJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - numToAppend := outputor.maxChunkSize - chk.NumRows() + numToAppend := joiner.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputor.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outputor.defaultInner) - chk.AppendPartialRow(outputor.defaultInner.Len(), outer) +func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, joiner.defaultInner) + chk.AppendPartialRow(joiner.defaultInner.Len(), outer) return } -type innerJoinResultGenerator struct { - baseJoinResultGenerator +type innerJoiner struct { + baseJoiner } -// tryToMatch implements joinResultGenerator interface. -func (outputor *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements recordJoiner interface. +func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - outputor.chk.Reset() - chkForJoin := outputor.chk - if len(outputor.conditions) == 0 { + joiner.chk.Reset() + chkForJoin := joiner.chk + if len(joiner.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), outputor.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if outputor.outerIsRight { - outputor.makeJoinRowToChunk(chkForJoin, inner, outer) + if joiner.outerIsRight { + joiner.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - outputor.makeJoinRowToChunk(chkForJoin, outer, inner) + joiner.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(outputor.conditions) == 0 { + if len(joiner.conditions) == 0 { return true, nil } - // reach here, chkForJoin is outputor.chk - matched, err := outputor.filter(chkForJoin, chk) + // reach here, chkForJoin is joiner.chk + matched, err := joiner.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (outputor *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index 1d23b19ef2274..f63498582850f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator joinResultGenerator + resultGenerator recordJoiner prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 40d4c5670291e..c631a28257a58 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, + generator := newRecordJoiner(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From ead095d4d0376585e7c57b7588b5c84d00585e5a Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Tue, 7 Aug 2018 16:58:41 +0800 Subject: [PATCH 08/10] rename back, again... --- executor/builder.go | 10 +- executor/index_lookup_join.go | 2 +- executor/join.go | 6 +- executor/join_result_generators.go | 222 ++++++++++++++--------------- executor/merge_join.go | 2 +- executor/pkg_test.go | 2 +- 6 files changed, 122 insertions(+), 122 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index a07a8d55fb1e5..cf5c37e58944c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,7 +716,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -818,9 +818,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]recordJoiner, e.concurrency) + e.resultGenerators = make([]joinResultGenerator, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1153,7 +1153,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newRecordJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, + generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1510,7 +1510,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut rowTypes: innerTypes, }, workerWg: new(sync.WaitGroup), - resultGenerator: newRecordJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 45634b4848c17..a68d69aa66ec4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator recordJoiner + resultGenerator joinResultGenerator indexRanges []*ranger.Range keyOff2IdxOff []int diff --git a/executor/join.go b/executor/join.go index 8161667dbce4a..f7e8eba8c8e57 100644 --- a/executor/join.go +++ b/executor/join.go @@ -59,8 +59,8 @@ type HashJoinExec struct { innerIdx int // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of recordJoiner.chk and recordJoiner.selected. - resultGenerators []recordJoiner + // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. + resultGenerators []joinResultGenerator outerKeyColIdx []int innerKeyColIdx []int @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator recordJoiner + resultGenerator joinResultGenerator outerSchema []*expression.CorrelatedColumn diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 4f0030922086d..d4b52514a862a 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -23,16 +23,16 @@ import ( ) var ( - _ recordJoiner = &semiJoiner{} - _ recordJoiner = &antiSemiJoiner{} - _ recordJoiner = &leftOuterSemiJoiner{} - _ recordJoiner = &antiLeftOuterSemiJoiner{} - _ recordJoiner = &leftOuterJoiner{} - _ recordJoiner = &rightOuterJoiner{} - _ recordJoiner = &innerJoiner{} + _ joinResultGenerator = &semiJoinResultGenerator{} + _ joinResultGenerator = &antiSemiJoinResultGenerator{} + _ joinResultGenerator = &leftOuterSemiJoinResultGenerator{} + _ joinResultGenerator = &antiLeftOuterSemiJoinResultGenerator{} + _ joinResultGenerator = &leftOuterJoinResultGenerator{} + _ joinResultGenerator = &rightOuterJoinResultGenerator{} + _ joinResultGenerator = &innerJoinResultGenerator{} ) -// recordJoiner is used to generate join results according the join type. +// joinResultGenerator is used to generate join results according the join type. // A typical instruction flow is: // // hasMatch := false @@ -46,7 +46,7 @@ var ( // } // // NOTE: This interface is **not** thread-safe. -type recordJoiner interface { +type joinResultGenerator interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined @@ -79,10 +79,10 @@ type recordJoiner interface { onMissMatch(outer chunk.Row, chk *chunk.Chunk) } -func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, +func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) recordJoiner { - base := baseJoiner{ + lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { + base := baseJoinResultGenerator{ ctx: ctx, conditions: filter, outerIsRight: outerIsRight, @@ -102,24 +102,24 @@ func newRecordJoiner(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: - return &semiJoiner{base} + return &semiJoinResultGenerator{base} case plan.AntiSemiJoin: - return &antiSemiJoiner{base} + return &antiSemiJoinResultGenerator{base} case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoiner{base} + return &leftOuterSemiJoinResultGenerator{base} case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoiner{base} + return &antiLeftOuterSemiJoinResultGenerator{base} case plan.LeftOuterJoin: - return &leftOuterJoiner{base} + return &leftOuterJoinResultGenerator{base} case plan.RightOuterJoin: - return &rightOuterJoiner{base} + return &rightOuterJoinResultGenerator{base} case plan.InnerJoin: - return &innerJoiner{base} + return &innerJoinResultGenerator{base} } - panic("unsupported join type in func newRecordJoiner()") + panic("unsupported join type in func newJoinResultGenerator()") } -type baseJoiner struct { +type baseJoinResultGenerator struct { ctx sessionctx.Context conditions []expression.Expression defaultInner chunk.Row @@ -129,26 +129,26 @@ type baseJoiner struct { maxChunkSize int } -func (joiner *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { +func (outputer *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { mutableRow := chunk.MutRowFromTypes(innerTypes) mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - joiner.defaultInner = mutableRow.ToRow() + outputer.defaultInner = mutableRow.ToRow() } -func (joiner *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { +func (outputer *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { // Call AppendRow() first to increment the virtual rows. // Fix: https://github.com/pingcap/tidb/issues/5771 chk.AppendRow(lhs) chk.AppendPartialRow(lhs.Len(), rhs) } -func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { - joiner.selected, err = expression.VectorizedFilter(joiner.ctx, joiner.conditions, chunk.NewIterator4Chunk(input), joiner.selected) +func (outputer *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { + outputer.selected, err = expression.VectorizedFilter(outputer.ctx, outputer.conditions, chunk.NewIterator4Chunk(input), outputer.selected) if err != nil { return false, errors.Trace(err) } - for i := 0; i < len(joiner.selected); i++ { - if !joiner.selected[i] { + for i := 0; i < len(outputer.selected); i++ { + if !outputer.selected[i] { continue } matched = true @@ -157,11 +157,11 @@ func (joiner *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err return matched, nil } -type semiJoiner struct { - baseJoiner +type semiJoinResultGenerator struct { + baseJoinResultGenerator } -func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -175,18 +175,18 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputer.chk.Reset() + if outputer.outerIsRight { + outputer.makeJoinRowToChunk(outputer.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -194,35 +194,35 @@ func (joiner *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk return false, nil } -func (joiner *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } -type antiSemiJoiner struct { - baseJoiner +type antiSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } defer inners.ReachEnd() - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(joiner.chk, inner, outer) + outputer.chk.Reset() + if outputer.outerIsRight { + outputer.makeJoinRowToChunk(outputer.chk, inner, outer) } else { - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) + outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -230,17 +230,17 @@ func (joiner *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, return false, nil } -func (joiner *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) return } -type leftOuterSemiJoiner struct { - baseJoiner +type leftOuterSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -255,14 +255,14 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err = expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputer.chk.Reset() + outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -270,18 +270,18 @@ func (joiner *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iter return false, nil } -func (joiner *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return } -type antiLeftOuterSemiJoiner struct { - baseJoiner +type antiLeftOuterSemiJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { if inners.Len() == 0 { return false, nil } @@ -296,14 +296,14 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. inners.ReachEnd() }() - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - joiner.chk.Reset() - joiner.makeJoinRowToChunk(joiner.chk, outer, inner) - matched, err := expression.EvalBool(joiner.ctx, joiner.conditions, joiner.chk.GetRow(0)) + outputer.chk.Reset() + outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil || matched { return true, errors.Trace(err) } @@ -311,115 +311,115 @@ func (joiner *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk. return false, nil } -func (joiner *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return } -type leftOuterJoiner struct { - baseJoiner +type leftOuterJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputer.chk.Reset() + chkForJoin := outputer.chk + if len(outputer.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputer.chk + matched, err := outputer.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), joiner.defaultInner) + chk.AppendPartialRow(outer.Len(), outputer.defaultInner) return } -type rightOuterJoiner struct { - baseJoiner +type rightOuterJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputer.chk.Reset() + chkForJoin := outputer.chk + if len(outputer.conditions) == 0 { chkForJoin = chk } - numToAppend := joiner.maxChunkSize - chk.NumRows() + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - joiner.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputer.chk + matched, err := outputer.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, joiner.defaultInner) - chk.AppendPartialRow(joiner.defaultInner.Len(), outer) +func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outputer.defaultInner) + chk.AppendPartialRow(outputer.defaultInner.Len(), outer) return } -type innerJoiner struct { - baseJoiner +type innerJoinResultGenerator struct { + baseJoinResultGenerator } -// tryToMatch implements recordJoiner interface. -func (joiner *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { +// tryToMatch implements joinResultGenerator interface. +func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { if inners.Len() == 0 { return false, nil } - joiner.chk.Reset() - chkForJoin := joiner.chk - if len(joiner.conditions) == 0 { + outputer.chk.Reset() + chkForJoin := outputer.chk + if len(outputer.conditions) == 0 { chkForJoin = chk } - inner, numToAppend := inners.Current(), joiner.maxChunkSize-chk.NumRows() + inner, numToAppend := inners.Current(), outputer.maxChunkSize-chk.NumRows() for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if joiner.outerIsRight { - joiner.makeJoinRowToChunk(chkForJoin, inner, outer) + if outputer.outerIsRight { + outputer.makeJoinRowToChunk(chkForJoin, inner, outer) } else { - joiner.makeJoinRowToChunk(chkForJoin, outer, inner) + outputer.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(joiner.conditions) == 0 { + if len(outputer.conditions) == 0 { return true, nil } - // reach here, chkForJoin is joiner.chk - matched, err := joiner.filter(chkForJoin, chk) + // reach here, chkForJoin is outputer.chk + matched, err := outputer.filter(chkForJoin, chk) return matched, errors.Trace(err) } -func (joiner *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { return } diff --git a/executor/merge_join.go b/executor/merge_join.go index f63498582850f..1d23b19ef2274 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -34,7 +34,7 @@ type MergeJoinExec struct { stmtCtx *stmtctx.StatementContext compareFuncs []chunk.CompareFunc - resultGenerator recordJoiner + resultGenerator joinResultGenerator prepared bool outerIdx int diff --git a/executor/pkg_test.go b/executor/pkg_test.go index c631a28257a58..40d4c5670291e 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,7 +80,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newRecordJoiner(sctx, plan.InnerJoin, false, + generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ From 491d5ccbda062db4060bb74fde03a336b6757819 Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Wed, 8 Aug 2018 13:53:07 +0800 Subject: [PATCH 09/10] fix comment --- executor/join_result_generators.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index d4b52514a862a..7c434814e2151 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -32,8 +32,8 @@ var ( _ joinResultGenerator = &innerJoinResultGenerator{} ) -// joinResultGenerator is used to generate join results according the join type. -// A typical instruction flow is: +// joinResultGenerator is used to generate join results according to the join +// type. A typical instruction flow is: // // hasMatch := false // for innerIter.Current() != innerIter.End() { @@ -50,7 +50,7 @@ type joinResultGenerator interface { // tryToMatch tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is // considered unmatched. Otherwise, the outer row is matched and some joined - // rows is appended to `chk`. The size of `chk` is limited to MaxChunkSize. + // rows are appended to `chk`. The size of `chk` is limited to MaxChunkSize. // // NOTE: Callers need to call this function multiple times to consume all // the inner rows for an outer row, and dicide whether the outer row can be @@ -60,21 +60,21 @@ type joinResultGenerator interface { // onMissMatch operates on the unmatched outer row according to the join // type. An outer row can be considered miss matched if: // 1. it can not pass the filter on the outer table side. - // 2. there does not exist an inner row with the same join key. + // 2. there is no inner row with the same join key.. // 3. all the joined rows can not pass the filter on the join result. // // On these conditions, the caller calls this function to handle the // unmatched outer rows according to the current join type: // 1. 'SemiJoin': ignores the unmatched outer row. // 2. 'AntiSemiJoin': appends the unmatched outer row to the result buffer. - // 3. 'LeftOuterSemiJoin': concates the unmatched outer row with 0 and - // appended it to the result buffer. - // 4. 'AntiLeftOuterSemiJoin': concates the unmatched outer row with 0 and - // appended to the result buffer. - // 5. 'LeftOuterJoin': concates the unmatched outer row with a row of NULLs - // and appended to the result buffer. - // 6. 'RightOuterJoin': concates the unmatched outer row with a row of NULLs - // and appended to the result buffer. + // 3. 'LeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 4. 'AntiLeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 5. 'LeftOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. + // 6. 'RightOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. // 7. 'InnerJoin': ignores the unmatched outer row. onMissMatch(outer chunk.Row, chk *chunk.Chunk) } From 1a338d6065fbacaf66c34266b6fd25f395772996 Mon Sep 17 00:00:00 2001 From: Jian Zhang Date: Wed, 8 Aug 2018 14:36:39 +0800 Subject: [PATCH 10/10] address comment --- executor/join_result_generators.go | 93 +++++++++++++++--------------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 7c434814e2151..9098502bc2825 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -60,7 +60,7 @@ type joinResultGenerator interface { // onMissMatch operates on the unmatched outer row according to the join // type. An outer row can be considered miss matched if: // 1. it can not pass the filter on the outer table side. - // 2. there is no inner row with the same join key.. + // 2. there is no inner row with the same join key. // 3. all the joined rows can not pass the filter on the join result. // // On these conditions, the caller calls this function to handle the @@ -166,16 +166,9 @@ func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chun return false, nil } - defer func() { - if !(err == nil && matched) { - return - } - // here we handle the matched outer. + if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) inners.ReachEnd() - }() - - if len(outputer.conditions) == 0 { return true, nil } @@ -186,16 +179,21 @@ func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chun } else { outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil || matched { - return true, errors.Trace(err) + if err != nil { + return false, errors.Trace(err) + } + if matched { + chk.AppendPartialRow(0, outer) + inners.ReachEnd() + return true, nil } } return false, nil } func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - return } type antiSemiJoinResultGenerator struct { @@ -208,9 +206,8 @@ func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners return false, nil } - defer inners.ReachEnd() - if len(outputer.conditions) == 0 { + inners.ReachEnd() return true, nil } @@ -223,8 +220,12 @@ func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners } matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil || matched { - return true, errors.Trace(err) + if err != nil { + return false, errors.Trace(err) + } + if matched { + inners.ReachEnd() + return true, nil } } return false, nil @@ -232,7 +233,6 @@ func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) - return } type leftOuterSemiJoinResultGenerator struct { @@ -245,35 +245,37 @@ func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, in return false, nil } - defer func() { - if !(err == nil && matched) { - return - } - // here we handle the matched outer. - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - inners.ReachEnd() - }() - if len(outputer.conditions) == 0 { + outputer.onMatch(outer, chk) + inners.ReachEnd() return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil || matched { - return true, errors.Trace(err) + if err != nil { + return false, errors.Trace(err) + } + if matched { + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } } return false, nil } +func (outputer *leftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 1) +} + func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) - return } type antiLeftOuterSemiJoinResultGenerator struct { @@ -286,17 +288,9 @@ func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row return false, nil } - defer func() { - if !(err == nil && matched) { - return - } - // here we handle the matched outer. - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - inners.ReachEnd() - }() - if len(outputer.conditions) == 0 { + outputer.onMatch(outer, chk) + inners.ReachEnd() return true, nil } @@ -304,17 +298,27 @@ func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil || matched { - return true, errors.Trace(err) + + if err != nil { + return false, errors.Trace(err) + } + if matched { + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } } return false, nil } +func (outputer *antiLeftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 0) +} + func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) - return } type leftOuterJoinResultGenerator struct { @@ -350,7 +354,6 @@ func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendPartialRow(outer.Len(), outputer.defaultInner) - return } type rightOuterJoinResultGenerator struct { @@ -386,7 +389,6 @@ func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inner func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outputer.defaultInner) chk.AppendPartialRow(outputer.defaultInner.Len(), outer) - return } type innerJoinResultGenerator struct { @@ -421,5 +423,4 @@ func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chu } func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - return }