Files
tidb/pkg/executor/join/inner_join_probe.go

107 lines
3.5 KiB
Go

// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package join
import (
"sync/atomic"
"unsafe"
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/sqlkiller"
)
type innerJoinProbe struct {
baseJoinProbe
}
func (j *innerJoinProbe) Probe(joinResult *hashjoinWorkerResult, sqlKiller *sqlkiller.SQLKiller) (ok bool, _ *hashjoinWorkerResult) {
if joinResult.chk.IsFull() {
return true, joinResult
}
hasOtherCondition := j.ctx.hasOtherCondition()
joinedChk, remainCap, err := j.prepareForProbe(joinResult.chk)
if err != nil {
joinResult.err = err
return false, joinResult
}
meta := j.ctx.hashTableMeta
isInCompleteChunk := joinedChk.IsInCompleteChunk()
// in case that virtual rows is not maintained correctly
joinedChk.SetNumVirtualRows(joinedChk.NumRows())
// always set in complete chunk during probe
joinedChk.SetInCompleteChunk(true)
defer joinedChk.SetInCompleteChunk(isInCompleteChunk)
for remainCap > 0 && j.currentProbeRow < j.chunkRows {
if j.matchedRowsHeaders[j.currentProbeRow] != 0 {
candidateRow := *(*unsafe.Pointer)(unsafe.Pointer(&j.matchedRowsHeaders[j.currentProbeRow]))
if isKeyMatched(meta.keyMode, j.serializedKeys[j.currentProbeRow], candidateRow, meta) {
// key matched, convert row to column for build side
j.appendBuildRowToCachedBuildRowsAndConstructBuildRowsIfNeeded(createMatchRowInfo(j.currentProbeRow, candidateRow), joinedChk, 0, hasOtherCondition)
j.matchedRowsForCurrentProbeRow++
remainCap--
} else {
if j.ctx.stats != nil {
atomic.AddInt64(&j.ctx.stats.hashStat.probeCollision, 1)
}
}
j.matchedRowsHeaders[j.currentProbeRow] = getNextRowAddress(candidateRow)
} else {
j.finishLookupCurrentProbeRow()
j.currentProbeRow++
}
}
err = checkSQLKiller(sqlKiller, "killedDuringProbe")
if err != nil {
joinResult.err = err
return false, joinResult
}
j.finishCurrentLookupLoop(joinedChk)
if j.ctx.hasOtherCondition() && joinedChk.NumRows() > 0 {
// eval other condition, and construct final chunk
j.selected = j.selected[:0]
j.selected, joinResult.err = expression.VectorizedFilter(j.ctx.SessCtx.GetExprCtx().GetEvalCtx(), j.ctx.SessCtx.GetSessionVars().EnableVectorizedExpression, j.ctx.OtherCondition, chunk.NewIterator4Chunk(joinedChk), j.selected)
if joinResult.err != nil {
return false, joinResult
}
joinResult.err = j.buildResultAfterOtherCondition(joinResult.chk, joinedChk)
}
// if there is no other condition, the joinedChk is the final result
if joinResult.err != nil {
return false, joinResult
}
return true, joinResult
}
func (*innerJoinProbe) NeedScanRowTable() bool {
return false
}
func (*innerJoinProbe) ScanRowTable(*hashjoinWorkerResult, *sqlkiller.SQLKiller) *hashjoinWorkerResult {
panic("should not reach here")
}
func (*innerJoinProbe) InitForScanRowTable() {
panic("should not reach here")
}
func (*innerJoinProbe) IsScanRowTableDone() bool {
panic("should not reach here")
}