From 0355fa5edd6102457d63b46129cdfc6d69739110 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 15 Oct 2021 18:47:20 -0700 Subject: [PATCH 1/4] colexecjoin: make cross/merge join streaming with regards to left input This commit refactors the cross and merge join to be streaming with regards to the left input. Previously, we were using two spilling queues to consume both inputs first before proceeding to building the cross product (in case of the merge join this is needed when building from the buffered group). That approach is suboptimal because buffering only one side is sufficient, so this commit switches the cross join builder to operate in a streaming fashion with regards to the left input. This is done by building all result rows that correspond to the current left batch before proceeding to the next left batch and allows us to significantly reduce amount of copying and, thus, improving the performance. Release note: None --- pkg/sql/colexec/colexecjoin/crossjoiner.eg.go | 2505 ++++++++--------- pkg/sql/colexec/colexecjoin/crossjoiner.go | 406 +-- .../colexec/colexecjoin/crossjoiner_tmpl.go | 339 +-- .../colexec/colexecjoin/mergejoinbase.eg.go | 52 +- .../colexec/colexecjoin/mergejoinbase_tmpl.go | 12 +- pkg/sql/colexec/colexecjoin/mergejoiner.go | 554 ++-- .../colexecjoin/mergejoiner_exceptall.eg.go | 993 ++++--- .../colexecjoin/mergejoiner_fullouter.eg.go | 967 ++++--- .../colexecjoin/mergejoiner_inner.eg.go | 958 ++++--- .../mergejoiner_intersectall.eg.go | 962 ++++--- .../colexecjoin/mergejoiner_leftanti.eg.go | 959 ++++--- .../colexecjoin/mergejoiner_leftouter.eg.go | 963 ++++--- .../colexecjoin/mergejoiner_leftsemi.eg.go | 954 ++++--- .../colexecjoin/mergejoiner_rightanti.eg.go | 955 ++++--- .../colexecjoin/mergejoiner_rightouter.eg.go | 963 ++++--- .../colexecjoin/mergejoiner_rightsemi.eg.go | 955 ++++--- .../colexec/colexecjoin/mergejoiner_tmpl.go | 356 ++- pkg/sql/colexec/crossjoiner_test.go | 30 +- pkg/sql/colexec/mergejoiner_test.go | 27 + .../logictest/testdata/logic_test/cross_join | 84 + .../exec/execbuilder/testdata/vectorize_local | 25 + 21 files changed, 7393 insertions(+), 6626 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/cross_join diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go b/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go index 85fd93c7462d..7514686a8d94 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go @@ -26,1386 +26,1299 @@ var ( _ = types.BoolFamily ) +// buildFromLeftBatch is the body of buildFromLeftInput that templates out the +// fact whether the current left batch has a selection vector or not. +// execgen:inline +const _ = "template_buildFromLeftBatch" + // buildFromLeftInput builds part of the output of a cross join that comes from // the vectors of the left input. The new output tuples are put starting at // index destStartIdx and will not exceed the capacity of the output batch. It -// is assumed that setupBuilder has been called. +// is assumed that setupLeftBuilder and prepareForNextLeftBatch have been +// called. // The goal of this method is to repeat each tuple from the left input -// leftNumRepeats times. For set-operation joins only first setOpLeftSrcIdx -// tuples are built from. +// leftNumRepeats times. Only the tuples in [curSrcStartIdx, leftSrcEndIdx) are +// used from the current left batch. func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx int) { - var err error currentBatch := b.builderState.left.currentBatch - if currentBatch == nil || b.builderState.left.curSrcStartIdx == currentBatch.Length() { - // We need to get the next batch to build from if it is the first one or - // we have fully processed the previous one. - currentBatch, err = b.left.tuples.Dequeue(ctx) - if err != nil { - colexecerror.InternalError(err) - } - b.builderState.left.currentBatch = currentBatch - b.builderState.left.curSrcStartIdx = 0 - b.builderState.left.numRepeatsIdx = 0 - } - initialBuilderState := b.builderState.left b.left.unlimitedAllocator.PerformOperation( b.output.ColVecs()[:len(b.left.types)], func() { - leftNumRepeats := b.builderState.setup.leftNumRepeats - isSetOp := b.joinType.IsSetOpJoin() - outputCapacity := b.output.Capacity() - batchLength := currentBatch.Length() - for batchLength > 0 { - // Loop over every column. - LeftColLoop: - for colIdx := range b.left.types { - outStartIdx := destStartIdx - src := currentBatch.ColVec(colIdx) - srcNulls := src.Nulls() - out := b.output.ColVec(colIdx) - outNulls := out.Nulls() - switch b.left.canonicalTypeFamilies[colIdx] { - case types.BoolFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Bool() - outCol := out.Bool() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + if sel := currentBatch.Selection(); sel != nil { + { + // We'll be modifying the builder state as we go, but we'll need to be able + // to restore the state to initial for each column. + initialBuilderState := b.builderState.left + bs := &b.builderState.left + leftNumRepeats := b.builderState.setup.leftNumRepeats + leftSrcEndIdx := b.builderState.setup.leftSrcEndIdx + outputCapacity := b.output.Capacity() + var srcStartIdx int + // Loop over every column. + for colIdx := range b.left.types { + if colIdx > 0 { + // Restore the builder state so that this new column started off + // fresh. + *bs = initialBuilderState + } + outStartIdx := destStartIdx + src := currentBatch.ColVec(colIdx) + srcNulls := src.Nulls() + out := b.output.ColVec(colIdx) + outNulls := out.Nulls() + switch b.left.canonicalTypeFamilies[colIdx] { + case types.BoolFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Bool() + outCol := out.Bool() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + outStartIdx++ + bs.curSrcStartIdx++ } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.BytesFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Bytes() + outCol := out.Bytes() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case types.BytesFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Bytes() - outCol := out.Bytes() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.DecimalFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Decimal() + outCol := out.Decimal() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) + outStartIdx++ + bs.curSrcStartIdx++ } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.IntFamily: + switch b.left.types[colIdx].Width() { + case 16: + srcCol := src.Int16() + outCol := out.Int16() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 - } - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 - } - } - } - case types.DecimalFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Decimal() - outCol := out.Decimal() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + case 32: + srcCol := src.Int32() + outCol := out.Int32() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) + } + outStartIdx++ + bs.curSrcStartIdx++ } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + case -1: + default: + srcCol := src.Int64() + outCol := out.Int64() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case types.IntFamily: - switch b.left.types[colIdx].Width() { - case 16: - srcCol := src.Int16() - outCol := out.Int16() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.FloatFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Float64() + outCol := out.Float64() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + outStartIdx++ + bs.curSrcStartIdx++ } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.TimestampTZFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Timestamp() + outCol := out.Timestamp() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - case 32: - srcCol := src.Int32() - outCol := out.Int32() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.IntervalFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Interval() + outCol := out.Interval() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ + outStartIdx++ + bs.curSrcStartIdx++ } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.JsonFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.JSON() + outCol := out.JSON() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - case -1: - default: - srcCol := src.Int64() - outCol := out.Int64() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case typeconv.DatumVecCanonicalTypeFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Datum() + outCol := out.Datum() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) + outStartIdx++ + bs.curSrcStartIdx++ } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = sel[bs.curSrcStartIdx] + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + default: + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) + } + } + } + } else { + { + var sel []int = nil + // Remove the unused warning. + _ = sel + // We'll be modifying the builder state as we go, but we'll need to be able + // to restore the state to initial for each column. + initialBuilderState := b.builderState.left + bs := &b.builderState.left + leftNumRepeats := b.builderState.setup.leftNumRepeats + leftSrcEndIdx := b.builderState.setup.leftSrcEndIdx + outputCapacity := b.output.Capacity() + var srcStartIdx int + // Loop over every column. + for colIdx := range b.left.types { + if colIdx > 0 { + // Restore the builder state so that this new column started off + // fresh. + *bs = initialBuilderState + } + outStartIdx := destStartIdx + src := currentBatch.ColVec(colIdx) + srcNulls := src.Nulls() + out := b.output.ColVec(colIdx) + outNulls := out.Nulls() + switch b.left.canonicalTypeFamilies[colIdx] { + case types.BoolFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Bool() + outCol := out.Bool() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case types.FloatFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Float64() - outCol := out.Float64() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.BytesFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Bytes() + outCol := out.Bytes() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + outStartIdx++ + bs.curSrcStartIdx++ } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.DecimalFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Decimal() + outCol := out.Decimal() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case types.TimestampTZFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Timestamp() - outCol := out.Timestamp() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.IntFamily: + switch b.left.types[colIdx].Width() { + case 16: + srcCol := src.Int16() + outCol := out.Int16() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ + outStartIdx++ + bs.curSrcStartIdx++ } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + case 32: + srcCol := src.Int32() + outCol := out.Int32() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 - } - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 - } - } - } - case types.IntervalFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Interval() - outCol := out.Interval() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + case -1: + default: + srcCol := src.Int64() + outCol := out.Int64() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) + } + outStartIdx++ + bs.curSrcStartIdx++ } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.FloatFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Float64() + outCol := out.Float64() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case types.JsonFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.JSON() - outCol := out.JSON() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.TimestampTZFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Timestamp() + outCol := out.Timestamp() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + outStartIdx++ + bs.curSrcStartIdx++ } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case types.IntervalFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Interval() + outCol := out.Interval() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } - } - case typeconv.DatumVecCanonicalTypeFamily: - switch b.left.types[colIdx].Width() { - case -1: - default: - srcCol := src.Datum() - outCol := out.Datum() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop + case types.JsonFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.JSON() + outCol := out.JSON() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) + outStartIdx++ + bs.curSrcStartIdx++ } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - b.builderState.left.setOpLeftSrcIdx += toAppend } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { + } + } + case typeconv.DatumVecCanonicalTypeFamily: + switch b.left.types[colIdx].Width() { + case -1: + default: + srcCol := src.Datum() + outCol := out.Datum() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + _ = true + srcStartIdx = bs.curSrcStartIdx + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) outCol.Set(outStartIdx, val) - outStartIdx++ } + outStartIdx++ + bs.curSrcStartIdx++ } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + _ = true + srcStartIdx = bs.curSrcStartIdx + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ } - return } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 } } + default: + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) } - default: - colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) } - if colIdx == len(b.left.types)-1 { - // We have appended some tuples into the output batch from the current - // batch (the latter is now fully processed), so we need to adjust - // destStartIdx accordingly for the next batch. - destStartIdx = outStartIdx - } else { - b.builderState.left = initialBuilderState - } - } - // We have processed all tuples in the current batch from the - // buffered group, so we need to Dequeue the next one. - currentBatch, err = b.left.tuples.Dequeue(ctx) - if err != nil { - colexecerror.InternalError(err) } - b.builderState.left.currentBatch = currentBatch - batchLength = currentBatch.Length() - // We have transitioned to building from a new batch, so we - // need to update the builder state to build from the beginning - // of the new batch. - b.builderState.left.curSrcStartIdx = 0 - b.builderState.left.numRepeatsIdx = 0 - // We also need to update 'initialBuilderState' so that the - // builder state gets reset correctly in-between different - // columns in the loop above. - initialBuilderState = b.builderState.left } }, ) @@ -1414,30 +1327,34 @@ func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx i // buildFromRightInput builds part of the output of a cross join that comes from // the vectors of the right input. The new output tuples are put starting at // index destStartIdx and will not exceed the capacity of the output batch. It -// is assumed that setupBuilder has been called. +// is assumed that the right input has been fully consumed and is stored in +// b.rightTuples spilling queue. // The goal of this method is to repeat all tuples from the right input // rightNumRepeats times (i.e. repeating the whole list of tuples at once). func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx int) { var err error + bs := &b.builderState.right + rightNumRepeats := b.builderState.setup.rightNumRepeats b.right.unlimitedAllocator.PerformOperation( b.output.ColVecs()[b.builderState.rightColOffset:], func() { outStartIdx := destStartIdx outputCapacity := b.output.Capacity() - // Repeat the buffered tuples rightNumRepeats times. - for ; b.builderState.right.numRepeatsIdx < b.builderState.setup.rightNumRepeats; b.builderState.right.numRepeatsIdx++ { - currentBatch := b.builderState.right.currentBatch + // Repeat the buffered tuples rightNumRepeats times until we fill + // the output capacity. + for ; outStartIdx < outputCapacity && bs.numRepeatsIdx < rightNumRepeats; bs.numRepeatsIdx++ { + currentBatch := bs.currentBatch if currentBatch == nil { - currentBatch, err = b.right.tuples.Dequeue(ctx) + currentBatch, err = b.rightTuples.Dequeue(ctx) if err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = currentBatch - b.builderState.right.curSrcStartIdx = 0 + bs.currentBatch = currentBatch + bs.curSrcStartIdx = 0 } batchLength := currentBatch.Length() for batchLength > 0 { - toAppend := batchLength - b.builderState.right.curSrcStartIdx + toAppend := batchLength - bs.curSrcStartIdx if outStartIdx+toAppend > outputCapacity { toAppend = outputCapacity - outStartIdx } @@ -1459,10 +1376,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1470,8 +1387,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1486,10 +1403,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1497,8 +1414,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1513,10 +1430,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1524,8 +1441,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1539,10 +1456,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1550,8 +1467,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1562,10 +1479,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1573,8 +1490,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1586,10 +1503,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1597,8 +1514,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1613,10 +1530,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1624,8 +1541,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1640,10 +1557,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1651,8 +1568,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1667,10 +1584,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1678,8 +1595,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1694,10 +1611,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1705,8 +1622,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1721,10 +1638,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -1732,8 +1649,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -1744,34 +1661,38 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx } outStartIdx += toAppend - if toAppend < batchLength-b.builderState.right.curSrcStartIdx { + if toAppend < batchLength-bs.curSrcStartIdx { // If we haven't materialized all the tuples from the // batch, then we are ready to emit the output batch. - b.builderState.right.curSrcStartIdx += toAppend + bs.curSrcStartIdx += toAppend return } // We have fully processed the current batch, so we need to // get the next one. - currentBatch, err = b.right.tuples.Dequeue(ctx) + currentBatch, err = b.rightTuples.Dequeue(ctx) if err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = currentBatch + bs.currentBatch = currentBatch batchLength = currentBatch.Length() - b.builderState.right.curSrcStartIdx = 0 - - if outStartIdx == outputCapacity { - // We reached the capacity of the output batch, so we - // can emit it. - return - } + bs.curSrcStartIdx = 0 } // We have fully processed all the batches from the right side, // so we need to Rewind the queue. - if err := b.right.tuples.Rewind(); err != nil { + if err := b.rightTuples.Rewind(); err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = nil + bs.currentBatch = nil } }) } + +// buildFromLeftBatch is the body of buildFromLeftInput that templates out the +// fact whether the current left batch has a selection vector or not. +// execgen:inline +const _ = "inlined_buildFromLeftBatch_true" + +// buildFromLeftBatch is the body of buildFromLeftInput that templates out the +// fact whether the current left batch has a selection vector or not. +// execgen:inline +const _ = "inlined_buildFromLeftBatch_false" diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner.go b/pkg/sql/colexec/colexecjoin/crossjoiner.go index 01da98daa21d..7029d9560378 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner.go @@ -63,16 +63,18 @@ type crossJoiner struct { *joinHelper unlimitedAllocator *colmem.Allocator - inputsConsumed bool + rightInputConsumed bool outputTypes []*types.T maxOutputBatchMemSize int64 - numTotalOutputTuples int - numAlreadyEmitted int // isLeftAllNulls and isRightAllNulls indicate whether the output vectors // corresponding to the left and right inputs, respectively, should consist // only of NULL values. This is the case when we have right or left, - // respectively, unmatched tuples. Note that only one can be set to true. + // respectively, unmatched tuples. isLeftAllNulls, isRightAllNulls bool + // done indicates that the cross joiner has fully built its output and + // closed the spilling queue. Once set to true, only zero-length batches are + // emitted. + done bool } var _ colexecop.ClosableOperator = &crossJoiner{} @@ -88,17 +90,21 @@ func (c *crossJoiner) Init(ctx context.Context) { } func (c *crossJoiner) Next() coldata.Batch { - if !c.inputsConsumed { - c.consumeInputs(c.Ctx) + if c.done { + return coldata.ZeroBatch + } + if !c.rightInputConsumed { + c.consumeRightInput(c.Ctx) c.setupForBuilding() } - if c.numTotalOutputTuples == c.numAlreadyEmitted { + willEmit := c.willEmit() + if willEmit == 0 { if err := c.Close(); err != nil { colexecerror.InternalError(err) } + c.done = true return coldata.ZeroBatch } - willEmit := c.numTotalOutputTuples - c.numAlreadyEmitted c.output, _ = c.unlimitedAllocator.ResetMaybeReallocate( c.outputTypes, c.output, willEmit, c.maxOutputBatchMemSize, ) @@ -120,112 +126,157 @@ func (c *crossJoiner) Next() coldata.Batch { } } c.output.SetLength(willEmit) - c.numAlreadyEmitted += willEmit + c.builderState.numEmittedCurLeftBatch += willEmit + c.builderState.numEmittedTotal += willEmit return c.output } -// consumeInputs determines the kind of information the cross joiner needs from -// its inputs (in some cases, we don't need to buffer all input tuples) and -// consumes the inputs accordingly. -func (c *crossJoiner) consumeInputs(ctx context.Context) { - c.inputsConsumed = true - var needLeftTuples bool +// readNextLeftBatch fetches the next batch from the left input, prepares the +// builder for it (assuming that all rows in the batch contribute to the cross +// product), and returns the length of the batch. +func (c *crossJoiner) readNextLeftBatch() int { + leftBatch := c.inputOne.Next() + c.prepareForNextLeftBatch(leftBatch, 0 /* startIdx */, leftBatch.Length()) + return leftBatch.Length() +} + +// consumeRightInput determines the kind of information the cross joiner needs +// from its right input (in some cases, we don't need to buffer all tuples from +// the right) and consumes the right input accordingly. It also checks whether +// we need any tuples from the left and possibly reads a single batch, depending +// on the join type. +func (c *crossJoiner) consumeRightInput(ctx context.Context) { + c.rightInputConsumed = true var needRightTuples, needOnlyNumRightTuples bool switch c.joinType { case descpb.InnerJoin, descpb.LeftOuterJoin, descpb.RightOuterJoin, descpb.FullOuterJoin: - needLeftTuples = true + c.needLeftTuples = true needRightTuples = true case descpb.LeftSemiJoin: // With LEFT SEMI join we only need to know whether the right input is // empty or not. - c.right.numTuples = c.inputTwo.Next().Length() - needLeftTuples = c.right.numTuples != 0 + c.numRightTuples = c.inputTwo.Next().Length() + c.needLeftTuples = c.numRightTuples != 0 case descpb.RightSemiJoin: // With RIGHT SEMI join we only need to know whether the left input is // empty or not. - c.left.numTuples = c.inputOne.Next().Length() - needRightTuples = c.left.numTuples != 0 + needRightTuples = c.readNextLeftBatch() != 0 case descpb.LeftAntiJoin: // With LEFT ANTI join we only need to know whether the right input is // empty or not. - c.right.numTuples = c.inputTwo.Next().Length() - needLeftTuples = c.right.numTuples == 0 + c.numRightTuples = c.inputTwo.Next().Length() + c.needLeftTuples = c.numRightTuples == 0 case descpb.RightAntiJoin: // With RIGHT ANTI join we only need to know whether the left input is // empty or not. - c.left.numTuples = c.inputOne.Next().Length() - needRightTuples = c.left.numTuples == 0 + needRightTuples = c.readNextLeftBatch() == 0 case descpb.IntersectAllJoin, descpb.ExceptAllJoin: // With set-operation joins we only need the number of tuples from the // right input. - needLeftTuples = true + c.needLeftTuples = true needOnlyNumRightTuples = true default: colexecerror.InternalError(errors.AssertionFailedf("unexpected join type %s", c.joinType.String())) } - if needRightTuples && needOnlyNumRightTuples { - colexecerror.InternalError(errors.AssertionFailedf("both needRightTuples and needOnlyNumRightTuples are true")) - } - - if needLeftTuples { - for { - batch := c.inputOne.Next() - c.left.tuples.Enqueue(ctx, batch) - if batch.Length() == 0 { - break - } - c.left.numTuples += batch.Length() - } - } - if needRightTuples { + if needRightTuples || needOnlyNumRightTuples { for { batch := c.inputTwo.Next() - c.right.tuples.Enqueue(ctx, batch) - if batch.Length() == 0 { - break + if needRightTuples { + c.rightTuples.Enqueue(ctx, batch) } - c.right.numTuples += batch.Length() - } - } - if needOnlyNumRightTuples { - for { - batch := c.inputTwo.Next() if batch.Length() == 0 { break } - c.right.numTuples += batch.Length() + c.numRightTuples += batch.Length() } } } +// setupForBuilding prepares the cross joiner to build the output. This method +// must be called after the right input has been fully "processed" (which might +// mean it wasn't fully read, depending on the join type). func (c *crossJoiner) setupForBuilding() { - c.numTotalOutputTuples = c.calculateOutputCount() - switch c.joinType { case descpb.LeftOuterJoin: - c.isRightAllNulls = c.right.numTuples == 0 + c.isRightAllNulls = c.numRightTuples == 0 case descpb.RightOuterJoin: - c.isLeftAllNulls = c.left.numTuples == 0 + c.isLeftAllNulls = c.readNextLeftBatch() == 0 case descpb.FullOuterJoin: - c.isLeftAllNulls = c.left.numTuples == 0 - c.isRightAllNulls = c.right.numTuples == 0 - } - // In order for buildFrom*Input methods to work in the unmatched cases, we - // "lie" that there is a single tuple on the opposite side which results in - // the builder methods repeating the tuples only once, and that's exactly - // what we want. - if c.isLeftAllNulls { - c.left.numTuples = 1 + c.isLeftAllNulls = c.readNextLeftBatch() == 0 + c.isRightAllNulls = c.numRightTuples == 0 + case descpb.ExceptAllJoin: + // For EXCEPT ALL joins we build # left tuples - # right tuples output + // rows (if positive), so we have to discard first numRightTuples rows + // from the left. + for c.numRightTuples > 0 { + leftBatch := c.inputOne.Next() + c.builderState.left.currentBatch = leftBatch + if leftBatch.Length() == 0 { + break + } else if leftBatch.Length() > c.numRightTuples { + // The current left batch is the first one that contains tuples + // without a "match". + c.prepareForNextLeftBatch(leftBatch, c.numRightTuples, leftBatch.Length()) + break + } + c.numRightTuples -= leftBatch.Length() + } } + // In order for canEmit method to work in the unmatched cases, we "lie" + // that there is a single tuple on the right side which results in the + // builder method repeating the tuples only once, and that's exactly what we + // want. if c.isRightAllNulls { - c.right.numTuples = 1 + c.numRightTuples = 1 } - c.setupBuilder() - if c.isLeftAllNulls { - c.left.numTuples = 0 + c.setupLeftBuilder() +} + +// willEmit returns the number of tuples the cross joiner will emit based on the +// current left batch. If the current left batch is exhausted, then a new left +// batch is fetched. If 0 is returned, then the cross joiner has fully emitted +// the output. +func (c *crossJoiner) willEmit() int { + if c.needLeftTuples { + if c.isLeftAllNulls { + if c.isRightAllNulls { + // This can happen only in FULL OUTER join when both inputs are + // empty. + return 0 + } + // All tuples from the right are unmatched and will be emitted once. + c.builderState.setup.rightNumRepeats = 1 + return c.numRightTuples - c.builderState.numEmittedCurLeftBatch + } + if c.builderState.left.currentBatch == nil || c.canEmit() == 0 { + // Get the next left batch if we haven't fetched one yet or we + // have fully built the output using the current left batch. + if c.readNextLeftBatch() == 0 { + return 0 + } + } + return c.canEmit() } - if c.isRightAllNulls { - c.right.numTuples = 0 + switch c.joinType { + case descpb.LeftSemiJoin, descpb.LeftAntiJoin: + // We don't need the left tuples, and in case of LEFT SEMI/ANTI this + // means that the right input was empty/non-empty, so the cross join + // is empty. + return 0 + case descpb.RightSemiJoin, descpb.RightAntiJoin: + if c.numRightTuples == 0 { + // For RIGHT SEMI, we didn't fetch any right tuples if the left + // input was empty; for RIGHT ANTI - if the left input wasn't + // empty. In both such cases the cross join is empty. + return 0 + } + return c.canEmit() + default: + colexecerror.InternalError(errors.AssertionFailedf( + "unexpectedly don't need left tuples for %s", c.joinType, + )) + // This code is unreachable, but the compiler cannot infer that. + return 0 } } @@ -245,15 +296,12 @@ func (c *crossJoiner) Reset(ctx context.Context) { r.Reset(ctx) } c.crossJoinerBase.Reset(ctx) - c.inputsConsumed = false - c.numTotalOutputTuples = 0 - c.numAlreadyEmitted = 0 + c.rightInputConsumed = false c.isLeftAllNulls = false c.isRightAllNulls = false + c.done = false } -// TODO(yuzefovich): use two separate unlimited allocators giving the right side -// larger limit (since it might need to be read multiple times). func newCrossJoinerBase( unlimitedAllocator *colmem.Allocator, joinType descpb.JoinType, @@ -269,32 +317,22 @@ func newCrossJoinerBase( unlimitedAllocator: unlimitedAllocator, types: leftTypes, canonicalTypeFamilies: typeconv.ToCanonicalTypeFamilies(leftTypes), - tuples: colexecutils.NewSpillingQueue( - &colexecutils.NewSpillingQueueArgs{ - UnlimitedAllocator: unlimitedAllocator, - Types: leftTypes, - MemoryLimit: memoryLimit, - DiskQueueCfg: cfg, - FDSemaphore: fdSemaphore, - DiskAcc: diskAcc, - }, - ), }, right: cjState{ unlimitedAllocator: unlimitedAllocator, types: rightTypes, canonicalTypeFamilies: typeconv.ToCanonicalTypeFamilies(rightTypes), - tuples: colexecutils.NewRewindableSpillingQueue( - &colexecutils.NewSpillingQueueArgs{ - UnlimitedAllocator: unlimitedAllocator, - Types: rightTypes, - MemoryLimit: memoryLimit, - DiskQueueCfg: cfg, - FDSemaphore: fdSemaphore, - DiskAcc: diskAcc, - }, - ), }, + rightTuples: colexecutils.NewRewindableSpillingQueue( + &colexecutils.NewSpillingQueueArgs{ + UnlimitedAllocator: unlimitedAllocator, + Types: rightTypes, + MemoryLimit: memoryLimit, + DiskQueueCfg: cfg, + FDSemaphore: fdSemaphore, + DiskAcc: diskAcc, + }, + ), } if joinType.ShouldIncludeLeftColsInOutput() { base.builderState.rightColOffset = len(leftTypes) @@ -303,12 +341,26 @@ func newCrossJoinerBase( } type crossJoinerBase struct { - initHelper colexecop.InitHelper - joinType descpb.JoinType - left, right cjState - builderState struct { + initHelper colexecop.InitHelper + joinType descpb.JoinType + left, right cjState + numRightTuples int + rightTuples *colexecutils.SpillingQueue + needLeftTuples bool + builderState struct { setup cjBuilderSetupState left, right cjMutableBuilderState + + // numEmittedCurLeftBatch tracks the number of joined rows returned + // based on the current left batch. It is reset on every call to + // prepareForNextLeftBatch. + numEmittedCurLeftBatch int + + // numEmittedTotal tracks the number of rows that have been emitted + // since the crossJoinerBase has been reset. It is only used in RIGHT + // SEMI, RIGHT ANTI, and INTERSECT ALL joins. + numEmittedTotal int + // rightColOffset indicates the number of vectors in the output batch // that should be "skipped" when building from the right input. rightColOffset int @@ -320,142 +372,108 @@ func (b *crossJoinerBase) init(ctx context.Context) { b.initHelper.Init(ctx) } -func (b *crossJoinerBase) setupBuilder() { - switch b.joinType { - case descpb.IntersectAllJoin: - // For INTERSECT ALL joins we build min(left.numTuples, right.numTuples) - // tuples. - if b.left.numTuples < b.right.numTuples { - b.builderState.setup.leftSrcEndIdx = b.left.numTuples - } else { - b.builderState.setup.leftSrcEndIdx = b.right.numTuples - } - case descpb.ExceptAllJoin: - // For EXCEPT ALL joins we build left.numTuples-right.numTuples tuples - // (if positive). - if b.left.numTuples > b.right.numTuples { - b.builderState.setup.leftSrcEndIdx = b.left.numTuples - b.right.numTuples - } - default: - b.builderState.setup.leftSrcEndIdx = b.left.numTuples - } +func (b *crossJoinerBase) setupLeftBuilder() { switch b.joinType { case descpb.LeftSemiJoin, descpb.IntersectAllJoin, descpb.ExceptAllJoin: b.builderState.setup.leftNumRepeats = 1 case descpb.LeftAntiJoin: // LEFT ANTI cross join emits all left tuples repeated once only if the // right input is empty. - if b.right.numTuples == 0 { + if b.numRightTuples == 0 { b.builderState.setup.leftNumRepeats = 1 } default: - b.builderState.setup.leftNumRepeats = b.right.numTuples + b.builderState.setup.leftNumRepeats = b.numRightTuples } +} + +// prepareForNextLeftBatch sets up the crossJoinerBase to build based on a new +// batch coming from the left input. Only rows with ordinals in +// [startIdx, endIdx) range will be used for the cross join. +func (b *crossJoinerBase) prepareForNextLeftBatch(batch coldata.Batch, startIdx, endIdx int) { + b.builderState.numEmittedCurLeftBatch = 0 + b.builderState.left.currentBatch = batch + b.builderState.left.curSrcStartIdx = startIdx + b.builderState.left.numRepeatsIdx = 0 + b.builderState.right.numRepeatsIdx = 0 + + if b.joinType == descpb.IntersectAllJoin { + // Intersect all is special because we need to count how many tuples + // from the right we have already used up. + if b.builderState.numEmittedTotal+endIdx-startIdx >= b.numRightTuples { + // The current left batch is the last one that contains tuples with + // a "match". + b.builderState.setup.leftSrcEndIdx = b.numRightTuples - b.builderState.numEmittedTotal + startIdx + } else { + // The current left batch is still emitted fully. + b.builderState.setup.leftSrcEndIdx = endIdx + } + } else { + b.builderState.setup.leftSrcEndIdx = endIdx + } + switch b.joinType { - case descpb.RightSemiJoin: + case descpb.InnerJoin, descpb.LeftOuterJoin, descpb.RightOuterJoin, descpb.FullOuterJoin: + b.builderState.setup.rightNumRepeats = endIdx - startIdx + case descpb.RightSemiJoin, descpb.RightAntiJoin: b.builderState.setup.rightNumRepeats = 1 - case descpb.RightAntiJoin: - // RIGHT ANTI cross join emits all right tuples repeated once only if - // the left input is empty. - if b.left.numTuples == 0 { - b.builderState.setup.rightNumRepeats = 1 - } - default: - b.builderState.setup.rightNumRepeats = b.left.numTuples } } -// calculateOutputCount returns the total number of tuples that are emitted by -// the cross join given already initialized left and right side states. -func (b *crossJoinerBase) calculateOutputCount() int { +// canEmit returns the number of output rows that can still be emitted based on +// the current left batch. It supports only the case when both left and right +// inputs are not empty. +func (b *crossJoinerBase) canEmit() int { switch b.joinType { - case descpb.InnerJoin: - return b.left.numTuples * b.right.numTuples - case descpb.LeftOuterJoin: - if b.right.numTuples == 0 { - return b.left.numTuples - } - return b.left.numTuples * b.right.numTuples - case descpb.RightOuterJoin: - if b.left.numTuples == 0 { - return b.right.numTuples - } - return b.left.numTuples * b.right.numTuples - case descpb.FullOuterJoin: - if b.left.numTuples == 0 || b.right.numTuples == 0 { - return b.left.numTuples + b.right.numTuples - } - return b.left.numTuples * b.right.numTuples - case descpb.LeftSemiJoin: - if b.right.numTuples == 0 { + case descpb.LeftSemiJoin, descpb.IntersectAllJoin, descpb.ExceptAllJoin: + return b.builderState.setup.leftSrcEndIdx - b.builderState.left.curSrcStartIdx + case descpb.LeftAntiJoin: + if b.numRightTuples != 0 { return 0 } - return b.left.numTuples + return b.builderState.setup.leftSrcEndIdx - b.builderState.left.curSrcStartIdx case descpb.RightSemiJoin: - if b.left.numTuples == 0 { - return 0 - } - return b.right.numTuples - case descpb.LeftAntiJoin: - if b.right.numTuples != 0 { + // RIGHT SEMI cross join emits all right tuples repeated once iff the + // left input is not empty. + if b.builderState.setup.leftSrcEndIdx == b.builderState.left.curSrcStartIdx { return 0 } - return b.left.numTuples + return b.numRightTuples - b.builderState.numEmittedTotal case descpb.RightAntiJoin: - if b.left.numTuples != 0 { - return 0 - } - return b.right.numTuples - case descpb.IntersectAllJoin: - if b.right.numTuples < b.left.numTuples { - return b.right.numTuples - } - return b.left.numTuples - case descpb.ExceptAllJoin: - if b.right.numTuples > b.left.numTuples { + // RIGHT ANTI cross join emits all right tuples repeated once iff the + // left input is empty. + if b.builderState.setup.leftSrcEndIdx != b.builderState.left.curSrcStartIdx { return 0 } - return b.left.numTuples - b.right.numTuples + return b.numRightTuples - b.builderState.numEmittedTotal default: - colexecerror.InternalError(errors.AssertionFailedf("unexpected join type %s", b.joinType.String())) - // Unreachable code. - return 0 + return b.builderState.setup.rightNumRepeats*b.numRightTuples - b.builderState.numEmittedCurLeftBatch } } func (b *crossJoinerBase) Reset(ctx context.Context) { - if b.left.tuples != nil { - b.left.tuples.Reset(ctx) - } - if b.right.tuples != nil { - b.right.tuples.Reset(ctx) + if b.rightTuples != nil { + b.rightTuples.Reset(ctx) } - b.left.numTuples = 0 - b.right.numTuples = 0 + b.numRightTuples = 0 b.builderState.left.reset() b.builderState.right.reset() + b.builderState.numEmittedCurLeftBatch = 0 + b.builderState.numEmittedTotal = 0 } func (b *crossJoinerBase) Close() error { ctx := b.initHelper.EnsureCtx() - var lastErr error - if b.left.tuples != nil { - lastErr = b.left.tuples.Close(ctx) - } - if b.right.tuples != nil { - if err := b.right.tuples.Close(ctx); err != nil { - lastErr = err - } + if b.rightTuples != nil { + return b.rightTuples.Close(ctx) } - return lastErr + return nil } type cjState struct { unlimitedAllocator *colmem.Allocator types []*types.T canonicalTypeFamilies []types.Family - tuples *colexecutils.SpillingQueue - numTuples int } type cjBuilderSetupState struct { @@ -481,16 +499,10 @@ type cjMutableBuilderState struct { // numRepeatsIdx tracks the number of times a "group" has already been // repeated. numRepeatsIdx int - // setOpLeftSrcIdx tracks the current tuple's index from the left input for - // set operation joins. INTERSECT ALL and EXCEPT ALL joins are special - // because they need to build the output partially (namely, for exactly - // leftSrcEndIdx number of tuples which could span multiple batches). - setOpLeftSrcIdx int } func (s *cjMutableBuilderState) reset() { s.currentBatch = nil s.curSrcStartIdx = 0 s.numRepeatsIdx = 0 - s.setOpLeftSrcIdx = 0 } diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go index 4f097649719c..e01a6da7316e 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go @@ -38,198 +38,125 @@ var ( _ = types.BoolFamily ) +// buildFromLeftBatch is the body of buildFromLeftInput that templates out the +// fact whether the current left batch has a selection vector or not. +// execgen:inline +// execgen:template +func buildFromLeftBatch(b *crossJoinerBase, currentBatch coldata.Batch, sel []int, hasSel bool) { + if !hasSel { + // Remove the unused warning. + _ = sel + } + // We'll be modifying the builder state as we go, but we'll need to be able + // to restore the state to initial for each column. + initialBuilderState := b.builderState.left + bs := &b.builderState.left + leftNumRepeats := b.builderState.setup.leftNumRepeats + leftSrcEndIdx := b.builderState.setup.leftSrcEndIdx + outputCapacity := b.output.Capacity() + var srcStartIdx int + // Loop over every column. + for colIdx := range b.left.types { + if colIdx > 0 { + // Restore the builder state so that this new column started off + // fresh. + *bs = initialBuilderState + } + outStartIdx := destStartIdx + src := currentBatch.ColVec(colIdx) + srcNulls := src.Nulls() + out := b.output.ColVec(colIdx) + outNulls := out.Nulls() + switch b.left.canonicalTypeFamilies[colIdx] { + // {{range .}} + case _CANONICAL_TYPE_FAMILY: + switch b.left.types[colIdx].Width() { + // {{range .WidthOverloads}} + case _TYPE_WIDTH: + srcCol := src.TemplateType() + outCol := out.TemplateType() + if leftNumRepeats == 1 { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each tuple one time. + if hasSel { + srcStartIdx = sel[bs.curSrcStartIdx] + } else { + srcStartIdx = bs.curSrcStartIdx + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNull(outStartIdx) + } else { + val := srcCol.Get(srcStartIdx) + outCol.Set(outStartIdx, val) + } + outStartIdx++ + bs.curSrcStartIdx++ + } + } else { + // Loop over every tuple in the current batch. + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + if hasSel { + srcStartIdx = sel[bs.curSrcStartIdx] + } else { + srcStartIdx = bs.curSrcStartIdx + } + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + if srcNulls.NullAt(srcStartIdx) { + outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) + outStartIdx += toAppend + } else { + val := srcCol.Get(srcStartIdx) + for i := 0; i < toAppend; i++ { + outCol.Set(outStartIdx, val) + outStartIdx++ + } + } + } + } + // {{end}} + } + // {{end}} + default: + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) + } + } +} + // buildFromLeftInput builds part of the output of a cross join that comes from // the vectors of the left input. The new output tuples are put starting at // index destStartIdx and will not exceed the capacity of the output batch. It -// is assumed that setupBuilder has been called. +// is assumed that setupLeftBuilder and prepareForNextLeftBatch have been +// called. // // The goal of this method is to repeat each tuple from the left input -// leftNumRepeats times. For set-operation joins only first setOpLeftSrcIdx -// tuples are built from. +// leftNumRepeats times. Only the tuples in [curSrcStartIdx, leftSrcEndIdx) are +// used from the current left batch. func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx int) { - var err error currentBatch := b.builderState.left.currentBatch - if currentBatch == nil || b.builderState.left.curSrcStartIdx == currentBatch.Length() { - // We need to get the next batch to build from if it is the first one or - // we have fully processed the previous one. - currentBatch, err = b.left.tuples.Dequeue(ctx) - if err != nil { - colexecerror.InternalError(err) - } - b.builderState.left.currentBatch = currentBatch - b.builderState.left.curSrcStartIdx = 0 - b.builderState.left.numRepeatsIdx = 0 - } - initialBuilderState := b.builderState.left b.left.unlimitedAllocator.PerformOperation( b.output.ColVecs()[:len(b.left.types)], func() { - leftNumRepeats := b.builderState.setup.leftNumRepeats - isSetOp := b.joinType.IsSetOpJoin() - outputCapacity := b.output.Capacity() - batchLength := currentBatch.Length() - for batchLength > 0 { - // Loop over every column. - LeftColLoop: - for colIdx := range b.left.types { - outStartIdx := destStartIdx - src := currentBatch.ColVec(colIdx) - srcNulls := src.Nulls() - out := b.output.ColVec(colIdx) - outNulls := out.Nulls() - switch b.left.canonicalTypeFamilies[colIdx] { - // {{range .}} - case _CANONICAL_TYPE_FAMILY: - switch b.left.types[colIdx].Width() { - // {{range .WidthOverloads}} - case _TYPE_WIDTH: - srcCol := src.TemplateType() - outCol := out.TemplateType() - if leftNumRepeats == 1 { - // Loop over every tuple in the current batch. - for b.builderState.left.curSrcStartIdx < batchLength { - // Repeat each tuple one time. - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - b.builderState.left.setOpLeftSrcIdx++ - } - - srcStartIdx := b.builderState.left.curSrcStartIdx - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNull(outStartIdx) - } else { - val := srcCol.Get(srcStartIdx) - outCol.Set(outStartIdx, val) - } - outStartIdx++ - b.builderState.left.curSrcStartIdx++ - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - } - } else { - // Loop over every tuple in the current batch. - for ; b.builderState.left.curSrcStartIdx < batchLength; b.builderState.left.curSrcStartIdx++ { - // Repeat each row leftNumRepeats times. - srcStartIdx := b.builderState.left.curSrcStartIdx - toAppend := leftNumRepeats - b.builderState.left.numRepeatsIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx - } - - if isSetOp { - if b.builderState.left.setOpLeftSrcIdx == b.builderState.setup.leftSrcEndIdx { - // We have fully materialized first leftSrcEndIdx - // tuples in the current column, so we need to - // either transition to the next column or exit. - if colIdx == len(b.left.types)-1 { - // This is the last column. - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - b.builderState.left.setOpLeftSrcIdx += toAppend - } - - if srcNulls.NullAt(srcStartIdx) { - outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend - } else { - val := srcCol.Get(srcStartIdx) - for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ - } - } - - if outStartIdx == outputCapacity { - // We reached the capacity of the output vector, - // so we move to the next column. - if colIdx == len(b.left.types)-1 { - // This is the last column. - b.builderState.left.numRepeatsIdx += toAppend - if b.builderState.left.numRepeatsIdx == leftNumRepeats { - // The current tuple has already been repeated - // the desired number of times, so we advance - // the source index. - b.builderState.left.curSrcStartIdx++ - b.builderState.left.numRepeatsIdx = 0 - } - return - } - // We need to start building the next column - // with the same initial builder state as the - // current column. - b.builderState.left = initialBuilderState - continue LeftColLoop - } - // We fully processed the current tuple for the current - // column, and before moving on to the next one, we need - // to reset numRepeatsIdx (so that the next tuple would - // be repeated leftNumRepeats times). - b.builderState.left.numRepeatsIdx = 0 - } - } - // {{end}} - } - // {{end}} - default: - colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) - } - if colIdx == len(b.left.types)-1 { - // We have appended some tuples into the output batch from the current - // batch (the latter is now fully processed), so we need to adjust - // destStartIdx accordingly for the next batch. - destStartIdx = outStartIdx - } else { - b.builderState.left = initialBuilderState - } - } - // We have processed all tuples in the current batch from the - // buffered group, so we need to Dequeue the next one. - currentBatch, err = b.left.tuples.Dequeue(ctx) - if err != nil { - colexecerror.InternalError(err) - } - b.builderState.left.currentBatch = currentBatch - batchLength = currentBatch.Length() - // We have transitioned to building from a new batch, so we - // need to update the builder state to build from the beginning - // of the new batch. - b.builderState.left.curSrcStartIdx = 0 - b.builderState.left.numRepeatsIdx = 0 - // We also need to update 'initialBuilderState' so that the - // builder state gets reset correctly in-between different - // columns in the loop above. - initialBuilderState = b.builderState.left + if sel := currentBatch.Selection(); sel != nil { + buildFromLeftBatch(b, currentBatch, sel, true /* hasSel */) + } else { + buildFromLeftBatch(b, currentBatch, nil, false /* hasSel */) } }, ) @@ -238,31 +165,35 @@ func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx i // buildFromRightInput builds part of the output of a cross join that comes from // the vectors of the right input. The new output tuples are put starting at // index destStartIdx and will not exceed the capacity of the output batch. It -// is assumed that setupBuilder has been called. +// is assumed that the right input has been fully consumed and is stored in +// b.rightTuples spilling queue. // // The goal of this method is to repeat all tuples from the right input // rightNumRepeats times (i.e. repeating the whole list of tuples at once). func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx int) { var err error + bs := &b.builderState.right + rightNumRepeats := b.builderState.setup.rightNumRepeats b.right.unlimitedAllocator.PerformOperation( b.output.ColVecs()[b.builderState.rightColOffset:], func() { outStartIdx := destStartIdx outputCapacity := b.output.Capacity() - // Repeat the buffered tuples rightNumRepeats times. - for ; b.builderState.right.numRepeatsIdx < b.builderState.setup.rightNumRepeats; b.builderState.right.numRepeatsIdx++ { - currentBatch := b.builderState.right.currentBatch + // Repeat the buffered tuples rightNumRepeats times until we fill + // the output capacity. + for ; outStartIdx < outputCapacity && bs.numRepeatsIdx < rightNumRepeats; bs.numRepeatsIdx++ { + currentBatch := bs.currentBatch if currentBatch == nil { - currentBatch, err = b.right.tuples.Dequeue(ctx) + currentBatch, err = b.rightTuples.Dequeue(ctx) if err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = currentBatch - b.builderState.right.curSrcStartIdx = 0 + bs.currentBatch = currentBatch + bs.curSrcStartIdx = 0 } batchLength := currentBatch.Length() for batchLength > 0 { - toAppend := batchLength - b.builderState.right.curSrcStartIdx + toAppend := batchLength - bs.curSrcStartIdx if outStartIdx+toAppend > outputCapacity { toAppend = outputCapacity - outStartIdx } @@ -285,10 +216,10 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx // Optimization in the case that group length is 1, use assign // instead of copy. if toAppend == 1 { - if srcNulls.NullAt(b.builderState.right.curSrcStartIdx) { + if srcNulls.NullAt(bs.curSrcStartIdx) { outNulls.SetNull(outStartIdx) } else { - v := srcCol.Get(b.builderState.right.curSrcStartIdx) + v := srcCol.Get(bs.curSrcStartIdx) outCol.Set(outStartIdx, v) } } else { @@ -296,8 +227,8 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx coldata.SliceArgs{ Src: src, DestIdx: outStartIdx, - SrcStartIdx: b.builderState.right.curSrcStartIdx, - SrcEndIdx: b.builderState.right.curSrcStartIdx + toAppend, + SrcStartIdx: bs.curSrcStartIdx, + SrcEndIdx: bs.curSrcStartIdx + toAppend, }, ) } @@ -310,34 +241,28 @@ func (b *crossJoinerBase) buildFromRightInput(ctx context.Context, destStartIdx } outStartIdx += toAppend - if toAppend < batchLength-b.builderState.right.curSrcStartIdx { + if toAppend < batchLength-bs.curSrcStartIdx { // If we haven't materialized all the tuples from the // batch, then we are ready to emit the output batch. - b.builderState.right.curSrcStartIdx += toAppend + bs.curSrcStartIdx += toAppend return } // We have fully processed the current batch, so we need to // get the next one. - currentBatch, err = b.right.tuples.Dequeue(ctx) + currentBatch, err = b.rightTuples.Dequeue(ctx) if err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = currentBatch + bs.currentBatch = currentBatch batchLength = currentBatch.Length() - b.builderState.right.curSrcStartIdx = 0 - - if outStartIdx == outputCapacity { - // We reached the capacity of the output batch, so we - // can emit it. - return - } + bs.curSrcStartIdx = 0 } // We have fully processed all the batches from the right side, // so we need to Rewind the queue. - if err := b.right.tuples.Rewind(); err != nil { + if err := b.rightTuples.Rewind(); err != nil { colexecerror.InternalError(err) } - b.builderState.right.currentBatch = nil + bs.currentBatch = nil } }) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoinbase.eg.go b/pkg/sql/colexec/colexecjoin/mergejoinbase.eg.go index ce0ffcb2286b..765aa1537a76 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoinbase.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoinbase.eg.go @@ -31,17 +31,13 @@ var ( ) // isBufferedGroupFinished checks to see whether or not the buffered group -// corresponding to input continues in batch. +// corresponding to the first tuple continues in batch. func (o *mergeJoinBase) isBufferedGroupFinished( - input *mergeJoinInput, batch coldata.Batch, rowIdx int, + input *mergeJoinInput, firstTuple []coldata.Vec, batch coldata.Batch, rowIdx int, ) bool { if batch.Length() == 0 { return true } - bufferedGroup := o.bufferedGroup.left - if input == &o.right { - bufferedGroup = o.bufferedGroup.right - } tupleToLookAtIdx := rowIdx sel := batch.Selection() if sel != nil { @@ -62,7 +58,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -73,7 +69,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Bool() + bufferedCol := firstTuple[colIdx].Bool() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Bool() curVal := col.Get(tupleToLookAtIdx) @@ -107,7 +103,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -118,7 +114,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Bytes() + bufferedCol := firstTuple[colIdx].Bytes() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Bytes() curVal := col.Get(tupleToLookAtIdx) @@ -144,7 +140,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -155,7 +151,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Decimal() + bufferedCol := firstTuple[colIdx].Decimal() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Decimal() curVal := col.Get(tupleToLookAtIdx) @@ -180,7 +176,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -191,7 +187,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Int16() + bufferedCol := firstTuple[colIdx].Int16() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Int16() curVal := col.Get(tupleToLookAtIdx) @@ -224,7 +220,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -235,7 +231,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Int32() + bufferedCol := firstTuple[colIdx].Int32() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Int32() curVal := col.Get(tupleToLookAtIdx) @@ -269,7 +265,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -280,7 +276,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Int64() + bufferedCol := firstTuple[colIdx].Int64() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Int64() curVal := col.Get(tupleToLookAtIdx) @@ -317,7 +313,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -328,7 +324,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Float64() + bufferedCol := firstTuple[colIdx].Float64() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Float64() curVal := col.Get(tupleToLookAtIdx) @@ -373,7 +369,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -384,7 +380,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Timestamp() + bufferedCol := firstTuple[colIdx].Timestamp() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Timestamp() curVal := col.Get(tupleToLookAtIdx) @@ -417,7 +413,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -428,7 +424,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Interval() + bufferedCol := firstTuple[colIdx].Interval() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Interval() curVal := col.Get(tupleToLookAtIdx) @@ -454,7 +450,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -465,7 +461,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].JSON() + bufferedCol := firstTuple[colIdx].JSON() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).JSON() curVal := col.Get(tupleToLookAtIdx) @@ -497,7 +493,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -508,7 +504,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].Datum() + bufferedCol := firstTuple[colIdx].Datum() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).Datum() curVal := col.Get(tupleToLookAtIdx) diff --git a/pkg/sql/colexec/colexecjoin/mergejoinbase_tmpl.go b/pkg/sql/colexec/colexecjoin/mergejoinbase_tmpl.go index d7168305d113..8b4b1641473e 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoinbase_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/mergejoinbase_tmpl.go @@ -57,17 +57,13 @@ func _ASSIGN_EQ(_, _, _, _, _, _ interface{}) int { // */}} // isBufferedGroupFinished checks to see whether or not the buffered group -// corresponding to input continues in batch. +// corresponding to the first tuple continues in batch. func (o *mergeJoinBase) isBufferedGroupFinished( - input *mergeJoinInput, batch coldata.Batch, rowIdx int, + input *mergeJoinInput, firstTuple []coldata.Vec, batch coldata.Batch, rowIdx int, ) bool { if batch.Length() == 0 { return true } - bufferedGroup := o.bufferedGroup.left - if input == &o.right { - bufferedGroup = o.bufferedGroup.right - } tupleToLookAtIdx := rowIdx sel := batch.Selection() if sel != nil { @@ -89,7 +85,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( // right side being an input) this check will always return false since // nulls couldn't be buffered up though. // TODO(yuzefovich): consider templating this. - bufferedNull := bufferedGroup.firstTuple[colIdx].MaybeHasNulls() && bufferedGroup.firstTuple[colIdx].Nulls().NullAt(0) + bufferedNull := firstTuple[colIdx].MaybeHasNulls() && firstTuple[colIdx].Nulls().NullAt(0) incomingNull := batch.ColVec(int(colIdx)).MaybeHasNulls() && batch.ColVec(int(colIdx)).Nulls().NullAt(tupleToLookAtIdx) if o.joinType.IsSetOpJoin() { if bufferedNull && incomingNull { @@ -100,7 +96,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( if bufferedNull || incomingNull { return true } - bufferedCol := bufferedGroup.firstTuple[colIdx].TemplateType() + bufferedCol := firstTuple[colIdx].TemplateType() prevVal := bufferedCol.Get(0) col := batch.ColVec(int(colIdx)).TemplateType() curVal := col.Get(tupleToLookAtIdx) diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner.go b/pkg/sql/colexec/colexecjoin/mergejoiner.go index 45ddc5ad89bb..b8593dcd39b6 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner.go @@ -32,6 +32,161 @@ import ( "github.com/marusama/semaphore" ) +// The merge join operator uses a probe and build approach to generate the join. +// What this means is that instead of going through and expanding the cross +// product row by row, the operator performs two passes. The first pass +// generates a list of groups of matching rows based on the equality columns +// (where a "group" represents a contiguous set of rows that match on the +// equality columns). The second pass is where the groups and their associated +// cross products are materialized into the full output. +// +// A group describes the indexes of rows within the probing batches that are +// equal on the equality columns. For example, if we have a group with indexes +// [0, 3) on the left and [1, 3) on the right, it means that first three rows +// from the left batch match with the 2nd and 3rd row from the right batch. In +// order to produce the output we need to repeat each left row the number of +// times equal to the size of the group on the right side (2 in the example); +// and we need to repeat all rows from the right side of the group the number of +// times equal to the size of the group on the left side (3 in the example). +// +// There is a complication, however, when a group might extend in the next +// batch, and we call such a group a "buffered group". For example, imagine we +// have a batch with rows {0, 0, 1, 2}, then the group containing the row with +// value 2 is not complete with the current batch, and it might (or might not) +// extend into the next batch. +// +// We handle these buffered groups definitely depending on the side of the join: +// - the left buffered group is processed in a streaming fashion, one batch from +// the left input at a time +// - the right buffered group needs to be fully buffered before we can build the +// output. +// +// Let's walk through a concrete example to show the general flow of state +// transitions as well as to better show the terminology we use throughout the +// merge joiner code. +// +// Imagine that we operate on batches of size 3 and have the following input +// data: +// left right +// batch 1 -1 batch 1 -1 +// -1 -2 +// -2 -3 +// batch 2 -2 batch 2 -3 +// -2 -3 +// -2 -3 +// batch 3 -3 batch 3 -3 +// -3 +// -3 +// batch 4 -3 +// +// We start of with reading first batches from both inputs (we store them in +// mjProberState) and performing the probing step. We find that we have a single +// group such that it contains rows with indices [0, 2) on the left and with +// indices [0, 1) on the right. This group is fully contained within the batch +// (i.e. it is not a buffered group), so we'll be able to populate the cross +// product during the build step. The last row with index 2 in the left batch +// starts the left buffered group (we don't know yet whether the second batch +// from the left will have a row with value -2 or not), so we will have to +// process the last row as the buffered group; we append a single row with index +// 1 from the right batch into the right buffered group. +// +// At this point the probing is done, and the merge joiner transitions into +// mjBuildFromBatch. Here we take the single group ([0, 2) on the left and +// [0, 1) on the right) to produce a cross product, so we add two result rows +// to the output batch. We don't emit the output yet because we want to fill up +// the output to capacity. The output is now {(-1, -1), (-1, -1)}. +// +// Since we have something in the buffered group state, we transition into +// building from the buffered group. There, we first have to make sure that we +// buffer all rows (into a spilling queue) that are part of the right buffered +// group; however, in this case we have found the boundaries of the right group +// during the probing, so we don't need to read any new batches - the right +// buffered group only contains a single row with index 1. At the same time, the +// left buffered group was started with a single row with index 2, so we are +// ready to perform the cross product. This produces a single row that we put +// into the output batch. Now the output is at capacity, so we emit {(-1, -1), +// (-1, -1), (-2, -2)}, but the merge joiner remains in mjBuildFromBufferedGroup +// state. +// +// When Next() is called again, we see that we have fully emitted the cross +// product based on the current left batch, so we fetch the second batch from +// the left in continueLeftBufferedGroup(). There, after fetching the second +// batch we first check whether the first row of the batch still belongs to the +// left buffered group (it does), and then we run the ordered distinct in order +// to find the first row that is not part of the current buffered group. In this +// case it turns out that all 3 rows still belong to the same buffered group +// that have value -2 in their equality columns. We produce a cross product of 3 +// rows on the left and a single row on the right, put it into the output and +// emit {(-2, -2), (-2, -2), (-2, -2)}. The merge joiner remains in +// mjBuildFromBufferedGroup state. +// +// When Next() is called, we fetch the third batch from the left and find that +// its first row (value -3) is different from the current left buffered group +// (value -2), so we have finished processing the buffered group for value -2. +// The merge joiner transitions into mjEntry state (where we don't do anything +// since we already have the prober state set up) and then into mjProbe state. +// +// As a reminder, we working with a batch {(-3), (-3), (-3)} from the left and +// {(-1), (-2), (-3)} from the right. Note that on the left we're starting from +// index 0 and from index 2 on the right. During probing, we find that all 3 +// rows from the left and the row with index 2 on the right are part of the same +// group which might extend into the next batches from either side, so we start +// a buffered group on the left and append the single row into the right +// buffered group. +// +// We don't have any groups fully contained within the probing batches to build +// from, so we transition into building from the buffered group. There, we will +// actually read the second and the third batches from the right and append all +// of them into the spilling queue. The right input is now exhausted. We now +// have 5 rows with value -3 in the spilling queue, and we proceed to build the +// cross product with 3 rows from the left. This will result in 15 rows emitted +// in 5 batches. +// +// Once we've emitted those 5 batches, we see that we have fully built the cross +// product based on the third left batch, so we fetch the last fourth batch from +// the left. It has a single row that is still part of the same group, so we +// will build the cross product against 5 rows in the spilling queue of the +// right buffered group. +// +// At this point, we have exhausted both inputs, and we're done. + +// mjState represents the state of the merge joiner. +type mjState int + +const ( + // mjEntry is the entry state of the merge joiner where all the batches and + // indices are properly set, regardless if Next was called the first time or + // the 1000th time. This state also routes into the correct state based on + // the prober state after setup. + mjEntry mjState = iota + + // mjSourceFinished is the state in which one of the input sources has no + // more available batches, thus signaling that the joiner should begin + // wrapping up execution by outputting any remaining groups in state. After + // reaching this state, we can only build from the batch. + mjSourceFinished + + // mjProbe is the main probing state in which the groups for the current + // batch are determined. + mjProbe + + // mjBuildFromBatch indicates that we should be building from the current + // probing batches. Note that in such case we might have multiple groups to + // build. + mjBuildFromBatch + + // mjBuildFromBufferedGroup indicates that we should be building from the + // current left batch and the right buffered group. Note that in such case + // we have at most one group to build and are building the output one batch + // from the left input at a time. + mjBuildFromBufferedGroup + + // mjDone is the final state of the merge joiner in which it'll be returning + // only zero-length batches. In this state, the disk infrastructure is + // cleaned up. + mjDone +) + // group is an ADT representing a contiguous set of rows that match on their // equality columns. type group struct { @@ -53,36 +208,14 @@ type group struct { unmatched bool } -// mjBuildFrom is an indicator of which source we're building the output from. -type mjBuildFrom int - -const ( - // mjBuildFromBatch indicates that we should be building from the current - // probing batches. Note that in such case we might have multiple groups to - // build. - mjBuildFromBatch mjBuildFrom = iota - // mjBuildFromBufferedGroup indicates that we should be building from the - // buffered group. Note that in such case we might have at most one group to - // build. - mjBuildFromBufferedGroup -) - // mjBuilderState contains all the state required to execute the build phase. type mjBuilderState struct { - buildFrom mjBuildFrom - // Fields to identify the groups in the input sources. lGroups []group rGroups []group // outCount keeps record of the current number of rows in the output. outCount int - // outFinished is used to determine if the builder is finished outputting - // the groups from input. - outFinished bool - - totalOutCountFromBufferedGroup int - alreadyEmittedFromBufferedGroup int // Cross product materialization state. left mjBuilderCrossProductState @@ -98,71 +231,47 @@ type mjBuilderCrossProductState struct { numRepeatsIdx int } -// mjBufferedGroup is a helper struct that stores information about the tuples -// from both inputs for the buffered group. -type mjBufferedGroup struct { - // firstTuple stores a single tuple that was first in the buffered group. - firstTuple []coldata.Vec - scratchBatch coldata.Batch -} - type mjBufferedGroupState struct { - // Local buffer for the last left and right groups which is used when the - // group ends with a batch and the group on each side needs to be saved to - // state in order to be able to continue it in the next batch. - left mjBufferedGroup - right mjBufferedGroup - helper *crossJoinerBase - needToReset bool + // leftFirstTuple is the first tuple of the left buffered group. It is set + // only in case the left buffered group spans more than one input batch. + leftFirstTuple []coldata.Vec + // leftGroupStartIdx is the position within the current left batch where the + // left buffered group starts. If the group spans multiple batches, this + // will be set to 0 on all consecutive batches. + // + // Note that proberState.lIdx indicates the exclusive end position for the + // left buffered group within the current batch. + leftGroupStartIdx int + // leftBatchDone indicates whether the output from the current left batch + // has been fully built. + leftBatchDone bool + // rightFirstTuple is the first tuple of the right buffered group. It is set + // only in case the right buffered group spans more than one input batch. + rightFirstTuple []coldata.Vec + // rightScratchBatch is a scratch space for copying the tuples out of the + // right input batches before enqueueing them into the spilling queue. + rightScratchBatch coldata.Batch + + // helper is the building facility for the cross join of the buffered group. + helper *crossJoinerBase } // mjProberState contains all the state required to execute in the probing // phase. type mjProberState struct { // Fields to save the "working" batches to state in between outputs. - lBatch coldata.Batch - rBatch coldata.Batch + lBatch coldata.Batch + rBatch coldata.Batch + // lIdx indicates the index of the first left tuple that hasn't been probed + // yet. lIdx int lLength int + // rIdx indicates the index of the first right tuple that hasn't been probed + // yet. rIdx int rLength int } -// mjState represents the state of the merge joiner. -type mjState int - -const ( - // mjEntry is the entry state of the merge joiner where all the batches and - // indices are properly set, regardless if Next was called the first time or - // the 1000th time. This state also routes into the correct state based on - // the prober state after setup. - mjEntry mjState = iota - - // mjSourceFinished is the state in which one of the input sources has no - // more available batches, thus signaling that the joiner should begin - // wrapping up execution by outputting any remaining groups in state. - mjSourceFinished - - // mjFinishBufferedGroup is the state in which the previous state resulted in - // a group that ended with a batch. Such a group was buffered, and this state - // finishes that group and builds the output. - mjFinishBufferedGroup - - // mjProbe is the main probing state in which the groups for the current - // batch are determined. - mjProbe - - // mjBuild is the state in which the groups determined by the probing states - // are built, i.e. materialized to the output member by creating the cross - // product. - mjBuild - - // mjDone is the final state of the merge joiner in which it'll be returning - // only zero-length batches. In this state, the disk infrastructure is - // cleaned up. - mjDone -) - type mergeJoinInput struct { // eqCols specify the indices of the source table equality columns during the // merge join. @@ -194,21 +303,6 @@ type mergeJoinInput struct { source colexecop.Operator } -// The merge join operator uses a probe and build approach to generate the -// join. What this means is that instead of going through and expanding the -// cross product row by row, the operator performs two passes. -// The first pass generates a list of groups of matching rows based on the -// equality columns (where a "group" represents a contiguous set of rows that -// match on the equality columns). -// The second pass is where the groups and their associated cross products are -// materialized into the full output. - -// Two buffers are used, one for the group on the left table and one for the -// group on the right table. These buffers are only used if the group ends with -// a batch, to make sure that we don't miss any cross product entries while -// expanding the groups (leftGroups and rightGroups) when a group spans -// multiple batches. - // NewMergeJoinOp returns a new merge join operator with the given spec that // implements sort-merge join. It performs a merge on the left and right input // sources, based on the equality columns, assuming both inputs are in sorted @@ -465,11 +559,9 @@ type mergeJoinBase struct { right mergeJoinInput // Output buffer definition. - output coldata.Batch - outputTypes []*types.T - // outputReady is a flag to indicate that merge joiner is ready to emit an - // output batch. - outputReady bool + output coldata.Batch + outputCapacity int + outputTypes []*types.T // Local buffer for the "working" repeated groups. groups circularGroupsBuffer @@ -492,10 +584,8 @@ func (o *mergeJoinBase) Reset(ctx context.Context) { if r, ok := o.right.source.(colexecop.Resetter); ok { r.Reset(ctx) } - o.outputReady = false o.state = mjEntry o.bufferedGroup.helper.Reset(ctx) - o.bufferedGroup.needToReset = false o.proberState.lBatch = nil o.proberState.rBatch = nil o.resetBuilderCrossProductState() @@ -506,10 +596,10 @@ func (o *mergeJoinBase) Init(ctx context.Context) { return } o.outputTypes = o.joinType.MakeOutputTypes(o.left.sourceTypes, o.right.sourceTypes) - o.bufferedGroup.left.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( + o.bufferedGroup.leftFirstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( o.left.sourceTypes, 1, /* capacity */ ).ColVecs() - o.bufferedGroup.right.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( + o.bufferedGroup.rightFirstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( o.right.sourceTypes, 1, /* capacity */ ).ColVecs() o.bufferedGroup.helper = newCrossJoinerBase( @@ -532,40 +622,49 @@ func (o *mergeJoinBase) resetBuilderCrossProductState() { o.builderState.right.reset() } -// appendToBufferedGroup appends all the tuples from batch that are part of the -// same group as the ones in the buffered group that corresponds to the input -// source. This needs to happen when a group starts at the end of an input +// startLeftBufferedGroup initializes the left buffered group. It will set the +// first tuple in case the left buffered group doesn't end in the current left +// batch. +func (o *mergeJoinBase) startLeftBufferedGroup(sel []int, groupStartIdx int, groupLength int) { + if groupStartIdx+groupLength < o.proberState.lLength { + // The left buffered group is complete within the current left batch, so + // we don't need to copy the first tuple. + return + } + o.unlimitedAllocator.PerformOperation(o.bufferedGroup.leftFirstTuple, func() { + for colIdx := range o.left.sourceTypes { + o.bufferedGroup.leftFirstTuple[colIdx].Copy( + coldata.SliceArgs{ + Src: o.proberState.lBatch.ColVec(colIdx), + Sel: sel, + DestIdx: 0, + SrcStartIdx: groupStartIdx, + SrcEndIdx: groupStartIdx + 1, + }, + ) + } + }) +} + +// appendToRightBufferedGroup appends the tuples in +// [groupStartIdx; groupStartIdx+groupLength) range from the current right +// batch. This needs to happen when a group starts at the end of an input // batch and can continue into the following batches. +// // A zero-length batch needs to be appended when no more batches will be -// appended to the buffered group. -func (o *mergeJoinBase) appendToBufferedGroup( - input *mergeJoinInput, batch coldata.Batch, sel []int, groupStartIdx int, groupLength int, -) { - var ( - bufferedGroup *mjBufferedGroup - sourceTypes []*types.T - bufferedTuples *colexecutils.SpillingQueue - numBufferedTuples int - ) - if input == &o.left { - sourceTypes = o.left.sourceTypes - bufferedGroup = &o.bufferedGroup.left - bufferedTuples = o.bufferedGroup.helper.left.tuples - numBufferedTuples = o.bufferedGroup.helper.left.numTuples - o.bufferedGroup.helper.left.numTuples += groupLength - } else { - sourceTypes = o.right.sourceTypes - bufferedGroup = &o.bufferedGroup.right - bufferedTuples = o.bufferedGroup.helper.right.tuples - numBufferedTuples = o.bufferedGroup.helper.right.numTuples - o.bufferedGroup.helper.right.numTuples += groupLength - } - if batch.Length() == 0 || groupLength == 0 { +// appended to the buffered group (which can be achieved by specifying an empty +// range with groupLength == 0). +func (o *mergeJoinBase) appendToRightBufferedGroup(sel []int, groupStartIdx int, groupLength int) { + bufferedTuples := o.bufferedGroup.helper.rightTuples + if groupLength == 0 { // We have finished appending to this buffered group, so we need to // Enqueue a zero-length batch per the contract of the spilling queue. bufferedTuples.Enqueue(o.Ctx, coldata.ZeroBatch) return } + sourceTypes := o.right.sourceTypes + numBufferedTuples := o.bufferedGroup.helper.numRightTuples + o.bufferedGroup.helper.numRightTuples += groupLength // TODO(yuzefovich): for LEFT/RIGHT ANTI joins we only need to store the // first tuple (in order to find the boundaries of the groups) since all // of the buffered tuples do have a match and, thus, don't contribute to @@ -574,12 +673,15 @@ func (o *mergeJoinBase) appendToBufferedGroup( // tuples from the left side and count the number of tuples on the right. // TODO(yuzefovich): for LEFT/RIGHT SEMI joins we only need to buffer tuples // from one side (left/right respectively). - if numBufferedTuples == 0 { - o.unlimitedAllocator.PerformOperation(bufferedGroup.firstTuple, func() { + if numBufferedTuples == 0 && groupStartIdx+groupLength == o.proberState.rLength { + // Set the right first tuple only if this is the first call to this + // method for the current right buffered group and if the group doesn't + // end in the current batch. + o.unlimitedAllocator.PerformOperation(o.bufferedGroup.rightFirstTuple, func() { for colIdx := range sourceTypes { - bufferedGroup.firstTuple[colIdx].Copy( + o.bufferedGroup.rightFirstTuple[colIdx].Copy( coldata.SliceArgs{ - Src: batch.ColVec(colIdx), + Src: o.proberState.rBatch.ColVec(colIdx), Sel: sel, DestIdx: 0, SrcStartIdx: groupStartIdx, @@ -593,14 +695,16 @@ func (o *mergeJoinBase) appendToBufferedGroup( // We don't impose any memory limits on the scratch batch because we rely on // the inputs to the merge joiner to produce reasonably sized batches. const maxBatchMemSize = math.MaxInt64 - bufferedGroup.scratchBatch, _ = o.unlimitedAllocator.ResetMaybeReallocate( - input.sourceTypes, bufferedGroup.scratchBatch, groupLength, maxBatchMemSize, + o.bufferedGroup.rightScratchBatch, _ = o.unlimitedAllocator.ResetMaybeReallocate( + sourceTypes, o.bufferedGroup.rightScratchBatch, groupLength, maxBatchMemSize, ) - o.unlimitedAllocator.PerformOperation(bufferedGroup.scratchBatch.ColVecs(), func() { - for colIdx := range input.sourceTypes { - bufferedGroup.scratchBatch.ColVec(colIdx).Copy( + // TODO(yuzefovich): SpillingQueue.Enqueue deep-copies the batch too. Think + // through whether the copy here can be avoided altogether. + o.unlimitedAllocator.PerformOperation(o.bufferedGroup.rightScratchBatch.ColVecs(), func() { + for colIdx := range sourceTypes { + o.bufferedGroup.rightScratchBatch.ColVec(colIdx).Copy( coldata.SliceArgs{ - Src: batch.ColVec(colIdx), + Src: o.proberState.rBatch.ColVec(colIdx), Sel: sel, DestIdx: 0, SrcStartIdx: groupStartIdx, @@ -608,83 +712,105 @@ func (o *mergeJoinBase) appendToBufferedGroup( }, ) } - bufferedGroup.scratchBatch.SetLength(groupLength) + o.bufferedGroup.rightScratchBatch.SetLength(groupLength) }) - bufferedTuples.Enqueue(o.Ctx, bufferedGroup.scratchBatch) + bufferedTuples.Enqueue(o.Ctx, o.bufferedGroup.rightScratchBatch) } -// setBuilderSourceToBatch sets the builder state to use groups from the -// circular group buffer and the batches from input. This happens when we have -// groups that are fully contained within a single input batch from each of the -// sources. -func (o *mergeJoinBase) setBuilderSourceToBatch() { - o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() - o.builderState.buildFrom = mjBuildFromBatch +// sourceFinished returns true if either of input sources has no more rows. +func (o *mergeJoinBase) sourceFinished() bool { + return o.proberState.lLength == 0 || o.proberState.rLength == 0 } -// initProberState sets the batches, lengths, and current indices to the right -// locations given the last iteration of the operator. -func (o *mergeJoinBase) initProberState() { - // If this is the first batch or we're done with the current batch, get the - // next batch. - if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { - o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() - o.proberState.lLength = o.proberState.lBatch.Length() - } - if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { - o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() - o.proberState.rLength = o.proberState.rBatch.Length() +// continueLeftBufferedGroup fetches the next batch from the left input and +// and updates the probing and buffered group states accordingly. +func (o *mergeJoinBase) continueLeftBufferedGroup() { + // Get the next batch from the left. + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + o.bufferedGroup.leftGroupStartIdx = 0 + if o.proberState.lLength == 0 { + // The left input has been fully exhausted. + return } - if o.bufferedGroup.needToReset { - o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + // Check whether the first tuple of this batch is still part of the left + // buffered group. + if o.isBufferedGroupFinished(&o.left, o.bufferedGroup.leftFirstTuple, o.proberState.lBatch, 0 /* rowIdx */) { + return } -} -// nonEmptyBufferedGroup returns true if there is a buffered group that needs -// to be finished. -func (o *mergeJoinBase) nonEmptyBufferedGroup() bool { - return o.bufferedGroup.helper.left.numTuples > 0 || o.bufferedGroup.helper.right.numTuples > 0 -} + // It is ok that we might call Init() multiple times - it'll be a noop after + // the first one. + o.left.distincter.Init(o.Ctx) + o.left.distincter.(colexecop.Resetter).Reset(o.Ctx) + // Ignore the first row of the distincter since we already know that we are + // in the same group and, thus, the row is not distinct, regardless of what + // the distincter outputs. + groupLength := 1 + var sel []int + o.left.distincterInput.SetBatch(o.proberState.lBatch) + o.left.distincter.Next() + + sel = o.proberState.lBatch.Selection() + if sel != nil { + for ; groupLength < o.proberState.lLength; groupLength++ { + if o.left.distinctOutput[sel[groupLength]] { + // We found the beginning of a new group! + break + } + } + } else { + for ; groupLength < o.proberState.lLength; groupLength++ { + if o.left.distinctOutput[groupLength] { + // We found the beginning of a new group! + break + } + } + } -// sourceFinished returns true if either of input sources has no more rows. -func (o *mergeJoinBase) sourceFinished() bool { - return o.proberState.lLength == 0 || o.proberState.rLength == 0 + // Zero out the distinct output for the next time we use the distincter on + // the left input. + copy(o.left.distinctOutput[:o.proberState.lLength], colexecutils.ZeroBoolColumn) + o.proberState.lIdx += groupLength } -// finishBufferedGroup appends a zero-length batch to the buffered group which -// is required by the contract of the spilling queue. -func (o *mergeJoinBase) finishBufferedGroup(input *mergeJoinInput) { - o.appendToBufferedGroup( - input, coldata.ZeroBatch, nil, /* sel */ - 0 /* groupStartIdx */, 0, /* groupLength */ +// finishRightBufferedGroup appends a zero-length batch to the right buffered +// group which is required by the contract of the spilling queue. Note that it +// is safe to call this method multiple times (only the first one is not a +// noop). +func (o *mergeJoinBase) finishRightBufferedGroup() { + o.appendToRightBufferedGroup( + nil /* sel */, 0 /* groupStartIdx */, 0, /* groupLength */ ) } -// completeBufferedGroup extends the buffered group corresponding to input. -// First, we check that the first row in batch is still part of the same group. -// If this is the case, we use the Distinct operator to find the first -// occurrence in batch (or subsequent batches) that doesn't match the current -// group. +// completeRightBufferedGroup extends the right buffered group. It will read all +// tuples from the right input that are part of the current right buffered group +// (which must have been initialized via appendToRightBufferedGroup). +// // NOTE: we will be buffering all batches until we find such non-matching tuple -// (or until we exhaust the input). -// TODO(yuzefovich): this can be refactored so that only the right side does -// unbounded buffering. -// SIDE EFFECT: can append to the buffered group corresponding to the source. -func (o *mergeJoinBase) completeBufferedGroup( - input *mergeJoinInput, batch coldata.Batch, rowIdx int, -) (_ coldata.Batch, idx int, batchLength int) { - batchLength = batch.Length() - if o.isBufferedGroupFinished(input, batch, rowIdx) { - o.finishBufferedGroup(input) - return batch, rowIdx, batchLength +// (or until we exhaust the right input). +func (o *mergeJoinBase) completeRightBufferedGroup() { + // Get the next batch from the right. + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() + // The right input has been fully exhausted. + if o.proberState.rLength == 0 { + o.finishRightBufferedGroup() + return + } + // Check whether the first tuple of this batch is still part of the right + // buffered group. + if o.isBufferedGroupFinished(&o.right, o.bufferedGroup.rightFirstTuple, o.proberState.rBatch, 0 /* rowIdx */) { + o.finishRightBufferedGroup() + return } isBufferedGroupComplete := false // It is ok that we might call Init() multiple times - it'll be a noop after // the first one. - input.distincter.Init(o.Ctx) - input.distincter.(colexecop.Resetter).Reset(o.Ctx) + o.right.distincter.Init(o.Ctx) + o.right.distincter.(colexecop.Resetter).Reset(o.Ctx) // Ignore the first row of the distincter in the first pass since we already // know that we are in the same group and, thus, the row is not distinct, // regardless of what the distincter outputs. @@ -696,22 +822,22 @@ func (o *mergeJoinBase) completeBufferedGroup( // previous iterations had only the matching tuples to the buffered group, // so the distincter - in a sense - compares the incoming tuples to the // first tuple of the first iteration (which we know is the same group). - input.distincterInput.SetBatch(batch) - input.distincter.Next() + o.right.distincterInput.SetBatch(o.proberState.rBatch) + o.right.distincter.Next() - sel = batch.Selection() + sel = o.proberState.rBatch.Selection() var groupLength int if sel != nil { - for groupLength = loopStartIndex; groupLength < batchLength; groupLength++ { - if input.distinctOutput[sel[groupLength]] { + for groupLength = loopStartIndex; groupLength < o.proberState.rLength; groupLength++ { + if o.right.distinctOutput[sel[groupLength]] { // We found the beginning of a new group! isBufferedGroupComplete = true break } } } else { - for groupLength = loopStartIndex; groupLength < batchLength; groupLength++ { - if input.distinctOutput[groupLength] { + for groupLength = loopStartIndex; groupLength < o.proberState.rLength; groupLength++ { + if o.right.distinctOutput[groupLength] { // We found the beginning of a new group! isBufferedGroupComplete = true break @@ -720,42 +846,26 @@ func (o *mergeJoinBase) completeBufferedGroup( } // Zero out the distinct output for the next pass. - copy(input.distinctOutput[:batchLength], colexecutils.ZeroBoolColumn) + copy(o.right.distinctOutput[:o.proberState.rLength], colexecutils.ZeroBoolColumn) loopStartIndex = 0 // Buffer all the tuples that are part of the buffered group. - o.appendToBufferedGroup(input, batch, sel, rowIdx, groupLength) - rowIdx += groupLength + o.appendToRightBufferedGroup(sel, o.proberState.rIdx, groupLength) + o.proberState.rIdx += groupLength if !isBufferedGroupComplete { // The buffered group is still not complete which means that we have // just appended all the tuples from batch to it, so we need to get a // fresh batch from the input. - rowIdx, batch = 0, input.source.Next() - batchLength = batch.Length() - if batchLength == 0 { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() + if o.proberState.rLength == 0 { // The input has been exhausted, so the buffered group is now complete. isBufferedGroupComplete = true - o.finishBufferedGroup(input) } } } - - return batch, rowIdx, batchLength -} - -// finishProbe completes the buffered groups on both sides of the input. -func (o *mergeJoinBase) finishProbe() { - o.proberState.lBatch, o.proberState.lIdx, o.proberState.lLength = o.completeBufferedGroup( - &o.left, - o.proberState.lBatch, - o.proberState.lIdx, - ) - o.proberState.rBatch, o.proberState.rIdx, o.proberState.rLength = o.completeBufferedGroup( - &o.right, - o.proberState.rBatch, - o.proberState.rIdx, - ) + o.finishRightBufferedGroup() } func (o *mergeJoinBase) Close() error { diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go index db453acee5f5..083dded076b4 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinExceptAllOp{} func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -214,16 +213,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -437,16 +436,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -652,16 +651,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -899,16 +898,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1154,16 +1153,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1410,16 +1409,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1693,16 +1692,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1948,16 +1947,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2170,16 +2169,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2403,16 +2402,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2630,16 +2629,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2727,7 +2726,6 @@ EqLoop: func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2893,16 +2891,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3116,16 +3114,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3331,16 +3329,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3578,16 +3576,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3833,16 +3831,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4089,16 +4087,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4372,16 +4370,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4627,16 +4625,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4849,16 +4847,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5082,16 +5080,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5309,16 +5307,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5406,7 +5404,6 @@ EqLoop: func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -5572,16 +5569,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5795,16 +5792,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6010,16 +6007,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6257,16 +6254,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6512,16 +6509,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6768,16 +6765,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7051,16 +7048,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7306,16 +7303,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7528,16 +7525,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7761,16 +7758,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7988,16 +7985,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8085,7 +8082,6 @@ EqLoop: func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -8251,16 +8247,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8474,16 +8470,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8689,16 +8685,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8936,16 +8932,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9191,16 +9187,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9447,16 +9443,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9730,16 +9726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9985,16 +9981,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10207,16 +10203,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10440,16 +10436,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10667,16 +10663,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10790,7 +10786,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -10838,8 +10833,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10903,8 +10898,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10968,8 +10963,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11032,8 +11027,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11093,8 +11088,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11155,8 +11150,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11220,8 +11215,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11285,8 +11280,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11350,8 +11345,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11415,8 +11410,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11480,8 +11475,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11554,8 +11549,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11618,8 +11613,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11682,8 +11677,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11745,8 +11740,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11805,8 +11800,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11866,8 +11861,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11930,8 +11925,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11994,8 +11989,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12058,8 +12053,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12122,8 +12117,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12186,8 +12181,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12258,7 +12253,6 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -12294,8 +12288,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12361,8 +12355,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12428,8 +12422,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12494,8 +12488,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12557,8 +12551,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12621,8 +12615,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12688,8 +12682,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12755,8 +12749,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12822,8 +12816,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12889,8 +12883,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12956,8 +12950,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13033,8 +13027,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13100,8 +13094,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13167,8 +13161,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13233,8 +13227,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13296,8 +13290,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13360,8 +13354,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13427,8 +13421,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13494,8 +13488,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13561,8 +13555,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13628,8 +13622,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13695,8 +13689,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -13780,20 +13774,6 @@ func (o *mergeJoinExceptAllOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinExceptAllOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -13820,14 +13800,13 @@ func (o *mergeJoinExceptAllOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinExceptAllOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinExceptAllOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { if !groups[i].unmatched { // "Matched" groups are not outputted in LEFT ANTI, RIGHT ANTI, @@ -13835,51 +13814,152 @@ func (o *mergeJoinExceptAllOp) calculateOutputCount(groups []group) int { // they do not contribute to the output count. continue } - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinExceptAllOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinExceptAllOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - } + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinExceptAllOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + // For EXCEPT ALL joins we build # left tuples - # right tuples output rows + // (if positive), so we have to discard first numRightTuples rows from the + // left. + numSkippedLeft := 0 + for { + groupLength := o.proberState.lIdx - o.bufferedGroup.leftGroupStartIdx + if numSkippedLeft+groupLength > o.bufferedGroup.helper.numRightTuples { + // The current left batch is the first one that contains tuples + // without a "match". + break + } + numSkippedLeft += groupLength + var groupFinished bool + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + groupFinished = true } else { - o.builderState.outFinished = true + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + groupFinished = o.proberState.lIdx == 0 } - o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) + if groupFinished { + // We have less matching tuples on the left than on the right, so we + // don't emit any output for this buffered group. + o.bufferedGroup.helper.Reset(o.Ctx) + o.state = mjEntry + return } + } + // We might need to skip some tuples in the current left batch since they + // still had matches with the right side. + toSkipInThisBatch := o.bufferedGroup.helper.numRightTuples - numSkippedLeft + startIdx := o.bufferedGroup.leftGroupStartIdx + toSkipInThisBatch - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinExceptAllOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) + } + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount + } else { + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) + } + o.builderState.outCount += willEmit + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false + } } } @@ -13887,30 +13967,29 @@ func (o *mergeJoinExceptAllOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the right one, @@ -13919,49 +13998,51 @@ func (o *mergeJoinExceptAllOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.lIdx < o.proberState.lLength { o.exhaustLeftSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the left into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go index a613f1b7086b..29a35664057a 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinFullOuterOp{} func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -211,16 +210,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -459,16 +458,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -691,16 +690,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -955,16 +954,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1238,16 +1237,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1522,16 +1521,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1833,16 +1832,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2124,16 +2123,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2370,16 +2369,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2620,16 +2619,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2870,16 +2869,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2989,7 +2988,6 @@ EqLoop: func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -3152,16 +3150,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3400,16 +3398,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3632,16 +3630,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3896,16 +3894,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4179,16 +4177,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4463,16 +4461,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4774,16 +4772,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5065,16 +5063,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5311,16 +5309,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5561,16 +5559,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5811,16 +5809,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5930,7 +5928,6 @@ EqLoop: func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -6093,16 +6090,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6341,16 +6338,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6573,16 +6570,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6837,16 +6834,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7120,16 +7117,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7404,16 +7401,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7715,16 +7712,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8006,16 +8003,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8252,16 +8249,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8502,16 +8499,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8752,16 +8749,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8871,7 +8868,6 @@ EqLoop: func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -9034,16 +9030,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9282,16 +9278,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9514,16 +9510,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9778,16 +9774,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10061,16 +10057,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10345,16 +10341,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10656,16 +10652,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -10947,16 +10943,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -11193,16 +11189,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -11443,16 +11439,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -11693,16 +11689,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -11838,7 +11834,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -11883,8 +11878,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -11948,8 +11943,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12013,8 +12008,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12077,8 +12072,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12138,8 +12133,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12200,8 +12195,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12265,8 +12260,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12330,8 +12325,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12395,8 +12390,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12460,8 +12455,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12525,8 +12520,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12599,8 +12594,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12663,8 +12658,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12727,8 +12722,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12790,8 +12785,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12850,8 +12845,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12911,8 +12906,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -12975,8 +12970,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -13039,8 +13034,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -13103,8 +13098,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -13167,8 +13162,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -13231,8 +13226,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -13306,7 +13301,6 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -13342,8 +13336,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13411,8 +13405,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13480,8 +13474,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13548,8 +13542,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13613,8 +13607,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13679,8 +13673,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13748,8 +13742,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13817,8 +13811,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13886,8 +13880,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -13955,8 +13949,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14024,8 +14018,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14103,8 +14097,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14172,8 +14166,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14241,8 +14235,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14309,8 +14303,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14374,8 +14368,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14440,8 +14434,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14509,8 +14503,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14578,8 +14572,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14647,8 +14641,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14716,8 +14710,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14785,8 +14779,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -14872,20 +14866,6 @@ func (o *mergeJoinFullOuterOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinFullOuterOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -14938,62 +14918,128 @@ func (o *mergeJoinFullOuterOp) exhaustRightSource() { o.proberState.rIdx = o.proberState.rLength } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinFullOuterOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinFullOuterOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinFullOuterOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinFullOuterOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinFullOuterOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinFullOuterOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -15001,30 +15047,29 @@ func (o *mergeJoinFullOuterOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the right one, @@ -15033,10 +15078,6 @@ func (o *mergeJoinFullOuterOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.lIdx < o.proberState.lLength { o.exhaustLeftSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the left into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } // At least one of the sources is finished. If it was the left one, // then we need to emit remaining tuples from the right source with @@ -15044,49 +15085,51 @@ func (o *mergeJoinFullOuterOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.rIdx < o.proberState.rLength { o.exhaustRightSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the right into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go index c9d8cf562b80..0ff915b3120c 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinInnerOp{} func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -189,16 +188,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -334,16 +333,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -479,16 +478,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -656,16 +655,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -830,16 +829,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1005,16 +1004,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1207,16 +1206,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1373,16 +1372,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1518,16 +1517,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1681,16 +1680,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1832,16 +1831,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1882,7 +1881,6 @@ EqLoop: func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2023,16 +2021,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2168,16 +2166,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2313,16 +2311,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2490,16 +2488,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2664,16 +2662,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2839,16 +2837,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3041,16 +3039,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3207,16 +3205,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3352,16 +3350,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3515,16 +3513,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3666,16 +3664,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3716,7 +3714,6 @@ EqLoop: func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -3857,16 +3854,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4002,16 +3999,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4147,16 +4144,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4324,16 +4321,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4498,16 +4495,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4673,16 +4670,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4875,16 +4872,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5041,16 +5038,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5186,16 +5183,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5349,16 +5346,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5500,16 +5497,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5550,7 +5547,6 @@ EqLoop: func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -5691,16 +5687,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5836,16 +5832,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5981,16 +5977,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6158,16 +6154,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6332,16 +6328,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6507,16 +6503,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6709,16 +6705,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6875,16 +6871,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7020,16 +7016,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7183,16 +7179,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7334,16 +7330,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7410,7 +7406,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -7455,8 +7450,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7517,8 +7512,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7579,8 +7574,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7640,8 +7635,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7698,8 +7693,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7757,8 +7752,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7819,8 +7814,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7881,8 +7876,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7943,8 +7938,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8005,8 +8000,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8067,8 +8062,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8138,8 +8133,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8199,8 +8194,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8260,8 +8255,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8320,8 +8315,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8377,8 +8372,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8435,8 +8430,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8496,8 +8491,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8557,8 +8552,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8618,8 +8613,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8679,8 +8674,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8740,8 +8735,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8812,7 +8807,6 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -8848,8 +8842,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8915,8 +8909,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8982,8 +8976,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9048,8 +9042,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9111,8 +9105,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9175,8 +9169,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9242,8 +9236,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9309,8 +9303,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9376,8 +9370,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9443,8 +9437,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9510,8 +9504,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9587,8 +9581,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9654,8 +9648,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9721,8 +9715,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9787,8 +9781,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9850,8 +9844,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9914,8 +9908,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9981,8 +9975,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10048,8 +10042,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10115,8 +10109,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10182,8 +10176,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10249,8 +10243,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10334,20 +10328,6 @@ func (o *mergeJoinInnerOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinInnerOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -10362,62 +10342,128 @@ func (o *mergeJoinInnerOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinInnerOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinInnerOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinInnerOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinInnerOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinInnerOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinInnerOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -10425,65 +10471,75 @@ func (o *mergeJoinInnerOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + o.builderState.lGroups = o.builderState.lGroups[:0] + o.builderState.rGroups = o.builderState.rGroups[:0] + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go index 11c0e2bfde2d..e785de22a4c0 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinIntersectAllOp{} func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -204,16 +203,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -368,16 +367,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -532,16 +531,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -728,16 +727,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -921,16 +920,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1115,16 +1114,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1336,16 +1335,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1521,16 +1520,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1685,16 +1684,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1867,16 +1866,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2037,16 +2036,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2091,7 +2090,6 @@ EqLoop: func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2247,16 +2245,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2411,16 +2409,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2575,16 +2573,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2771,16 +2769,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2964,16 +2962,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3158,16 +3156,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3379,16 +3377,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3564,16 +3562,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3728,16 +3726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3910,16 +3908,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4080,16 +4078,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4134,7 +4132,6 @@ EqLoop: func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -4290,16 +4287,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4454,16 +4451,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4618,16 +4615,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4814,16 +4811,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5007,16 +5004,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5201,16 +5198,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5422,16 +5419,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5607,16 +5604,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5771,16 +5768,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5953,16 +5950,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6123,16 +6120,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6177,7 +6174,6 @@ EqLoop: func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -6333,16 +6329,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6497,16 +6493,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6661,16 +6657,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6857,16 +6853,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7050,16 +7046,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7244,16 +7240,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7465,16 +7461,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7650,16 +7646,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7814,16 +7810,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7996,16 +7992,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8166,16 +8162,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8246,7 +8242,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -8291,8 +8286,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8353,8 +8348,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8415,8 +8410,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8476,8 +8471,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8534,8 +8529,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8593,8 +8588,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8655,8 +8650,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8717,8 +8712,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8779,8 +8774,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8841,8 +8836,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8903,8 +8898,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8974,8 +8969,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9035,8 +9030,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9096,8 +9091,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9156,8 +9151,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9213,8 +9208,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9271,8 +9266,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9332,8 +9327,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9393,8 +9388,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9454,8 +9449,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9515,8 +9510,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9576,8 +9571,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9648,7 +9643,6 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -9684,8 +9678,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9751,8 +9745,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9818,8 +9812,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9884,8 +9878,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9947,8 +9941,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10011,8 +10005,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10078,8 +10072,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10145,8 +10139,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10212,8 +10206,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10279,8 +10273,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10346,8 +10340,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10423,8 +10417,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10490,8 +10484,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10557,8 +10551,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10623,8 +10617,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10686,8 +10680,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10750,8 +10744,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10817,8 +10811,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10884,8 +10878,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10951,8 +10945,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11018,8 +11012,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11085,8 +11079,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11170,20 +11164,6 @@ func (o *mergeJoinIntersectAllOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinIntersectAllOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -11198,60 +11178,134 @@ func (o *mergeJoinIntersectAllOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinIntersectAllOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinIntersectAllOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinIntersectAllOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinIntersectAllOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinIntersectAllOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinIntersectAllOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if bg.helper.builderState.numEmittedTotal == bg.helper.numRightTuples { + // For INTERSECT ALL joins we build min(# left tuples, # right + // tuples), and we have already reached the number of tuples + // from the right. Thus, we have to skip all tuples from the + // left that are part of the buffered group since they don't + // have a match. + skipLeftBufferedGroup = true + } + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -11259,65 +11313,75 @@ func (o *mergeJoinIntersectAllOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + o.builderState.lGroups = o.builderState.lGroups[:0] + o.builderState.rGroups = o.builderState.rGroups[:0] + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go index b674d672e283..ddec5991258b 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinLeftAntiOp{} func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -200,16 +199,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -396,16 +395,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -584,16 +583,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -804,16 +803,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1032,16 +1031,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1261,16 +1260,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1517,16 +1516,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1745,16 +1744,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1940,16 +1939,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2146,16 +2145,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2346,16 +2345,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2430,7 +2429,6 @@ EqLoop: func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2582,16 +2580,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2778,16 +2776,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2966,16 +2964,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3186,16 +3184,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3414,16 +3412,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3643,16 +3641,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3899,16 +3897,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4127,16 +4125,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4322,16 +4320,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4528,16 +4526,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4728,16 +4726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4812,7 +4810,6 @@ EqLoop: func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -4964,16 +4961,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5160,16 +5157,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5348,16 +5345,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5568,16 +5565,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5796,16 +5793,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6025,16 +6022,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6281,16 +6278,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6509,16 +6506,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6704,16 +6701,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6910,16 +6907,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7110,16 +7107,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7194,7 +7191,6 @@ EqLoop: func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -7346,16 +7342,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7542,16 +7538,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7730,16 +7726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7950,16 +7946,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8178,16 +8174,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8407,16 +8403,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8663,16 +8659,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8891,16 +8887,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9086,16 +9082,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9292,16 +9288,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9492,16 +9488,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9602,7 +9598,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -9650,8 +9645,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9715,8 +9710,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9780,8 +9775,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9844,8 +9839,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9905,8 +9900,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9967,8 +9962,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10032,8 +10027,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10097,8 +10092,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10162,8 +10157,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10227,8 +10222,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10292,8 +10287,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10366,8 +10361,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10430,8 +10425,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10494,8 +10489,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10557,8 +10552,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10617,8 +10612,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10678,8 +10673,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10742,8 +10737,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10806,8 +10801,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10870,8 +10865,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10934,8 +10929,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10998,8 +10993,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11070,7 +11065,6 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -11106,8 +11100,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11173,8 +11167,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11240,8 +11234,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11306,8 +11300,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11369,8 +11363,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11433,8 +11427,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11500,8 +11494,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11567,8 +11561,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11634,8 +11628,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11701,8 +11695,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11768,8 +11762,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11845,8 +11839,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11912,8 +11906,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11979,8 +11973,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12045,8 +12039,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12108,8 +12102,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12172,8 +12166,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12239,8 +12233,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12306,8 +12300,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12373,8 +12367,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12440,8 +12434,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12507,8 +12501,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12592,20 +12586,6 @@ func (o *mergeJoinLeftAntiOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinLeftAntiOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -12632,14 +12612,13 @@ func (o *mergeJoinLeftAntiOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinLeftAntiOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinLeftAntiOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { if !groups[i].unmatched { // "Matched" groups are not outputted in LEFT ANTI, RIGHT ANTI, @@ -12647,51 +12626,118 @@ func (o *mergeJoinLeftAntiOp) calculateOutputCount(groups []group) int { // they do not contribute to the output count. continue } - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinLeftAntiOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinLeftAntiOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftAntiOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftAntiOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -12699,30 +12745,29 @@ func (o *mergeJoinLeftAntiOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the right one, @@ -12731,49 +12776,51 @@ func (o *mergeJoinLeftAntiOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.lIdx < o.proberState.lLength { o.exhaustLeftSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the left into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go index fd05374386f7..823ca30d0842 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinLeftOuterOp{} func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -200,16 +199,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -397,16 +396,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -586,16 +585,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -807,16 +806,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1036,16 +1035,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1266,16 +1265,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1523,16 +1522,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1752,16 +1751,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1948,16 +1947,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2155,16 +2154,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2356,16 +2355,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2441,7 +2440,6 @@ EqLoop: func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2593,16 +2591,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2790,16 +2788,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2979,16 +2977,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3200,16 +3198,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3429,16 +3427,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3659,16 +3657,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3916,16 +3914,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4145,16 +4143,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4341,16 +4339,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4548,16 +4546,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4749,16 +4747,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4834,7 +4832,6 @@ EqLoop: func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -4986,16 +4983,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5183,16 +5180,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5372,16 +5369,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5593,16 +5590,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5822,16 +5819,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6052,16 +6049,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6309,16 +6306,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6538,16 +6535,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6734,16 +6731,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6941,16 +6938,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7142,16 +7139,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7227,7 +7224,6 @@ EqLoop: func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -7379,16 +7375,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7576,16 +7572,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7765,16 +7761,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7986,16 +7982,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8215,16 +8211,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8445,16 +8441,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8702,16 +8698,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8931,16 +8927,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9127,16 +9123,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9334,16 +9330,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9535,16 +9531,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9646,7 +9642,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -9691,8 +9686,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9753,8 +9748,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9815,8 +9810,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9876,8 +9871,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9934,8 +9929,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9993,8 +9988,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10055,8 +10050,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10117,8 +10112,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10179,8 +10174,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10241,8 +10236,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10303,8 +10298,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10374,8 +10369,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10435,8 +10430,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10496,8 +10491,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10556,8 +10551,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10613,8 +10608,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10671,8 +10666,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10732,8 +10727,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10793,8 +10788,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10854,8 +10849,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10915,8 +10910,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10976,8 +10971,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11048,7 +11043,6 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -11084,8 +11078,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11153,8 +11147,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11222,8 +11216,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11290,8 +11284,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11355,8 +11349,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11421,8 +11415,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11490,8 +11484,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11559,8 +11553,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11628,8 +11622,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11697,8 +11691,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11766,8 +11760,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11845,8 +11839,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11914,8 +11908,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -11983,8 +11977,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12051,8 +12045,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12116,8 +12110,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12182,8 +12176,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12251,8 +12245,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12320,8 +12314,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12389,8 +12383,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12458,8 +12452,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12527,8 +12521,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if rightGroup.nullGroup { @@ -12614,20 +12608,6 @@ func (o *mergeJoinLeftOuterOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinLeftOuterOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -12662,62 +12642,128 @@ func (o *mergeJoinLeftOuterOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinLeftOuterOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinLeftOuterOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinLeftOuterOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinLeftOuterOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftOuterOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftOuterOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -12725,30 +12771,29 @@ func (o *mergeJoinLeftOuterOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the right one, @@ -12757,49 +12802,51 @@ func (o *mergeJoinLeftOuterOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.lIdx < o.proberState.lLength { o.exhaustLeftSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the left into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go index 1460f19892e7..7a1f6af3fa0e 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinLeftSemiOp{} func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -189,16 +188,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -333,16 +332,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -477,16 +476,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -653,16 +652,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -826,16 +825,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1000,16 +999,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1201,16 +1200,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1366,16 +1365,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1510,16 +1509,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1672,16 +1671,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1822,16 +1821,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1871,7 +1870,6 @@ EqLoop: func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2012,16 +2010,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2156,16 +2154,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2300,16 +2298,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2476,16 +2474,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2649,16 +2647,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2823,16 +2821,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3024,16 +3022,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3189,16 +3187,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3333,16 +3331,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3495,16 +3493,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3645,16 +3643,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3694,7 +3692,6 @@ EqLoop: func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -3835,16 +3832,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3979,16 +3976,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4123,16 +4120,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4299,16 +4296,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4472,16 +4469,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4646,16 +4643,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4847,16 +4844,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5012,16 +5009,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5156,16 +5153,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5318,16 +5315,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5468,16 +5465,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5517,7 +5514,6 @@ EqLoop: func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -5658,16 +5654,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5802,16 +5798,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5946,16 +5942,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6122,16 +6118,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6295,16 +6291,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6469,16 +6465,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6670,16 +6666,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6835,16 +6831,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6979,16 +6975,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7141,16 +7137,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7291,16 +7287,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7366,7 +7362,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -7411,8 +7406,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7473,8 +7468,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7535,8 +7530,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7596,8 +7591,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7654,8 +7649,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7713,8 +7708,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7775,8 +7770,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7837,8 +7832,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7899,8 +7894,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7961,8 +7956,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8023,8 +8018,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8094,8 +8089,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8155,8 +8150,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8216,8 +8211,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8276,8 +8271,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8333,8 +8328,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8391,8 +8386,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8452,8 +8447,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8513,8 +8508,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8574,8 +8569,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8635,8 +8630,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8696,8 +8691,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8768,7 +8763,6 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -8804,8 +8798,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8871,8 +8865,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8938,8 +8932,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9004,8 +8998,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9067,8 +9061,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9131,8 +9125,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9198,8 +9192,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9265,8 +9259,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9332,8 +9326,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9399,8 +9393,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9466,8 +9460,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9543,8 +9537,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9610,8 +9604,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9677,8 +9671,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9743,8 +9737,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9806,8 +9800,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9870,8 +9864,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9937,8 +9931,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10004,8 +9998,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10071,8 +10065,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10138,8 +10132,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10205,8 +10199,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10290,20 +10284,6 @@ func (o *mergeJoinLeftSemiOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinLeftSemiOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -10318,60 +10298,126 @@ func (o *mergeJoinLeftSemiOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinLeftSemiOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinLeftSemiOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinLeftSemiOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinLeftSemiOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftSemiOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinLeftSemiOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -10379,65 +10425,75 @@ func (o *mergeJoinLeftSemiOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + o.builderState.lGroups = o.builderState.lGroups[:0] + o.builderState.rGroups = o.builderState.rGroups[:0] + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go index b4509265bfd2..224822d898b9 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinRightAntiOp{} func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -200,16 +199,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -395,16 +394,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -582,16 +581,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -801,16 +800,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1028,16 +1027,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1256,16 +1255,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1511,16 +1510,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1738,16 +1737,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1932,16 +1931,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2137,16 +2136,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2336,16 +2335,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2419,7 +2418,6 @@ EqLoop: func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2571,16 +2569,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2766,16 +2764,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2953,16 +2951,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3172,16 +3170,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3399,16 +3397,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3627,16 +3625,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3882,16 +3880,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4109,16 +4107,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4303,16 +4301,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4508,16 +4506,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4707,16 +4705,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4790,7 +4788,6 @@ EqLoop: func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -4942,16 +4939,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5137,16 +5134,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5324,16 +5321,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5543,16 +5540,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5770,16 +5767,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5998,16 +5995,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6253,16 +6250,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6480,16 +6477,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6674,16 +6671,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6879,16 +6876,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7078,16 +7075,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7161,7 +7158,6 @@ EqLoop: func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -7313,16 +7309,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7508,16 +7504,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7695,16 +7691,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7914,16 +7910,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8141,16 +8137,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8369,16 +8365,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8624,16 +8620,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8851,16 +8847,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9045,16 +9041,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9250,16 +9246,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9449,16 +9445,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9558,7 +9554,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -9603,8 +9598,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9668,8 +9663,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9733,8 +9728,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9797,8 +9792,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9858,8 +9853,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9920,8 +9915,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9985,8 +9980,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10050,8 +10045,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10115,8 +10110,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10180,8 +10175,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10245,8 +10240,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10319,8 +10314,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10383,8 +10378,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10447,8 +10442,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10510,8 +10505,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10570,8 +10565,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10631,8 +10626,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10695,8 +10690,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10759,8 +10754,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10823,8 +10818,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10887,8 +10882,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10951,8 +10946,8 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -11026,7 +11021,6 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -11062,8 +11056,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11129,8 +11123,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11196,8 +11190,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11262,8 +11256,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11325,8 +11319,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11389,8 +11383,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11456,8 +11450,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11523,8 +11517,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11590,8 +11584,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11657,8 +11651,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11724,8 +11718,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11801,8 +11795,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11868,8 +11862,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11935,8 +11929,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12001,8 +11995,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12064,8 +12058,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12128,8 +12122,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12195,8 +12189,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12262,8 +12256,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12329,8 +12323,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12396,8 +12390,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12463,8 +12457,8 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12548,20 +12542,6 @@ func (o *mergeJoinRightAntiOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinRightAntiOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -12586,14 +12566,13 @@ func (o *mergeJoinRightAntiOp) exhaustRightSource() { o.proberState.rIdx = o.proberState.rLength } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinRightAntiOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinRightAntiOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { if !groups[i].unmatched { // "Matched" groups are not outputted in LEFT ANTI, RIGHT ANTI, @@ -12601,49 +12580,116 @@ func (o *mergeJoinRightAntiOp) calculateOutputCount(groups []group) int { // they do not contribute to the output count. continue } - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinRightAntiOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinRightAntiOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.rGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.rGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightAntiOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightAntiOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -12651,30 +12697,29 @@ func (o *mergeJoinRightAntiOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the left one, @@ -12683,49 +12728,51 @@ func (o *mergeJoinRightAntiOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.rIdx < o.proberState.rLength { o.exhaustRightSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the right into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go index 4bc582ea5d27..b5661129f242 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinRightOuterOp{} func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -200,16 +199,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -396,16 +395,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -584,16 +583,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -804,16 +803,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1032,16 +1031,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1261,16 +1260,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1517,16 +1516,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1745,16 +1744,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1940,16 +1939,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2146,16 +2145,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2346,16 +2345,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2430,7 +2429,6 @@ EqLoop: func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2582,16 +2580,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2778,16 +2776,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2966,16 +2964,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3186,16 +3184,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3414,16 +3412,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3643,16 +3641,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3899,16 +3897,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4127,16 +4125,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4322,16 +4320,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4528,16 +4526,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4728,16 +4726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4812,7 +4810,6 @@ EqLoop: func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -4964,16 +4961,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5160,16 +5157,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5348,16 +5345,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5568,16 +5565,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5796,16 +5793,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6025,16 +6022,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6281,16 +6278,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6509,16 +6506,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6704,16 +6701,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6910,16 +6907,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7110,16 +7107,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7194,7 +7191,6 @@ EqLoop: func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -7346,16 +7342,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7542,16 +7538,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7730,16 +7726,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7950,16 +7946,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8178,16 +8174,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8407,16 +8403,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8663,16 +8659,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -8891,16 +8887,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9086,16 +9082,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9292,16 +9288,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9492,16 +9488,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -9602,7 +9598,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -9647,8 +9642,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9712,8 +9707,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9777,8 +9772,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9841,8 +9836,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9902,8 +9897,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -9964,8 +9959,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10029,8 +10024,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10094,8 +10089,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10159,8 +10154,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10224,8 +10219,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10289,8 +10284,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10363,8 +10358,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10427,8 +10422,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10491,8 +10486,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10554,8 +10549,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10614,8 +10609,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10675,8 +10670,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10739,8 +10734,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10803,8 +10798,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10867,8 +10862,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10931,8 +10926,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -10995,8 +10990,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } if leftGroup.nullGroup { @@ -11070,7 +11065,6 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -11106,8 +11100,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11173,8 +11167,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11240,8 +11234,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11306,8 +11300,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11369,8 +11363,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11433,8 +11427,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11500,8 +11494,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11567,8 +11561,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11634,8 +11628,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11701,8 +11695,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11768,8 +11762,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11845,8 +11839,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11912,8 +11906,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -11979,8 +11973,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12045,8 +12039,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12108,8 +12102,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12172,8 +12166,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12239,8 +12233,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12306,8 +12300,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12373,8 +12367,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12440,8 +12434,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12507,8 +12501,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -12592,20 +12586,6 @@ func (o *mergeJoinRightOuterOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinRightOuterOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -12638,62 +12618,128 @@ func (o *mergeJoinRightOuterOp) exhaustRightSource() { o.proberState.rIdx = o.proberState.rLength } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinRightOuterOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinRightOuterOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinRightOuterOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinRightOuterOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightOuterOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightOuterOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -12701,30 +12747,29 @@ func (o *mergeJoinRightOuterOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] // At least one of the sources is finished. If it was the left one, @@ -12733,49 +12778,51 @@ func (o *mergeJoinRightOuterOp) Next() coldata.Batch { // finished, then there is nothing left to do. if o.proberState.rIdx < o.proberState.rLength { o.exhaustRightSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the right into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go index a864e99a40b7..6eca76b2c3cc 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go @@ -48,7 +48,6 @@ var _ colexecop.Operator = &mergeJoinRightSemiOp{} func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -189,16 +188,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -332,16 +331,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -475,16 +474,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -650,16 +649,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -822,16 +821,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -995,16 +994,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1195,16 +1194,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1359,16 +1358,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1502,16 +1501,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1663,16 +1662,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1812,16 +1811,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -1860,7 +1859,6 @@ EqLoop: func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -2001,16 +1999,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2144,16 +2142,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2287,16 +2285,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2462,16 +2460,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2634,16 +2632,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -2807,16 +2805,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3007,16 +3005,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3171,16 +3169,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3314,16 +3312,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3475,16 +3473,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3624,16 +3622,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3672,7 +3670,6 @@ EqLoop: func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -3813,16 +3810,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -3956,16 +3953,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4099,16 +4096,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4274,16 +4271,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4446,16 +4443,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4619,16 +4616,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4819,16 +4816,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -4983,16 +4980,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5126,16 +5123,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5287,16 +5284,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5436,16 +5433,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5484,7 +5481,6 @@ EqLoop: func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -5625,16 +5621,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5768,16 +5764,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -5911,16 +5907,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6086,16 +6082,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6258,16 +6254,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6431,16 +6427,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6631,16 +6627,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6795,16 +6791,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -6938,16 +6934,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7099,16 +7095,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7248,16 +7244,16 @@ EqLoop: curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -7322,7 +7318,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -7367,8 +7362,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7429,8 +7424,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7491,8 +7486,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7552,8 +7547,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7610,8 +7605,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7669,8 +7664,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7731,8 +7726,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7793,8 +7788,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7855,8 +7850,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7917,8 +7912,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -7979,8 +7974,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8050,8 +8045,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8111,8 +8106,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8172,8 +8167,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8232,8 +8227,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8289,8 +8284,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8347,8 +8342,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8408,8 +8403,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8469,8 +8464,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8530,8 +8525,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8591,8 +8586,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8652,8 +8647,8 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8724,7 +8719,6 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -8760,8 +8754,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8827,8 +8821,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8894,8 +8888,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -8960,8 +8954,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9023,8 +9017,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9087,8 +9081,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9154,8 +9148,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9221,8 +9215,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9288,8 +9282,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9355,8 +9349,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9422,8 +9416,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9499,8 +9493,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9566,8 +9560,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9633,8 +9627,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9699,8 +9693,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9762,8 +9756,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9826,8 +9820,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9893,8 +9887,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -9960,8 +9954,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10027,8 +10021,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10094,8 +10088,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10161,8 +10155,8 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } { @@ -10246,20 +10240,6 @@ func (o *mergeJoinRightSemiOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoinRightSemiOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -10274,58 +10254,129 @@ func (o *mergeJoinRightSemiOp) exhaustRightSource() { // ignored in all joins except for RIGHT OUTER and FULL OUTER. } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoinRightSemiOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoinRightSemiOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoinRightSemiOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoinRightSemiOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - o.builderState.outCount = o.calculateOutputCount(o.builderState.rGroups) - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + numBuilt := o.numBuiltFromBatch(o.builderState.rGroups) + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightSemiOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + startIdx := o.bufferedGroup.leftGroupStartIdx + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoinRightSemiOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + // For RIGHT SEMI joins we have already fully built the output based + // on all tuples in the right buffered group using the match from + // the current left batch. This allows us to simply skip all tuples + // that are part of the left buffered group. + skipLeftBufferedGroup = true + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true + } + if willEmit > 0 && len(o.outputTypes) != 0 { + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) } o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - } } @@ -10333,65 +10384,75 @@ func (o *mergeJoinRightSemiOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + o.builderState.lGroups = o.builderState.lGroups[:0] + o.builderState.rGroups = o.builderState.rGroups[:0] + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go index d509a9b0ac40..620994a13745 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go @@ -267,16 +267,16 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / curRIdx++ } - // Last equality column and either group is incomplete. Save state - // and have it handled in the next iteration. + // Last equality column and either group is incomplete. if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { - o.appendToBufferedGroup(&o.left, o.proberState.lBatch, lSel, beginLIdx, lGroupLength) + // Store the state about the buffered group. + o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) + o.bufferedGroup.leftGroupStartIdx = beginLIdx o.proberState.lIdx = lGroupLength + beginLIdx - o.appendToBufferedGroup(&o.right, o.proberState.rBatch, rSel, beginRIdx, rGroupLength) + o.appendToRightBufferedGroup(rSel, beginRIdx, rGroupLength) o.proberState.rIdx = rGroupLength + beginRIdx - o.groups.finishedCol() - break EqLoop + return } if eqColIdx < len(o.left.eqCols)-1 { @@ -638,7 +638,6 @@ func _PROCESS_NOT_LAST_GROUP_IN_COLUMN_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} func (o *mergeJoin_JOIN_TYPE_STRINGOp) probeBodyLSel_IS_L_SELRSel_IS_R_SEL() { lSel := o.proberState.lBatch.Selection() rSel := o.proberState.rBatch.Selection() -EqLoop: for eqColIdx := 0; eqColIdx < len(o.left.eqCols); eqColIdx++ { leftColIdx := o.left.eqCols[eqColIdx] rightColIdx := o.right.eqCols[eqColIdx] @@ -706,8 +705,8 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } // {{if or _JOIN_TYPE.IsRightOuter _JOIN_TYPE.IsRightAnti}} @@ -791,7 +790,6 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { @@ -844,8 +842,8 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputCapacity { - toAppend = outputCapacity - outStartIdx + if outStartIdx+toAppend > o.outputCapacity { + toAppend = o.outputCapacity - outStartIdx } // {{if _JOIN_TYPE.IsLeftOuter}} @@ -944,7 +942,6 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightGroupsFromBatch( o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { - outputCapacity := o.output.Capacity() // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { @@ -992,20 +989,6 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) probe() { } } -// setBuilderSourceToBufferedGroup sets up the builder state to use the -// buffered group. -func (o *mergeJoin_JOIN_TYPE_STRINGOp) setBuilderSourceToBufferedGroup() { - o.builderState.buildFrom = mjBuildFromBufferedGroup - o.bufferedGroup.helper.setupBuilder() - o.builderState.totalOutCountFromBufferedGroup = o.bufferedGroup.helper.calculateOutputCount() - o.builderState.alreadyEmittedFromBufferedGroup = 0 - - // We cannot yet reset the buffered groups because the builder will be taking - // input from them. The actual reset will take place on the next call to - // initProberState(). - o.bufferedGroup.needToReset = true -} - // exhaustLeftSource sets up the builder to process any remaining tuples from // the left source. It should only be called when the right source has been // exhausted. @@ -1082,14 +1065,13 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) exhaustRightSource() { // {{end}} } -// calculateOutputCount uses the toBuild field of each group and the output -// batch size to determine the output count. Note that as soon as a group is +// numBuiltFromBatch uses the toBuild field of each group and the output +// capacity to determine the output count. Note that as soon as a group is // materialized partially or fully to output, its toBuild field is updated -// accordingly. -func (o *mergeJoin_JOIN_TYPE_STRINGOp) calculateOutputCount(groups []group) int { - count := o.builderState.outCount - outputCapacity := o.output.Capacity() - +// accordingly. The number of tuples that will be built from batch during the +// current iteration is returned. +func (o *mergeJoin_JOIN_TYPE_STRINGOp) numBuiltFromBatch(groups []group) (numBuilt int) { + outCount := o.builderState.outCount for i := 0; i < len(groups); i++ { // {{if or _JOIN_TYPE.IsLeftAnti _JOIN_TYPE.IsRightAnti}} if !groups[i].unmatched { @@ -1099,65 +1081,190 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) calculateOutputCount(groups []group) int continue } // {{end}} - count += groups[i].toBuild + outCount += groups[i].toBuild groups[i].toBuild = 0 - if count > o.output.Capacity() { - groups[i].toBuild = count - outputCapacity - count = outputCapacity - return count + if outCount > o.outputCapacity { + groups[i].toBuild = outCount - o.outputCapacity + return o.outputCapacity - o.builderState.outCount } } - o.builderState.outFinished = true - return count + return outCount - o.builderState.outCount } -// build creates the cross product, and writes it to the output member. -func (o *mergeJoin_JOIN_TYPE_STRINGOp) build() { +// buildFromBatch builds as many output rows as possible from the groups that +// were complete in the probing batches. New rows are put starting at +// o.builderState.outCount position until either the capacity is reached or all +// groups are processed. +func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildFromBatch() { outStartIdx := o.builderState.outCount - switch o.builderState.buildFrom { - case mjBuildFromBatch: - // {{if or _JOIN_TYPE.IsRightSemi _JOIN_TYPE.IsRightAnti}} - o.builderState.outCount = o.calculateOutputCount(o.builderState.rGroups) - // {{else}} - o.builderState.outCount = o.calculateOutputCount(o.builderState.lGroups) + // {{if or _JOIN_TYPE.IsRightSemi _JOIN_TYPE.IsRightAnti}} + numBuilt := o.numBuiltFromBatch(o.builderState.rGroups) + // {{else}} + numBuilt := o.numBuiltFromBatch(o.builderState.lGroups) + // {{end}} + o.builderState.outCount += numBuilt + if numBuilt > 0 && len(o.outputTypes) != 0 { + // We will be actually building the output if we have columns in the output + // batch (meaning that we're not doing query like 'SELECT count(*) ...') + // and when builderState.outCount has increased (meaning that we have + // something to build). + colOffsetForRightGroups := 0 + // {{if not (or _JOIN_TYPE.IsRightSemi _JOIN_TYPE.IsRightAnti)}} + o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) + colOffsetForRightGroups = len(o.left.sourceTypes) + _ = colOffsetForRightGroups // {{end}} - if o.output.Width() != 0 && o.builderState.outCount > outStartIdx { - // We will be actually building the output if we have columns in the output - // batch (meaning that we're not doing query like 'SELECT count(*) ...') - // and when builderState.outCount has increased (meaning that we have - // something to build). - colOffsetForRightGroups := 0 - // {{if not (or _JOIN_TYPE.IsRightSemi _JOIN_TYPE.IsRightAnti)}} - o.buildLeftGroupsFromBatch(o.builderState.lGroups, &o.left, o.proberState.lBatch, outStartIdx) - colOffsetForRightGroups = len(o.left.sourceTypes) - _ = colOffsetForRightGroups - // {{end}} - // {{if not (or _JOIN_TYPE.IsLeftSemi _JOIN_TYPE.IsLeftAnti)}} - o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + // {{if not (or _JOIN_TYPE.IsLeftSemi _JOIN_TYPE.IsLeftAnti)}} + o.buildRightGroupsFromBatch(o.builderState.rGroups, colOffsetForRightGroups, &o.right, o.proberState.rBatch, outStartIdx) + // {{end}} + } +} + +// transitionIntoBuildingFromBufferedGroup should be called once we have +// non-empty right buffered group in order to setup the buffered group builder. +// +// It will complete the right buffered group (meaning it'll read all batches +// from the right input until either the new group is found or the input is +// exhausted). +// +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoin_JOIN_TYPE_STRINGOp) transitionIntoBuildingFromBufferedGroup() { + if o.proberState.rIdx == o.proberState.rLength { + // The right buffered group might extend into the next batch, so we have + // to complete it first. + o.completeRightBufferedGroup() + } + + o.bufferedGroup.helper.setupLeftBuilder() + + // {{if and _JOIN_TYPE.IsLeftAnti _JOIN_TYPE.IsSetOp}} + // For EXCEPT ALL joins we build # left tuples - # right tuples output rows + // (if positive), so we have to discard first numRightTuples rows from the + // left. + numSkippedLeft := 0 + for { + groupLength := o.proberState.lIdx - o.bufferedGroup.leftGroupStartIdx + if numSkippedLeft+groupLength > o.bufferedGroup.helper.numRightTuples { + // The current left batch is the first one that contains tuples + // without a "match". + break + } + numSkippedLeft += groupLength + var groupFinished bool + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + groupFinished = true + } else { + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + groupFinished = o.proberState.lIdx == 0 + } + if groupFinished { + // We have less matching tuples on the left than on the right, so we + // don't emit any output for this buffered group. + o.bufferedGroup.helper.Reset(o.Ctx) + o.state = mjEntry + return + } + } + // We might need to skip some tuples in the current left batch since they + // still had matches with the right side. + toSkipInThisBatch := o.bufferedGroup.helper.numRightTuples - numSkippedLeft + startIdx := o.bufferedGroup.leftGroupStartIdx + toSkipInThisBatch + // {{else}} + startIdx := o.bufferedGroup.leftGroupStartIdx + // {{end}} + + o.bufferedGroup.helper.prepareForNextLeftBatch(o.proberState.lBatch, startIdx, o.proberState.lIdx) + o.state = mjBuildFromBufferedGroup +} + +// buildFromBufferedGroup builds the output based on the current buffered group +// and puts new tuples starting at position b.builderState.outCount. It returns +// true once the output for the buffered group has been fully populated. +// +// It is assumed that transitionIntoBuildingFromBufferedGroup has been called. +// +// For a more detailed explanation and an example please refer to the comment at +// the top of mergejoiner.go. +func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildFromBufferedGroup() (bufferedGroupComplete bool) { + bg := &o.bufferedGroup + // Iterate until either we use up the whole capacity of the output batch or + // we complete the buffered group. + for { + if bg.helper.builderState.left.curSrcStartIdx == o.proberState.lLength { + // The output has been fully built from the current left batch. + bg.leftBatchDone = true + } + if bg.leftBatchDone { + // The current left batch has been fully processed with regards to + // the buffered group. + bg.leftBatchDone = false + if o.proberState.lIdx < o.proberState.lLength { + // The group on the left is finished within the current left + // batch. + return true + } + var skipLeftBufferedGroup bool + // {{if _JOIN_TYPE.IsRightSemi}} + // For RIGHT SEMI joins we have already fully built the output based + // on all tuples in the right buffered group using the match from + // the current left batch. This allows us to simply skip all tuples + // that are part of the left buffered group. + skipLeftBufferedGroup = true + // {{else if and _JOIN_TYPE.IsLeftSemi _JOIN_TYPE.IsSetOp}} + if bg.helper.builderState.numEmittedTotal == bg.helper.numRightTuples { + // For INTERSECT ALL joins we build min(# left tuples, # right + // tuples), and we have already reached the number of tuples + // from the right. Thus, we have to skip all tuples from the + // left that are part of the buffered group since they don't + // have a match. + skipLeftBufferedGroup = true + } // {{end}} + if skipLeftBufferedGroup { + // Keep fetching the next batch from the left input until we + // either find the start of the new group or we exhaust the + // input. + for o.proberState.lIdx == o.proberState.lLength && o.proberState.lLength > 0 { + o.continueLeftBufferedGroup() + } + return true + } + // Fetch the next batch from the left input and calculate the + // boundaries of the buffered group. + o.continueLeftBufferedGroup() + if o.proberState.lIdx == 0 { + return true + } + bg.helper.prepareForNextLeftBatch( + o.proberState.lBatch, bg.leftGroupStartIdx, o.proberState.lIdx, + ) } - case mjBuildFromBufferedGroup: - willEmit := o.builderState.totalOutCountFromBufferedGroup - o.builderState.alreadyEmittedFromBufferedGroup - if o.builderState.outCount+willEmit > o.output.Capacity() { - willEmit = o.output.Capacity() - o.builderState.outCount + willEmit := bg.helper.canEmit() + if o.builderState.outCount+willEmit > o.outputCapacity { + willEmit = o.outputCapacity - o.builderState.outCount } else { - o.builderState.outFinished = true + bg.leftBatchDone = true } - o.builderState.outCount += willEmit - o.builderState.alreadyEmittedFromBufferedGroup += willEmit - if o.output.Width() != 0 && willEmit > 0 { + if willEmit > 0 && len(o.outputTypes) != 0 { // {{if not (or _JOIN_TYPE.IsRightSemi _JOIN_TYPE.IsRightAnti)}} - o.bufferedGroup.helper.buildFromLeftInput(o.Ctx, outStartIdx) + bg.helper.buildFromLeftInput(o.Ctx, o.builderState.outCount) // {{end}} // {{if not (or _JOIN_TYPE.IsLeftSemi _JOIN_TYPE.IsLeftAnti)}} - o.bufferedGroup.helper.buildFromRightInput(o.Ctx, outStartIdx) + bg.helper.buildFromRightInput(o.Ctx, o.builderState.outCount) // {{end}} } - - default: - colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) - + o.builderState.outCount += willEmit + bg.helper.builderState.numEmittedCurLeftBatch += willEmit + bg.helper.builderState.numEmittedTotal += willEmit + if o.builderState.outCount == o.outputCapacity { + return false + } } } @@ -1167,18 +1274,8 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) build() { // builder. func _SOURCE_FINISHED_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{define "sourceFinishedSwitch" -}} - o.outputReady = true - o.builderState.buildFrom = mjBuildFromBatch - // {{if or $.JoinType.IsInner (or $.JoinType.IsLeftSemi $.JoinType.IsRightSemi)}} - o.setBuilderSourceToBufferedGroup() - // {{else}} - // Next, we need to make sure that builder state is set up for a case when - // neither exhaustLeftSource nor exhaustRightSource is called below. In such - // scenario the merge joiner is done, so it'll be outputting zero-length - // batches from now on. o.builderState.lGroups = o.builderState.lGroups[:0] o.builderState.rGroups = o.builderState.rGroups[:0] - // {{end}} // {{if or $.JoinType.IsLeftOuter $.JoinType.IsLeftAnti}} // At least one of the sources is finished. If it was the right one, // then we need to emit remaining tuples from the left source with @@ -1186,10 +1283,6 @@ func _SOURCE_FINISHED_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // finished, then there is nothing left to do. if o.proberState.lIdx < o.proberState.lLength { o.exhaustLeftSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the left into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } // {{end}} // {{if or $.JoinType.IsRightOuter $.JoinType.IsRightAnti}} @@ -1199,10 +1292,6 @@ func _SOURCE_FINISHED_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // finished, then there is nothing left to do. if o.proberState.rIdx < o.proberState.rLength { o.exhaustRightSource() - // We unset o.outputReady here because we want to put as many unmatched - // tuples from the right into the output batch. Once outCount reaches the - // desired output batch size, the output will be returned. - o.outputReady = false } // {{end}} // {{end}} @@ -1215,63 +1304,74 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next() coldata.Batch { o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate( o.outputTypes, o.output, 1 /* minDesiredCapacity */, o.memoryLimit, ) + o.outputCapacity = o.output.Capacity() o.bufferedGroup.helper.output = o.output + o.builderState.outCount = 0 for { switch o.state { case mjEntry: - o.initProberState() - - if o.nonEmptyBufferedGroup() { - o.state = mjFinishBufferedGroup - break + // If this is the first batch or we're done with the current batch, + // get the next batch. + if o.proberState.lBatch == nil || (o.proberState.lLength != 0 && o.proberState.lIdx == o.proberState.lLength) { + o.proberState.lIdx, o.proberState.lBatch = 0, o.left.source.Next() + o.proberState.lLength = o.proberState.lBatch.Length() + } + if o.proberState.rBatch == nil || (o.proberState.rLength != 0 && o.proberState.rIdx == o.proberState.rLength) { + o.proberState.rIdx, o.proberState.rBatch = 0, o.right.source.Next() + o.proberState.rLength = o.proberState.rBatch.Length() } - if o.sourceFinished() { o.state = mjSourceFinished break } - o.state = mjProbe + case mjSourceFinished: _SOURCE_FINISHED_SWITCH(_JOIN_TYPE) - o.state = mjBuild - case mjFinishBufferedGroup: - o.finishProbe() - o.setBuilderSourceToBufferedGroup() - o.state = mjBuild + if len(o.builderState.lGroups) == 0 && len(o.builderState.rGroups) == 0 { + o.state = mjDone + o.output.SetLength(o.builderState.outCount) + return o.output + } + o.state = mjBuildFromBatch + case mjProbe: o.probe() - o.setBuilderSourceToBatch() - o.state = mjBuild - case mjBuild: - o.build() - - if o.builderState.outFinished { + o.builderState.lGroups, o.builderState.rGroups = o.groups.getGroups() + if len(o.builderState.lGroups) > 0 || len(o.builderState.rGroups) > 0 { + o.state = mjBuildFromBatch + } else if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { o.state = mjEntry - o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.output.Capacity() { - if o.builderState.outCount == 0 { - // We have already fully emitted the result of the join, so we - // transition to "finished" state. - o.state = mjDone - continue - } + case mjBuildFromBatch: + o.buildFromBatch() + if o.builderState.outCount == o.outputCapacity { o.output.SetLength(o.builderState.outCount) - // Reset builder out count. - o.builderState.outCount = 0 - o.outputReady = false return o.output } - case mjDone: - // Note that resetting of buffered group will close disk queues - // (if there are any). - if o.bufferedGroup.needToReset { + if o.bufferedGroup.helper.numRightTuples != 0 { + o.transitionIntoBuildingFromBufferedGroup() + } else { + o.state = mjEntry + } + + case mjBuildFromBufferedGroup: + bufferedGroupComplete := o.buildFromBufferedGroup() + if bufferedGroupComplete { o.bufferedGroup.helper.Reset(o.Ctx) - o.bufferedGroup.needToReset = false + o.state = mjEntry } + if o.builderState.outCount == o.outputCapacity { + o.output.SetLength(o.builderState.outCount) + return o.output + } + + case mjDone: return coldata.ZeroBatch + default: colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } diff --git a/pkg/sql/colexec/crossjoiner_test.go b/pkg/sql/colexec/crossjoiner_test.go index 05eefdd2a6e4..b138175c9a0e 100644 --- a/pkg/sql/colexec/crossjoiner_test.go +++ b/pkg/sql/colexec/crossjoiner_test.go @@ -190,7 +190,7 @@ func getCJTestCases() []*joinTestCase { leftTypes: []*types.T{types.Int}, rightTypes: []*types.T{types.Int}, leftTuples: colexectestutils.Tuples{{0}, {1}, {2}}, - rightTuples: colexectestutils.Tuples{{3}}, + rightTuples: colexectestutils.Tuples{{3}, {4}}, leftOutCols: []uint32{0}, joinType: descpb.LeftSemiJoin, expected: colexectestutils.Tuples{{0}, {1}, {2}}, @@ -254,7 +254,7 @@ func getCJTestCases() []*joinTestCase { expected: colexectestutils.Tuples{{0}, {1}, {2}, {3}}, }, { - description: "intersect all join, right non-empty", + description: "intersect all join, right smaller", leftTypes: []*types.T{types.Int}, rightTypes: []*types.T{types.Int}, leftTuples: colexectestutils.Tuples{{0}, {1}, {2}, {3}, {4}}, @@ -263,6 +263,16 @@ func getCJTestCases() []*joinTestCase { joinType: descpb.IntersectAllJoin, expected: colexectestutils.Tuples{{0}, {1}, {2}}, }, + { + description: "intersect all join, right larger", + leftTypes: []*types.T{types.Int}, + rightTypes: []*types.T{types.Int}, + leftTuples: colexectestutils.Tuples{{0}, {1}, {2}}, + rightTuples: colexectestutils.Tuples{{3}, {nil}, {3}, {3}, {4}}, + leftOutCols: []uint32{0}, + joinType: descpb.IntersectAllJoin, + expected: colexectestutils.Tuples{{0}, {1}, {2}}, + }, { description: "intersect all join, left empty", leftTypes: []*types.T{types.Int}, @@ -288,14 +298,26 @@ func getCJTestCases() []*joinTestCase { expected: colexectestutils.Tuples{}, }, { - description: "except all join, right non-empty", + description: "except all join, right smaller", leftTypes: []*types.T{types.Int}, rightTypes: []*types.T{types.Int}, leftTuples: colexectestutils.Tuples{{0}, {1}, {2}, {3}, {4}}, rightTuples: colexectestutils.Tuples{{3}, {nil}, {3}}, leftOutCols: []uint32{0}, joinType: descpb.ExceptAllJoin, - expected: colexectestutils.Tuples{{0}, {1}}, + expected: colexectestutils.Tuples{{3}, {4}}, + }, + { + description: "except all join, right larger", + leftTypes: []*types.T{types.Int}, + rightTypes: []*types.T{types.Int}, + leftTuples: colexectestutils.Tuples{{0}, {1}, {2}}, + rightTuples: colexectestutils.Tuples{{3}, {nil}, {3}, {3}, {4}}, + leftOutCols: []uint32{0}, + joinType: descpb.ExceptAllJoin, + // Injecting nulls into the right input won't change the output. + skipAllNullsInjection: true, + expected: colexectestutils.Tuples{}, }, { description: "except all join, left empty", diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 77006a79d576..81f7c1c3c940 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -1612,6 +1612,33 @@ func getMJTestCases() []*joinTestCase { rightEqColsAreKey: true, expected: colexectestutils.Tuples{{1}}, }, + { + description: "LEFT SEMI join with non-unique eq column", + joinType: descpb.LeftSemiJoin, + leftTypes: []*types.T{types.Int, types.Int}, + rightTypes: []*types.T{types.Int, types.Int}, + leftTuples: colexectestutils.Tuples{{nil, 4}, {nil, 2}, {0, nil}, {0, 3}, {0, nil}, {1, nil}, {3, 3}, {3, nil}, {3, 0}, {4, 0}}, + rightTuples: colexectestutils.Tuples{{1, nil}, {nil, nil}, {nil, 0}, {3, 1}, {3, 1}, {1, 1}, {nil, 2}, {2, 2}, {3, 3}, {2, 4}}, + leftEqCols: []uint32{0}, + rightEqCols: []uint32{1}, + leftOutCols: []uint32{0, 1}, + expected: colexectestutils.Tuples{{0, nil}, {0, 3}, {0, nil}, {1, nil}, {3, 3}, {3, nil}, {3, 0}, {4, 0}}, + }, + { + description: "FULL OUTER join with nulls and DESC", + joinType: descpb.FullOuterJoin, + leftTypes: []*types.T{types.Int, types.Int}, + rightTypes: []*types.T{types.Int, types.Int}, + leftTuples: colexectestutils.Tuples{{0, 4}, {nil, 0}, {3, 0}}, + rightTuples: colexectestutils.Tuples{{0, 1}, {nil, 0}, {4, nil}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{1}, + rightEqCols: []uint32{1}, + leftDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, + rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, + expected: colexectestutils.Tuples{{0, 4, nil, nil}, {nil, nil, 0, 1}, {nil, 0, nil, 0}, {3, 0, nil, 0}, {nil, nil, 4, nil}}, + }, } return withMirrors(mjTestCases) } diff --git a/pkg/sql/logictest/testdata/logic_test/cross_join b/pkg/sql/logictest/testdata/logic_test/cross_join new file mode 100644 index 000000000000..919b74bcfc31 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/cross_join @@ -0,0 +1,84 @@ +# Check that the cross join is performed in a streaming fashion with regards to +# the left input (if it wasn't, the query below would hit the temporary disk +# storage limit in fakedist-disk config). + +statement ok +CREATE TABLE t ( + i2 INT2, + i4 INT4, + i8 INT8, + f4 FLOAT4, + f8 FLOAT8, + s STRING, + c CHAR, + b BYTES, + dc DECIMAL, + ival INTERVAL, + oid OID, + tstz TIMESTAMPTZ, + ts TIMESTAMP, + da DATE, + inet INET, + vb VARBIT +); +INSERT + INTO t +SELECT i::INT2, + i::INT4, + i::INT8, + i::FLOAT4, + i::FLOAT8, + i::STRING, + i::CHAR, + i::STRING::BYTES, + i::DECIMAL, + i::INTERVAL, + i::OID, + i::TIMESTAMPTZ, + i::TIMESTAMP, + i::DATE, + ('127.0.0.' || i::STRING)::INET, + i::VARBIT + FROM ( + SELECT i FROM generate_series(1, 2) as t(i) + UNION ALL SELECT NULL + ); +SELECT * + FROM ( + SELECT a.i2, + b.i4, + c.i8, + d.f4, + e.f8, + f.s, + g.c, + h.b, + i.dc, + j.ival, + k.oid, + l.tstz, + m.ts, + n.da, + o.inet, + p.vb, + random() AS r + FROM t AS a, + t AS b, + t AS c, + t AS d, + t AS e, + t AS f, + t AS g, + t AS h, + t AS i, + t AS j, + t AS k, + t AS l, + t AS m, + t AS n, + t AS o, + t AS p + ) + WHERE r < .01 + LIMIT 1 + diff --git a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local index 0cc75705a171..aba3ce5f0de5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local +++ b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local @@ -447,3 +447,28 @@ EXPLAIN (VEC) DELETE FROM t70438 WHERE k=3 OR v=6 └ *colexec.SerialUnorderedSynchronizer ├ *colfetcher.ColBatchScan └ *colfetcher.ColBatchScan + +# Some tests for set-op cross joins. +statement ok +CREATE TABLE t (); +CREATE TABLE u (); +INSERT INTO t (rowid) VALUES (1), (2); +INSERT INTO u (rowid) VALUES (1); + +query T +EXPLAIN (VEC) SELECT * FROM t INTERSECT ALL SELECT * FROM u +---- +│ +└ Node 1 + └ *colexecjoin.crossJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan + +query T +EXPLAIN (VEC) SELECT * FROM t EXCEPT ALL SELECT * FROM u +---- +│ +└ Node 1 + └ *colexecjoin.crossJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan From 7b5a0a9a3f51fc3d87a1342240519ffd04b26d73 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 16 Aug 2021 20:40:42 -0700 Subject: [PATCH 2/4] colexecjoin: improve probing in the merge joiner with nulls For non set-operation joins whenever we have nulls in both columns we can advance both pointers since neither of the rows will have a match. This commit takes advantage of this observation as well as refactors (hopefully making it cleaner) the probing mechanism a bit. Release note: None --- .../colexecjoin/mergejoiner_exceptall.eg.go | 1368 +++++----- .../colexecjoin/mergejoiner_fullouter.eg.go | 2424 +++++++++-------- .../colexecjoin/mergejoiner_inner.eg.go | 1456 ++++++---- .../mergejoiner_intersectall.eg.go | 884 +++--- .../colexecjoin/mergejoiner_leftanti.eg.go | 1940 +++++++------ .../colexecjoin/mergejoiner_leftouter.eg.go | 1940 +++++++------ .../colexecjoin/mergejoiner_leftsemi.eg.go | 1456 ++++++---- .../colexecjoin/mergejoiner_rightanti.eg.go | 1940 +++++++------ .../colexecjoin/mergejoiner_rightouter.eg.go | 1940 +++++++------ .../colexecjoin/mergejoiner_rightsemi.eg.go | 1456 ++++++---- .../colexec/colexecjoin/mergejoiner_tmpl.go | 56 +- .../colexec/colexecjoin/mergejoiner_util.go | 6 + pkg/sql/colexec/mergejoiner_test.go | 13 + 13 files changed, 9726 insertions(+), 7153 deletions(-) diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go index 083dded076b4..86e91620a528 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,19 +77,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -214,7 +213,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -225,7 +224,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -288,7 +287,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -298,10 +297,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -323,19 +327,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -437,7 +439,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -448,7 +450,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -503,7 +505,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -513,10 +515,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -538,19 +545,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -652,7 +657,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -663,7 +668,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -718,7 +723,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -728,10 +733,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -752,19 +762,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -899,7 +907,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -910,7 +918,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -976,7 +984,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -986,10 +994,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -1007,19 +1020,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1154,7 +1165,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1165,7 +1176,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -1231,7 +1242,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1241,10 +1252,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1263,19 +1279,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1410,7 +1424,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1421,7 +1435,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -1487,7 +1501,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1497,10 +1511,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1522,19 +1541,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1693,7 +1710,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1704,7 +1721,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -1778,7 +1795,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1788,10 +1805,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1813,19 +1835,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1948,7 +1968,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1959,7 +1979,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -2021,7 +2041,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2031,10 +2051,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -2056,19 +2081,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -2170,7 +2193,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2181,7 +2204,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -2236,7 +2259,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2246,10 +2269,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2271,19 +2299,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -2403,7 +2429,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2414,7 +2440,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -2475,7 +2501,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2485,10 +2511,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2510,19 +2541,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -2630,7 +2659,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2641,7 +2670,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -2698,7 +2727,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2708,10 +2737,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2732,6 +2766,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2754,19 +2789,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2892,7 +2925,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2903,7 +2936,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -2966,7 +2999,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2976,10 +3009,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -3001,19 +3039,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3115,7 +3151,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3126,7 +3162,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -3181,7 +3217,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3191,10 +3227,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -3216,19 +3257,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3330,7 +3369,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3341,7 +3380,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -3396,7 +3435,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3406,10 +3445,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3430,19 +3474,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3577,7 +3619,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3588,7 +3630,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -3654,7 +3696,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3664,10 +3706,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -3685,19 +3732,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3832,7 +3877,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3843,7 +3888,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -3909,7 +3954,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3919,10 +3964,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3941,19 +3991,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -4088,7 +4136,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4099,7 +4147,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -4165,7 +4213,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4175,10 +4223,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -4200,19 +4253,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -4371,7 +4422,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4382,7 +4433,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -4456,7 +4507,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4466,10 +4517,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4491,19 +4547,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -4626,7 +4680,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4637,7 +4691,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -4699,7 +4753,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4709,10 +4763,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -4734,19 +4793,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -4848,7 +4905,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4859,7 +4916,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -4914,7 +4971,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4924,10 +4981,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -4949,19 +5011,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -5081,7 +5141,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5092,7 +5152,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -5153,7 +5213,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5163,10 +5223,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -5188,19 +5253,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -5308,7 +5371,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5319,7 +5382,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -5376,7 +5439,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5386,10 +5449,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -5410,6 +5478,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -5432,19 +5501,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5570,7 +5637,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5581,7 +5648,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -5644,7 +5711,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5654,10 +5721,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5679,19 +5751,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5793,7 +5863,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5804,7 +5874,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -5859,7 +5929,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5869,10 +5939,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5894,19 +5969,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -6008,7 +6081,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6019,7 +6092,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -6074,7 +6147,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6084,10 +6157,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -6108,19 +6186,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -6255,7 +6331,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6266,7 +6342,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -6332,7 +6408,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6342,10 +6418,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6363,19 +6444,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -6510,7 +6589,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6521,7 +6600,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -6587,7 +6666,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6597,10 +6676,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -6619,19 +6703,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -6766,7 +6848,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6777,7 +6859,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -6843,7 +6925,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6853,10 +6935,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6878,19 +6965,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -7049,7 +7134,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7060,7 +7145,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -7134,7 +7219,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7144,10 +7229,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -7169,19 +7259,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -7304,7 +7392,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7315,7 +7403,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -7377,7 +7465,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7387,10 +7475,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -7412,19 +7505,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -7526,7 +7617,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7537,7 +7628,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -7592,7 +7683,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7602,10 +7693,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -7627,19 +7723,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -7759,7 +7853,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7770,7 +7864,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -7831,7 +7925,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7841,10 +7935,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7866,19 +7965,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -7986,7 +8083,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7997,7 +8094,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -8054,7 +8151,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8064,10 +8161,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -8088,6 +8190,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -8110,19 +8213,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -8248,7 +8349,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8259,7 +8360,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -8322,7 +8423,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8332,10 +8433,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -8357,19 +8463,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -8471,7 +8575,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8482,7 +8586,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -8537,7 +8641,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8547,10 +8651,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -8572,19 +8681,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -8686,7 +8793,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8697,7 +8804,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -8752,7 +8859,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8762,10 +8869,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -8786,19 +8898,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -8933,7 +9043,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8944,7 +9054,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -9010,7 +9120,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9020,10 +9130,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -9041,19 +9156,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -9188,7 +9301,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9199,7 +9312,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -9265,7 +9378,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9275,10 +9388,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -9297,19 +9415,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -9444,7 +9560,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9455,7 +9571,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -9521,7 +9637,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9531,10 +9647,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -9556,19 +9677,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -9727,7 +9846,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9738,7 +9857,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -9812,7 +9931,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9822,10 +9941,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -9847,19 +9971,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -9982,7 +10104,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9993,7 +10115,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -10055,7 +10177,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10065,10 +10187,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -10090,19 +10217,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -10204,7 +10329,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10215,7 +10340,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -10270,7 +10395,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10280,10 +10405,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -10305,19 +10435,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -10437,7 +10565,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10448,7 +10576,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -10509,7 +10637,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10519,10 +10647,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -10544,19 +10677,17 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -10664,7 +10795,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10675,7 +10806,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // For EXCEPT ALL join we add (lGroupLength - rGroupLength) number @@ -10732,7 +10863,7 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10742,10 +10873,15 @@ func (o *mergeJoinExceptAllOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go index 29a35664057a..0d8d1fde22d9 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,43 +77,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -211,7 +212,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -222,7 +223,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -303,7 +304,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -313,7 +314,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -323,10 +324,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -348,43 +354,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -459,7 +465,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -470,7 +476,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -535,7 +541,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -545,7 +551,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -555,10 +561,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -580,43 +591,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -691,7 +702,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -702,7 +713,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -767,7 +778,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -777,7 +788,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -787,10 +798,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -811,43 +827,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -955,7 +971,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -966,7 +982,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1053,7 +1069,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1063,7 +1079,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1073,10 +1089,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -1094,43 +1115,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1238,7 +1259,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1249,7 +1270,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1336,7 +1357,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1346,7 +1367,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1356,10 +1377,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1378,43 +1404,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1522,7 +1548,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1533,7 +1559,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1620,7 +1646,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1630,7 +1656,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1640,10 +1666,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1665,43 +1696,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1833,7 +1864,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1844,7 +1875,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1947,7 +1978,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1957,7 +1988,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1967,10 +1998,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1992,43 +2028,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2124,7 +2160,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2135,7 +2171,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2214,7 +2250,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2224,7 +2260,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2234,10 +2270,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -2259,43 +2300,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2370,7 +2411,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2381,7 +2422,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2446,7 +2487,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2456,7 +2497,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2466,10 +2507,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2491,43 +2537,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2620,7 +2666,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2631,7 +2677,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2708,7 +2754,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2718,7 +2764,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2728,10 +2774,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2753,43 +2804,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2870,7 +2921,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2881,7 +2932,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2950,7 +3001,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2960,7 +3011,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2970,10 +3021,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2994,6 +3050,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -3016,43 +3073,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3151,7 +3208,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3162,7 +3219,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3243,7 +3300,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3253,7 +3310,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3263,10 +3320,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -3288,43 +3350,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3399,7 +3461,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3410,7 +3472,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3475,7 +3537,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3485,7 +3547,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3495,10 +3557,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -3520,43 +3587,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3631,7 +3698,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3642,7 +3709,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3707,7 +3774,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3717,7 +3784,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3727,10 +3794,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3751,43 +3823,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3895,7 +3967,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3906,7 +3978,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3993,7 +4065,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4003,7 +4075,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4013,10 +4085,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -4034,43 +4111,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4178,7 +4255,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4189,7 +4266,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4276,7 +4353,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4286,7 +4363,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4296,10 +4373,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -4318,43 +4400,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4462,7 +4544,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4473,7 +4555,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4560,7 +4642,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4570,7 +4652,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4580,10 +4662,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -4605,43 +4692,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4773,7 +4860,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4784,7 +4871,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4887,7 +4974,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4897,7 +4984,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4907,10 +4994,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4932,43 +5024,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5064,7 +5156,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5075,7 +5167,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5154,7 +5246,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5164,7 +5256,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5174,10 +5266,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -5199,43 +5296,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5310,7 +5407,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5321,7 +5418,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5386,7 +5483,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5396,7 +5493,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5406,10 +5503,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -5431,43 +5533,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5560,7 +5662,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5571,7 +5673,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5648,7 +5750,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5658,7 +5760,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5668,10 +5770,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -5693,43 +5800,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5810,7 +5917,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5821,7 +5928,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5890,7 +5997,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5900,7 +6007,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5910,10 +6017,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -5934,6 +6046,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -5956,43 +6069,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6091,7 +6204,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6102,7 +6215,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6183,7 +6296,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6193,7 +6306,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6203,10 +6316,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -6228,43 +6346,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6339,7 +6457,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6350,7 +6468,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6415,7 +6533,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6425,7 +6543,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6435,10 +6553,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -6460,43 +6583,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6571,7 +6694,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6582,7 +6705,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6647,7 +6770,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6657,7 +6780,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6667,10 +6790,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -6691,43 +6819,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6835,7 +6963,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6846,7 +6974,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6933,7 +7061,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6943,7 +7071,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6953,10 +7081,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6974,43 +7107,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7118,7 +7251,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7129,7 +7262,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7216,7 +7349,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7226,7 +7359,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7236,10 +7369,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -7258,43 +7396,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7402,7 +7540,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7413,7 +7551,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7500,7 +7638,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7510,7 +7648,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7520,10 +7658,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -7545,43 +7688,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7713,7 +7856,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7724,7 +7867,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7827,7 +7970,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7837,7 +7980,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7847,10 +7990,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -7872,43 +8020,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8004,7 +8152,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8015,7 +8163,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8094,7 +8242,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8104,7 +8252,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8114,10 +8262,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -8139,43 +8292,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8250,7 +8403,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8261,7 +8414,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8326,7 +8479,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8336,7 +8489,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8346,10 +8499,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -8371,43 +8529,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8500,7 +8658,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8511,7 +8669,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8588,7 +8746,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8598,7 +8756,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8608,10 +8766,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -8633,43 +8796,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8750,7 +8913,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8761,7 +8924,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8830,7 +8993,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8840,7 +9003,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8850,10 +9013,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -8874,6 +9042,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -8896,43 +9065,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9031,7 +9200,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9042,7 +9211,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9123,7 +9292,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9133,7 +9302,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9143,10 +9312,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -9168,43 +9342,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9279,7 +9453,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9290,7 +9464,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9355,7 +9529,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9365,7 +9539,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9375,10 +9549,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -9400,43 +9579,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9511,7 +9690,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9522,7 +9701,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9587,7 +9766,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9597,7 +9776,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9607,10 +9786,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -9631,43 +9815,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9775,7 +9959,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9786,7 +9970,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9873,7 +10057,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9883,7 +10067,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9893,10 +10077,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -9914,43 +10103,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -10058,7 +10247,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10069,7 +10258,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -10156,7 +10345,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10166,7 +10355,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -10176,10 +10365,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -10198,43 +10392,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -10342,7 +10536,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10353,7 +10547,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -10440,7 +10634,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10450,7 +10644,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -10460,10 +10654,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -10485,43 +10684,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -10653,7 +10852,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10664,7 +10863,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -10767,7 +10966,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -10777,7 +10976,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -10787,10 +10986,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -10812,43 +11016,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -10944,7 +11148,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -10955,7 +11159,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -11034,7 +11238,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -11044,7 +11248,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -11054,10 +11258,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -11079,43 +11288,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -11190,7 +11399,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -11201,7 +11410,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -11266,7 +11475,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -11276,7 +11485,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -11286,10 +11495,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -11311,43 +11525,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -11440,7 +11654,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -11451,7 +11665,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -11528,7 +11742,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -11538,7 +11752,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -11548,10 +11762,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -11573,43 +11792,43 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -11690,7 +11909,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -11701,7 +11920,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -11770,7 +11989,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -11780,7 +11999,7 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -11790,10 +12009,15 @@ func (o *mergeJoinFullOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go index 0ff915b3120c..fdd9d665d452 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,21 +77,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -189,7 +192,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -200,7 +203,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -220,10 +223,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -245,21 +253,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -334,7 +344,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -345,7 +355,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -365,10 +375,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -390,21 +405,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -479,7 +496,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -490,7 +507,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -510,10 +527,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -534,21 +556,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -656,7 +680,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -667,7 +691,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -687,10 +711,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -708,21 +737,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -830,7 +861,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -841,7 +872,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -861,10 +892,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -883,21 +919,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1005,7 +1043,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1016,7 +1054,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1036,10 +1074,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1061,21 +1104,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1207,7 +1252,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1218,7 +1263,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1238,10 +1283,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1263,21 +1313,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1373,7 +1425,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1384,7 +1436,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1404,10 +1456,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1429,21 +1486,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1518,7 +1577,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1529,7 +1588,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1549,10 +1608,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -1574,21 +1638,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1681,7 +1747,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1692,7 +1758,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1712,10 +1778,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -1737,21 +1808,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1832,7 +1905,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1843,7 +1916,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1863,10 +1936,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -1887,6 +1965,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -1909,21 +1988,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2022,7 +2103,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2033,7 +2114,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2053,10 +2134,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2078,21 +2164,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2167,7 +2255,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2178,7 +2266,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2198,10 +2286,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2223,21 +2316,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2312,7 +2407,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2323,7 +2418,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2343,10 +2438,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -2367,21 +2467,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2489,7 +2591,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2500,7 +2602,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2520,10 +2622,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -2541,21 +2648,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2663,7 +2772,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2674,7 +2783,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2694,10 +2803,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -2716,21 +2830,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2838,7 +2954,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2849,7 +2965,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2869,10 +2985,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -2894,21 +3015,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3040,7 +3163,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3051,7 +3174,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3071,10 +3194,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -3096,21 +3224,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3206,7 +3336,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3217,7 +3347,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3237,10 +3367,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -3262,21 +3397,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3351,7 +3488,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3362,7 +3499,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3382,10 +3519,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -3407,21 +3549,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3514,7 +3658,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3525,7 +3669,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3545,10 +3689,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -3570,21 +3719,23 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3665,7 +3816,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3676,7 +3827,7 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3696,10 +3847,15 @@ func (o *mergeJoinInnerOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -3720,6 +3876,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -3742,21 +3899,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3855,7 +4014,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3866,7 +4025,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3886,10 +4045,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -3911,21 +4075,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4000,7 +4166,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4011,7 +4177,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4031,10 +4197,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -4056,21 +4227,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4145,7 +4318,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4156,7 +4329,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4176,10 +4349,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -4200,21 +4378,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4322,7 +4502,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4333,7 +4513,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4353,10 +4533,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -4374,21 +4559,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4496,7 +4683,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4507,7 +4694,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4527,10 +4714,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -4549,21 +4741,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4671,7 +4865,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4682,7 +4876,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4702,10 +4896,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -4727,21 +4926,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4873,7 +5074,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4884,7 +5085,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4904,10 +5105,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4929,21 +5135,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5039,7 +5247,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5050,7 +5258,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5070,10 +5278,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -5095,21 +5308,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5184,7 +5399,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5195,7 +5410,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5215,10 +5430,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -5240,21 +5460,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5347,7 +5569,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5358,7 +5580,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5378,10 +5600,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -5403,21 +5630,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5498,7 +5727,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5509,7 +5738,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5529,10 +5758,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -5553,6 +5787,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -5575,21 +5810,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5688,7 +5925,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5699,7 +5936,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5719,10 +5956,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5744,21 +5986,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5833,7 +6077,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5844,7 +6088,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5864,10 +6108,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5889,21 +6138,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5978,7 +6229,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5989,7 +6240,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6009,10 +6260,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -6033,21 +6289,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6155,7 +6413,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6166,7 +6424,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6186,10 +6444,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6207,21 +6470,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6329,7 +6594,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6340,7 +6605,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6360,10 +6625,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -6382,21 +6652,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6504,7 +6776,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6515,7 +6787,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6535,10 +6807,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6560,21 +6837,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6706,7 +6985,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6717,7 +6996,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6737,10 +7016,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6762,21 +7046,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6872,7 +7158,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6883,7 +7169,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6903,10 +7189,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6928,21 +7219,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7017,7 +7310,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7028,7 +7321,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7048,10 +7341,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -7073,21 +7371,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7180,7 +7480,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7191,7 +7491,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7211,10 +7511,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7236,21 +7541,23 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7331,7 +7638,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7342,7 +7649,7 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7362,10 +7669,15 @@ func (o *mergeJoinInnerOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go index e785de22a4c0..55c13e7a4139 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,9 +77,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -204,7 +204,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -215,7 +215,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -239,10 +239,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -264,9 +269,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -368,7 +372,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -379,7 +383,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -403,10 +407,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -428,9 +437,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -532,7 +540,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -543,7 +551,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -567,10 +575,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -591,9 +604,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -728,7 +740,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -739,7 +751,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -763,10 +775,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -784,9 +801,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -921,7 +937,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -932,7 +948,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -956,10 +972,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -978,9 +999,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1115,7 +1135,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1126,7 +1146,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1150,10 +1170,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1175,9 +1200,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1336,7 +1360,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1347,7 +1371,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1371,10 +1395,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1396,9 +1425,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1521,7 +1549,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1532,7 +1560,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1556,10 +1584,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1581,9 +1614,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1685,7 +1717,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1696,7 +1728,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1720,10 +1752,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -1745,9 +1782,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -1867,7 +1903,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1878,7 +1914,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1902,10 +1938,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -1927,9 +1968,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -2037,7 +2077,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2048,7 +2088,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2072,10 +2112,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2096,6 +2141,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2118,9 +2164,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2246,7 +2291,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2257,7 +2302,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2281,10 +2326,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2306,9 +2356,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2410,7 +2459,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2421,7 +2470,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2445,10 +2494,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2470,9 +2524,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2574,7 +2627,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2585,7 +2638,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2609,10 +2662,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -2633,9 +2691,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2770,7 +2827,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2781,7 +2838,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2805,10 +2862,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -2826,9 +2888,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -2963,7 +3024,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2974,7 +3035,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2998,10 +3059,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3020,9 +3086,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3157,7 +3222,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3168,7 +3233,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3192,10 +3257,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -3217,9 +3287,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3378,7 +3447,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3389,7 +3458,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3413,10 +3482,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -3438,9 +3512,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3563,7 +3636,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3574,7 +3647,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3598,10 +3671,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -3623,9 +3701,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3727,7 +3804,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3738,7 +3815,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3762,10 +3839,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -3787,9 +3869,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -3909,7 +3990,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3920,7 +4001,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3944,10 +4025,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -3969,9 +4055,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) @@ -4079,7 +4164,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4090,7 +4175,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4114,10 +4199,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -4138,6 +4228,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -4160,9 +4251,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -4288,7 +4378,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4299,7 +4389,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4323,10 +4413,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -4348,9 +4443,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -4452,7 +4546,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4463,7 +4557,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4487,10 +4581,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -4512,9 +4611,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -4616,7 +4714,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4627,7 +4725,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4651,10 +4749,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -4675,9 +4778,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -4812,7 +4914,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4823,7 +4925,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4847,10 +4949,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -4868,9 +4975,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5005,7 +5111,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5016,7 +5122,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5040,10 +5146,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -5062,9 +5173,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5199,7 +5309,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5210,7 +5320,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5234,10 +5344,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -5259,9 +5374,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5420,7 +5534,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5431,7 +5545,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5455,10 +5569,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -5480,9 +5599,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5605,7 +5723,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5616,7 +5734,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5640,10 +5758,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -5665,9 +5788,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5769,7 +5891,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5780,7 +5902,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5804,10 +5926,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -5829,9 +5956,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -5951,7 +6077,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5962,7 +6088,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5986,10 +6112,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -6011,9 +6142,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) @@ -6121,7 +6251,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6132,7 +6262,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6156,10 +6286,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -6180,6 +6315,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -6202,9 +6338,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -6330,7 +6465,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6341,7 +6476,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6365,10 +6500,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -6390,9 +6530,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -6494,7 +6633,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6505,7 +6644,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6529,10 +6668,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -6554,9 +6698,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -6658,7 +6801,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6669,7 +6812,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6693,10 +6836,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -6717,9 +6865,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -6854,7 +7001,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6865,7 +7012,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6889,10 +7036,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6910,9 +7062,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7047,7 +7198,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7058,7 +7209,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7082,10 +7233,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -7104,9 +7260,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7241,7 +7396,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7252,7 +7407,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7276,10 +7431,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -7301,9 +7461,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7462,7 +7621,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7473,7 +7632,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7497,10 +7656,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -7522,9 +7686,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7647,7 +7810,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7658,7 +7821,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7682,10 +7845,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -7707,9 +7875,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7811,7 +7978,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7822,7 +7989,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7846,10 +8013,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -7871,9 +8043,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -7993,7 +8164,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8004,7 +8175,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -8028,10 +8199,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -8053,9 +8229,8 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) @@ -8163,7 +8338,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8174,7 +8349,7 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -8198,10 +8373,15 @@ func (o *mergeJoinIntersectAllOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go index ddec5991258b..dbc70f4fa95c 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,32 +77,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -200,7 +202,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -211,7 +213,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -261,7 +263,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -271,10 +273,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -296,32 +303,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -396,7 +404,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -407,7 +415,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -449,7 +457,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -459,10 +467,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -484,32 +497,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -584,7 +598,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -595,7 +609,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -637,7 +651,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -647,10 +661,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -671,32 +690,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -804,7 +824,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -815,7 +835,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -868,7 +888,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -878,10 +898,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -899,32 +924,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1032,7 +1058,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1043,7 +1069,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -1096,7 +1122,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1106,10 +1132,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1128,32 +1159,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1261,7 +1293,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1272,7 +1304,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -1325,7 +1357,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1335,10 +1367,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1360,32 +1397,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1517,7 +1555,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1528,7 +1566,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -1589,7 +1627,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1599,10 +1637,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1624,32 +1667,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1745,7 +1789,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1756,7 +1800,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -1805,7 +1849,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1815,10 +1859,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1840,32 +1889,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1940,7 +1990,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1951,7 +2001,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -1993,7 +2043,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2003,10 +2053,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2028,32 +2083,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2146,7 +2202,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2157,7 +2213,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -2205,7 +2261,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2215,10 +2271,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2240,32 +2301,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2346,7 +2408,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2357,7 +2419,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -2401,7 +2463,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2411,10 +2473,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2435,6 +2502,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2457,32 +2525,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2581,7 +2650,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2592,7 +2661,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -2642,7 +2711,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2652,10 +2721,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2677,32 +2751,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2777,7 +2852,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2788,7 +2863,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -2830,7 +2905,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2840,10 +2915,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2865,32 +2945,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2965,7 +3046,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2976,7 +3057,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -3018,7 +3099,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3028,10 +3109,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3052,32 +3138,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3185,7 +3272,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3196,7 +3283,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -3249,7 +3336,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3259,10 +3346,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -3280,32 +3372,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3413,7 +3506,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3424,7 +3517,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -3477,7 +3570,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3487,10 +3580,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3509,32 +3607,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3642,7 +3741,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3653,7 +3752,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -3706,7 +3805,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3716,10 +3815,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -3741,32 +3845,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3898,7 +4003,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3909,7 +4014,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -3970,7 +4075,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3980,10 +4085,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4005,32 +4115,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4126,7 +4237,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4137,7 +4248,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -4186,7 +4297,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4196,10 +4307,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -4221,32 +4337,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4321,7 +4438,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4332,7 +4449,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -4374,7 +4491,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4384,10 +4501,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -4409,32 +4531,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4527,7 +4650,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4538,7 +4661,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -4586,7 +4709,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4596,10 +4719,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -4621,32 +4749,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4727,7 +4856,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4738,7 +4867,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -4782,7 +4911,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4792,10 +4921,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -4816,6 +4950,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -4838,32 +4973,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4962,7 +5098,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4973,7 +5109,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -5023,7 +5159,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5033,10 +5169,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5058,32 +5199,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5158,7 +5300,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5169,7 +5311,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -5211,7 +5353,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5221,10 +5363,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5246,32 +5393,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5346,7 +5494,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5357,7 +5505,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -5399,7 +5547,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5409,10 +5557,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5433,32 +5586,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5566,7 +5720,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5577,7 +5731,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -5630,7 +5784,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5640,10 +5794,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -5661,32 +5820,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5794,7 +5954,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5805,7 +5965,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -5858,7 +6018,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5868,10 +6028,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -5890,32 +6055,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6023,7 +6189,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6034,7 +6200,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -6087,7 +6253,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6097,10 +6263,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6122,32 +6293,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6279,7 +6451,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6290,7 +6462,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -6351,7 +6523,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6361,10 +6533,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6386,32 +6563,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6507,7 +6685,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6518,7 +6696,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -6567,7 +6745,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6577,10 +6755,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6602,32 +6785,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6702,7 +6886,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6713,7 +6897,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -6755,7 +6939,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6765,10 +6949,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -6790,32 +6979,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6908,7 +7098,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6919,7 +7109,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -6967,7 +7157,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6977,10 +7167,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7002,32 +7197,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7108,7 +7304,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7119,7 +7315,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -7163,7 +7359,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7173,10 +7369,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -7197,6 +7398,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -7219,32 +7421,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7343,7 +7546,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7354,7 +7557,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -7404,7 +7607,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7414,10 +7617,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -7439,32 +7647,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7539,7 +7748,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7550,7 +7759,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -7592,7 +7801,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7602,10 +7811,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -7627,32 +7841,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7727,7 +7942,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7738,7 +7953,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -7780,7 +7995,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7790,10 +8005,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -7814,32 +8034,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7947,7 +8168,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7958,7 +8179,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -8011,7 +8232,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8021,10 +8242,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -8042,32 +8268,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8175,7 +8402,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8186,7 +8413,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -8239,7 +8466,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8249,10 +8476,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -8271,32 +8503,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8404,7 +8637,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8415,7 +8648,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -8468,7 +8701,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8478,10 +8711,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -8503,32 +8741,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8660,7 +8899,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8671,7 +8910,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -8732,7 +8971,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8742,10 +8981,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -8767,32 +9011,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8888,7 +9133,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8899,7 +9144,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -8948,7 +9193,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8958,10 +9203,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -8983,32 +9233,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9083,7 +9334,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9094,7 +9345,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -9136,7 +9387,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9146,10 +9397,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -9171,32 +9427,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9289,7 +9546,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9300,7 +9557,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -9348,7 +9605,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9358,10 +9615,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -9383,32 +9645,33 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9489,7 +9752,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9500,7 +9763,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With LEFT ANTI join, we are only interested in unmatched tuples @@ -9544,7 +9807,7 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9554,10 +9817,15 @@ func (o *mergeJoinLeftAntiOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go index 823ca30d0842..f8d6f065d6bd 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,32 +77,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -200,7 +202,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -211,7 +213,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -262,7 +264,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -272,10 +274,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -297,32 +304,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -397,7 +405,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -408,7 +416,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -451,7 +459,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -461,10 +469,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -486,32 +499,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -586,7 +600,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -597,7 +611,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -640,7 +654,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -650,10 +664,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -674,32 +693,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -807,7 +827,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -818,7 +838,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -872,7 +892,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -882,10 +902,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -903,32 +928,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1036,7 +1062,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1047,7 +1073,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1101,7 +1127,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1111,10 +1137,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1133,32 +1164,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1266,7 +1298,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1277,7 +1309,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1331,7 +1363,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1341,10 +1373,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1366,32 +1403,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1523,7 +1561,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1534,7 +1572,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1596,7 +1634,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1606,10 +1644,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1631,32 +1674,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1752,7 +1796,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1763,7 +1807,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1813,7 +1857,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -1823,10 +1867,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1848,32 +1897,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1948,7 +1998,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1959,7 +2009,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2002,7 +2052,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2012,10 +2062,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2037,32 +2092,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2155,7 +2211,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2166,7 +2222,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2215,7 +2271,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2225,10 +2281,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2250,32 +2311,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2356,7 +2418,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2367,7 +2429,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2412,7 +2474,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2422,10 +2484,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2446,6 +2513,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2468,32 +2536,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2592,7 +2661,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2603,7 +2672,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2654,7 +2723,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2664,10 +2733,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2689,32 +2763,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2789,7 +2864,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2800,7 +2875,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2843,7 +2918,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -2853,10 +2928,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2878,32 +2958,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2978,7 +3059,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2989,7 +3070,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3032,7 +3113,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3042,10 +3123,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3066,32 +3152,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3199,7 +3286,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3210,7 +3297,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3264,7 +3351,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3274,10 +3361,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -3295,32 +3387,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3428,7 +3521,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3439,7 +3532,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3493,7 +3586,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3503,10 +3596,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3525,32 +3623,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3658,7 +3757,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3669,7 +3768,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3723,7 +3822,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3733,10 +3832,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -3758,32 +3862,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3915,7 +4020,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3926,7 +4031,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3988,7 +4093,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -3998,10 +4103,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4023,32 +4133,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4144,7 +4255,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4155,7 +4266,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4205,7 +4316,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4215,10 +4326,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -4240,32 +4356,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4340,7 +4457,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4351,7 +4468,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4394,7 +4511,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4404,10 +4521,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -4429,32 +4551,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4547,7 +4670,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4558,7 +4681,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4607,7 +4730,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4617,10 +4740,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -4642,32 +4770,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4748,7 +4877,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4759,7 +4888,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4804,7 +4933,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -4814,10 +4943,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSeltrueRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -4838,6 +4972,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -4860,32 +4995,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4984,7 +5120,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4995,7 +5131,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5046,7 +5182,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5056,10 +5192,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5081,32 +5222,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5181,7 +5323,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5192,7 +5334,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5235,7 +5377,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5245,10 +5387,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5270,32 +5417,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5370,7 +5518,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5381,7 +5529,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5424,7 +5572,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5434,10 +5582,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5458,32 +5611,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5591,7 +5745,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5602,7 +5756,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5656,7 +5810,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5666,10 +5820,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -5687,32 +5846,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5820,7 +5980,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5831,7 +5991,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5885,7 +6045,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -5895,10 +6055,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -5917,32 +6082,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6050,7 +6216,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6061,7 +6227,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6115,7 +6281,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6125,10 +6291,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6150,32 +6321,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6307,7 +6479,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6318,7 +6490,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6380,7 +6552,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6390,10 +6562,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6415,32 +6592,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6536,7 +6714,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6547,7 +6725,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6597,7 +6775,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6607,10 +6785,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6632,32 +6815,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6732,7 +6916,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6743,7 +6927,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6786,7 +6970,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -6796,10 +6980,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -6821,32 +7010,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6939,7 +7129,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6950,7 +7140,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6999,7 +7189,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7009,10 +7199,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7034,32 +7229,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7140,7 +7336,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7151,7 +7347,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7196,7 +7392,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7206,10 +7402,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSeltrue() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -7230,6 +7431,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -7252,32 +7454,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7376,7 +7579,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7387,7 +7590,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7438,7 +7641,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7448,10 +7651,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -7473,32 +7681,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7573,7 +7782,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7584,7 +7793,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7627,7 +7836,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7637,10 +7846,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -7662,32 +7876,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7762,7 +7977,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7773,7 +7988,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7816,7 +8031,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -7826,10 +8041,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -7850,32 +8070,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7983,7 +8204,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7994,7 +8215,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8048,7 +8269,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8058,10 +8279,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -8079,32 +8305,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8212,7 +8439,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8223,7 +8450,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8277,7 +8504,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8287,10 +8514,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -8309,32 +8541,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8442,7 +8675,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8453,7 +8686,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8507,7 +8740,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8517,10 +8750,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -8542,32 +8780,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8699,7 +8938,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8710,7 +8949,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8772,7 +9011,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8782,10 +9021,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -8807,32 +9051,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8928,7 +9173,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8939,7 +9184,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8989,7 +9234,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -8999,10 +9244,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -9024,32 +9274,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9124,7 +9375,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9135,7 +9386,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9178,7 +9429,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9188,10 +9439,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -9213,32 +9469,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9331,7 +9588,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9342,7 +9599,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9391,7 +9648,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9401,10 +9658,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -9426,32 +9688,33 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9532,7 +9795,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9543,7 +9806,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9588,7 +9851,7 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -9598,10 +9861,15 @@ func (o *mergeJoinLeftOuterOp) probeBodyLSelfalseRSelfalse() { curLIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go index 7a1f6af3fa0e..ddab25e50f6c 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,21 +77,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -189,7 +192,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -200,7 +203,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -219,10 +222,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -244,21 +252,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -333,7 +343,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -344,7 +354,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -363,10 +373,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -388,21 +403,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -477,7 +494,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -488,7 +505,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -507,10 +524,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -531,21 +553,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -653,7 +677,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -664,7 +688,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -683,10 +707,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -704,21 +733,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -826,7 +857,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -837,7 +868,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -856,10 +887,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -878,21 +914,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1000,7 +1038,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1011,7 +1049,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1030,10 +1068,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1055,21 +1098,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1201,7 +1246,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1212,7 +1257,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1231,10 +1276,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1256,21 +1306,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1366,7 +1418,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1377,7 +1429,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1396,10 +1448,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1421,21 +1478,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1510,7 +1569,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1521,7 +1580,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1540,10 +1599,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -1565,21 +1629,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1672,7 +1738,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1683,7 +1749,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1702,10 +1768,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -1727,21 +1798,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1822,7 +1895,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1833,7 +1906,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -1852,10 +1925,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -1876,6 +1954,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -1898,21 +1977,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2011,7 +2092,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2022,7 +2103,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2041,10 +2122,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2066,21 +2152,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2155,7 +2243,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2166,7 +2254,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2185,10 +2273,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2210,21 +2303,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2299,7 +2394,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2310,7 +2405,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2329,10 +2424,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -2353,21 +2453,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2475,7 +2577,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2486,7 +2588,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2505,10 +2607,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -2526,21 +2633,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2648,7 +2757,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2659,7 +2768,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2678,10 +2787,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -2700,21 +2814,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2822,7 +2938,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2833,7 +2949,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -2852,10 +2968,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -2877,21 +2998,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3023,7 +3146,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3034,7 +3157,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3053,10 +3176,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -3078,21 +3206,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3188,7 +3318,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3199,7 +3329,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3218,10 +3348,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -3243,21 +3378,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3332,7 +3469,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3343,7 +3480,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3362,10 +3499,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -3387,21 +3529,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3494,7 +3638,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3505,7 +3649,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3524,10 +3668,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -3549,21 +3698,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3644,7 +3795,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3655,7 +3806,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3674,10 +3825,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -3698,6 +3854,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -3720,21 +3877,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3833,7 +3992,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3844,7 +4003,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -3863,10 +4022,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -3888,21 +4052,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3977,7 +4143,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3988,7 +4154,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4007,10 +4173,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -4032,21 +4203,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4121,7 +4294,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4132,7 +4305,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4151,10 +4324,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -4175,21 +4353,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4297,7 +4477,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4308,7 +4488,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4327,10 +4507,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -4348,21 +4533,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4470,7 +4657,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4481,7 +4668,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4500,10 +4687,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -4522,21 +4714,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4644,7 +4838,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4655,7 +4849,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4674,10 +4868,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -4699,21 +4898,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4845,7 +5046,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4856,7 +5057,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -4875,10 +5076,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4900,21 +5106,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5010,7 +5218,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5021,7 +5229,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5040,10 +5248,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -5065,21 +5278,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5154,7 +5369,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5165,7 +5380,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5184,10 +5399,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -5209,21 +5429,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5316,7 +5538,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5327,7 +5549,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5346,10 +5568,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -5371,21 +5598,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5466,7 +5695,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5477,7 +5706,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5496,10 +5725,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -5520,6 +5754,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -5542,21 +5777,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5655,7 +5892,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5666,7 +5903,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5685,10 +5922,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5710,21 +5952,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5799,7 +6043,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5810,7 +6054,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5829,10 +6073,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5854,21 +6103,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5943,7 +6194,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5954,7 +6205,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -5973,10 +6224,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5997,21 +6253,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6119,7 +6377,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6130,7 +6388,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6149,10 +6407,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6170,21 +6433,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6292,7 +6557,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6303,7 +6568,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6322,10 +6587,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -6344,21 +6614,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6466,7 +6738,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6477,7 +6749,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6496,10 +6768,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6521,21 +6798,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6667,7 +6946,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6678,7 +6957,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6697,10 +6976,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6722,21 +7006,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6832,7 +7118,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6843,7 +7129,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -6862,10 +7148,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6887,21 +7178,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6976,7 +7269,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6987,7 +7280,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7006,10 +7299,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -7031,21 +7329,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7138,7 +7438,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7149,7 +7449,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7168,10 +7468,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7193,21 +7498,23 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7288,7 +7595,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7299,7 +7606,7 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { leftSemiGroupLength := lGroupLength @@ -7318,10 +7625,15 @@ func (o *mergeJoinLeftSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go index 224822d898b9..d782f4429bf2 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,32 +77,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -200,7 +202,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -211,7 +213,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -260,7 +262,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -270,10 +272,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -295,32 +302,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -395,7 +403,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -406,7 +414,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -447,7 +455,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -457,10 +465,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -482,32 +495,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -582,7 +596,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -593,7 +607,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -634,7 +648,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -644,10 +658,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -668,32 +687,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -801,7 +821,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -812,7 +832,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -864,7 +884,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -874,10 +894,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -895,32 +920,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1028,7 +1054,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1039,7 +1065,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -1091,7 +1117,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1101,10 +1127,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1123,32 +1154,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1256,7 +1288,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1267,7 +1299,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -1319,7 +1351,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1329,10 +1361,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1354,32 +1391,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1511,7 +1549,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1522,7 +1560,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -1582,7 +1620,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1592,10 +1630,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1617,32 +1660,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1738,7 +1782,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1749,7 +1793,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -1797,7 +1841,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1807,10 +1851,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1832,32 +1881,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1932,7 +1982,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1943,7 +1993,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -1984,7 +2034,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1994,10 +2044,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2019,32 +2074,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2137,7 +2193,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2148,7 +2204,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -2195,7 +2251,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2205,10 +2261,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2230,32 +2291,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2336,7 +2398,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2347,7 +2409,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -2390,7 +2452,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2400,10 +2462,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2424,6 +2491,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2446,32 +2514,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2570,7 +2639,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2581,7 +2650,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -2630,7 +2699,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2640,10 +2709,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2665,32 +2739,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2765,7 +2840,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2776,7 +2851,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -2817,7 +2892,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2827,10 +2902,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2852,32 +2932,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2952,7 +3033,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2963,7 +3044,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -3004,7 +3085,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3014,10 +3095,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3038,32 +3124,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3171,7 +3258,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3182,7 +3269,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -3234,7 +3321,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3244,10 +3331,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -3265,32 +3357,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3398,7 +3491,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3409,7 +3502,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -3461,7 +3554,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3471,10 +3564,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3493,32 +3591,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3626,7 +3725,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3637,7 +3736,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -3689,7 +3788,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3699,10 +3798,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -3724,32 +3828,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3881,7 +3986,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3892,7 +3997,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -3952,7 +4057,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3962,10 +4067,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -3987,32 +4097,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4108,7 +4219,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4119,7 +4230,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -4167,7 +4278,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4177,10 +4288,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -4202,32 +4318,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4302,7 +4419,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4313,7 +4430,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -4354,7 +4471,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4364,10 +4481,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -4389,32 +4511,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4507,7 +4630,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4518,7 +4641,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -4565,7 +4688,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4575,10 +4698,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -4600,32 +4728,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4706,7 +4835,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4717,7 +4846,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -4760,7 +4889,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4770,10 +4899,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -4794,6 +4928,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -4816,32 +4951,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4940,7 +5076,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4951,7 +5087,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -5000,7 +5136,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5010,10 +5146,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5035,32 +5176,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5135,7 +5277,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5146,7 +5288,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -5187,7 +5329,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5197,10 +5339,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5222,32 +5369,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5322,7 +5470,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5333,7 +5481,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -5374,7 +5522,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5384,10 +5532,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5408,32 +5561,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5541,7 +5695,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5552,7 +5706,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -5604,7 +5758,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5614,10 +5768,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -5635,32 +5794,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5768,7 +5928,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5779,7 +5939,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -5831,7 +5991,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5841,10 +6001,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -5863,32 +6028,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5996,7 +6162,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6007,7 +6173,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -6059,7 +6225,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6069,10 +6235,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6094,32 +6265,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6251,7 +6423,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6262,7 +6434,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -6322,7 +6494,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6332,10 +6504,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6357,32 +6534,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6478,7 +6656,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6489,7 +6667,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -6537,7 +6715,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6547,10 +6725,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6572,32 +6755,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6672,7 +6856,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6683,7 +6867,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -6724,7 +6908,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6734,10 +6918,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -6759,32 +6948,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6877,7 +7067,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6888,7 +7078,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -6935,7 +7125,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6945,10 +7135,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -6970,32 +7165,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7076,7 +7272,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7087,7 +7283,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -7130,7 +7326,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7140,10 +7336,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -7164,6 +7365,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -7186,32 +7388,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7310,7 +7513,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7321,7 +7524,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -7370,7 +7573,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7380,10 +7583,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -7405,32 +7613,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7505,7 +7714,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7516,7 +7725,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -7557,7 +7766,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7567,10 +7776,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -7592,32 +7806,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7692,7 +7907,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7703,7 +7918,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -7744,7 +7959,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7754,10 +7969,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -7778,32 +7998,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7911,7 +8132,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7922,7 +8143,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -7974,7 +8195,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7984,10 +8205,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -8005,32 +8231,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8138,7 +8365,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8149,7 +8376,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -8201,7 +8428,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8211,10 +8438,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -8233,32 +8465,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8366,7 +8599,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8377,7 +8610,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -8429,7 +8662,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8439,10 +8672,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -8464,32 +8702,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8621,7 +8860,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8632,7 +8871,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -8692,7 +8931,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8702,10 +8941,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -8727,32 +8971,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8848,7 +9093,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8859,7 +9104,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -8907,7 +9152,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8917,10 +9162,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -8942,32 +9192,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9042,7 +9293,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9053,7 +9304,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -9094,7 +9345,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9104,10 +9355,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -9129,32 +9385,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9247,7 +9504,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9258,7 +9515,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -9305,7 +9562,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9315,10 +9572,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -9340,32 +9602,33 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9446,7 +9709,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9457,7 +9720,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // With RIGHT ANTI join, we are only interested in unmatched tuples @@ -9500,7 +9763,7 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9510,10 +9773,15 @@ func (o *mergeJoinRightAntiOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go index b5661129f242..bd6ae5db6595 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,32 +77,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -200,7 +202,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -211,7 +213,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -261,7 +263,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -271,10 +273,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -296,32 +303,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -396,7 +404,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -407,7 +415,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -449,7 +457,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -459,10 +467,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -484,32 +497,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -584,7 +598,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -595,7 +609,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -637,7 +651,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -647,10 +661,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -671,32 +690,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -804,7 +824,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -815,7 +835,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -868,7 +888,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -878,10 +898,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -899,32 +924,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1032,7 +1058,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1043,7 +1069,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1096,7 +1122,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1106,10 +1132,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -1128,32 +1159,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1261,7 +1293,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1272,7 +1304,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1325,7 +1357,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1335,10 +1367,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1360,32 +1397,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1517,7 +1555,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1528,7 +1566,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1589,7 +1627,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1599,10 +1637,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1624,32 +1667,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1745,7 +1789,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1756,7 +1800,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1805,7 +1849,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -1815,10 +1859,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1840,32 +1889,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1940,7 +1990,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1951,7 +2001,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -1993,7 +2043,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2003,10 +2053,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -2028,32 +2083,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2146,7 +2202,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2157,7 +2213,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2205,7 +2261,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2215,10 +2271,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -2240,32 +2301,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2346,7 +2408,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2357,7 +2419,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2401,7 +2463,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2411,10 +2473,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -2435,6 +2502,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -2457,32 +2525,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2581,7 +2650,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2592,7 +2661,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2642,7 +2711,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2652,10 +2721,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2677,32 +2751,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2777,7 +2852,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2788,7 +2863,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -2830,7 +2905,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -2840,10 +2915,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2865,32 +2945,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2965,7 +3046,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2976,7 +3057,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3018,7 +3099,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3028,10 +3109,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -3052,32 +3138,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3185,7 +3272,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3196,7 +3283,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3249,7 +3336,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3259,10 +3346,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -3280,32 +3372,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3413,7 +3506,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3424,7 +3517,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3477,7 +3570,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3487,10 +3580,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -3509,32 +3607,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3642,7 +3741,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3653,7 +3752,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3706,7 +3805,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3716,10 +3815,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -3741,32 +3845,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3898,7 +4003,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3909,7 +4014,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -3970,7 +4075,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -3980,10 +4085,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4005,32 +4115,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4126,7 +4237,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4137,7 +4248,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4186,7 +4297,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4196,10 +4307,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -4221,32 +4337,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4321,7 +4438,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4332,7 +4449,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4374,7 +4491,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4384,10 +4501,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -4409,32 +4531,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4527,7 +4650,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4538,7 +4661,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4586,7 +4709,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4596,10 +4719,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -4621,32 +4749,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4727,7 +4856,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4738,7 +4867,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -4782,7 +4911,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -4792,10 +4921,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSeltrueRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -4816,6 +4950,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -4838,32 +4973,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4962,7 +5098,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4973,7 +5109,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5023,7 +5159,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5033,10 +5169,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5058,32 +5199,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5158,7 +5300,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5169,7 +5311,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5211,7 +5353,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5221,10 +5363,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5246,32 +5393,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5346,7 +5494,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5357,7 +5505,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5399,7 +5547,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5409,10 +5557,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5433,32 +5586,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5566,7 +5720,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5577,7 +5731,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5630,7 +5784,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5640,10 +5794,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -5661,32 +5820,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5794,7 +5954,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5805,7 +5965,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -5858,7 +6018,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -5868,10 +6028,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -5890,32 +6055,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6023,7 +6189,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6034,7 +6200,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6087,7 +6253,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6097,10 +6263,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6122,32 +6293,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6279,7 +6451,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6290,7 +6462,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6351,7 +6523,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6361,10 +6533,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6386,32 +6563,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6507,7 +6685,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6518,7 +6696,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6567,7 +6745,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6577,10 +6755,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6602,32 +6785,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6702,7 +6886,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6713,7 +6897,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6755,7 +6939,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6765,10 +6949,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -6790,32 +6979,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6908,7 +7098,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6919,7 +7109,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -6967,7 +7157,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -6977,10 +7167,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7002,32 +7197,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7108,7 +7304,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7119,7 +7315,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7163,7 +7359,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7173,10 +7369,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSeltrue() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -7197,6 +7398,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -7219,32 +7421,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7343,7 +7546,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7354,7 +7557,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7404,7 +7607,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7414,10 +7617,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -7439,32 +7647,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7539,7 +7748,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7550,7 +7759,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7592,7 +7801,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7602,10 +7811,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -7627,32 +7841,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7727,7 +7942,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7738,7 +7953,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -7780,7 +7995,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -7790,10 +8005,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -7814,32 +8034,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7947,7 +8168,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7958,7 +8179,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8011,7 +8232,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8021,10 +8242,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -8042,32 +8268,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8175,7 +8402,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8186,7 +8413,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8239,7 +8466,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8249,10 +8476,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -8271,32 +8503,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8404,7 +8637,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8415,7 +8648,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8468,7 +8701,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8478,10 +8711,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -8503,32 +8741,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8660,7 +8899,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8671,7 +8910,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8732,7 +8971,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8742,10 +8981,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -8767,32 +9011,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -8888,7 +9133,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -8899,7 +9144,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -8948,7 +9193,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -8958,10 +9203,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -8983,32 +9233,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9083,7 +9334,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9094,7 +9345,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9136,7 +9387,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9146,10 +9397,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -9171,32 +9427,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9289,7 +9546,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9300,7 +9557,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9348,7 +9605,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9358,10 +9615,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -9383,32 +9645,33 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -9489,7 +9752,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -9500,7 +9763,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // Neither group ends with the batch, so add the group to the @@ -9544,7 +9807,7 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { } } } - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -9554,10 +9817,15 @@ func (o *mergeJoinRightOuterOp) probeBodyLSelfalseRSelfalse() { curRIdx++ } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go index 6eca76b2c3cc..d2cd18ca9289 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go @@ -54,6 +54,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -76,21 +77,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -189,7 +192,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -200,7 +203,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -218,10 +221,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -243,21 +251,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -332,7 +342,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -343,7 +353,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -361,10 +371,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -386,21 +401,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -475,7 +492,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -486,7 +503,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -504,10 +521,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -528,21 +550,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -650,7 +674,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -661,7 +685,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -679,10 +703,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -700,21 +729,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -822,7 +853,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -833,7 +864,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -851,10 +882,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -873,21 +909,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -995,7 +1033,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1006,7 +1044,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1024,10 +1062,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -1049,21 +1092,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1195,7 +1240,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1206,7 +1251,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1224,10 +1269,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -1249,21 +1299,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1359,7 +1411,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1370,7 +1422,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1388,10 +1440,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -1413,21 +1470,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1502,7 +1561,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1513,7 +1572,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1531,10 +1590,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -1556,21 +1620,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1663,7 +1729,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1674,7 +1740,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1692,10 +1758,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -1717,21 +1788,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -1812,7 +1885,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -1823,7 +1896,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -1841,10 +1914,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -1865,6 +1943,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -1887,21 +1966,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2000,7 +2081,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2011,7 +2092,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2029,10 +2110,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -2054,21 +2140,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2143,7 +2231,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2154,7 +2242,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2172,10 +2260,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -2197,21 +2290,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2286,7 +2381,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2297,7 +2392,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2315,10 +2410,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -2339,21 +2439,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2461,7 +2563,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2472,7 +2574,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2490,10 +2592,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -2511,21 +2618,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2633,7 +2742,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2644,7 +2753,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2662,10 +2771,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -2684,21 +2798,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -2806,7 +2922,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -2817,7 +2933,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -2835,10 +2951,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -2860,21 +2981,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3006,7 +3129,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3017,7 +3140,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3035,10 +3158,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -3060,21 +3188,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3170,7 +3300,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3181,7 +3311,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3199,10 +3329,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -3224,21 +3359,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3313,7 +3450,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3324,7 +3461,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3342,10 +3479,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -3367,21 +3509,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3474,7 +3618,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3485,7 +3629,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3503,10 +3647,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -3528,21 +3677,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(lSel[curLIdx]) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3623,7 +3774,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3634,7 +3785,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3652,10 +3803,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSeltrueRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -3676,6 +3832,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -3698,21 +3855,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3811,7 +3970,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3822,7 +3981,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3840,10 +3999,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -3865,21 +4029,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -3954,7 +4120,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -3965,7 +4131,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -3983,10 +4149,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -4008,21 +4179,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4097,7 +4270,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4108,7 +4281,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -4126,10 +4299,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -4150,21 +4328,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4272,7 +4452,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4283,7 +4463,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -4301,10 +4481,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -4322,21 +4507,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4444,7 +4631,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4455,7 +4642,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -4473,10 +4660,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -4495,21 +4687,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4617,7 +4811,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4628,7 +4822,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -4646,10 +4840,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -4671,21 +4870,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4817,7 +5018,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4828,7 +5029,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -4846,10 +5047,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -4871,21 +5077,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -4981,7 +5189,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -4992,7 +5200,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5010,10 +5218,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -5035,21 +5248,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5124,7 +5339,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5135,7 +5350,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5153,10 +5368,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -5178,21 +5398,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5285,7 +5507,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5296,7 +5518,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5314,10 +5536,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -5339,21 +5566,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(rSel[curRIdx]) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5434,7 +5663,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5445,7 +5674,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5463,10 +5692,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSeltrue() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: @@ -5487,6 +5721,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 lNulls := lVec.Nulls() rNulls := rVec.Nulls() switch lVec.CanonicalTypeFamily() { @@ -5509,21 +5744,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5622,7 +5859,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5633,7 +5870,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5651,10 +5888,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.BytesFamily: @@ -5676,21 +5918,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5765,7 +6009,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5776,7 +6020,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5794,10 +6038,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.DecimalFamily: @@ -5819,21 +6068,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -5908,7 +6159,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -5919,7 +6170,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -5937,10 +6188,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntFamily: @@ -5961,21 +6217,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6083,7 +6341,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6094,7 +6352,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6112,10 +6370,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case 32: lKeys := lVec.Int32() @@ -6133,21 +6396,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6255,7 +6520,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6266,7 +6531,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6284,10 +6549,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } case -1: default: @@ -6306,21 +6576,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6428,7 +6700,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6439,7 +6711,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6457,10 +6729,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.FloatFamily: @@ -6482,21 +6759,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6628,7 +6907,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6639,7 +6918,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6657,10 +6936,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.TimestampTZFamily: @@ -6682,21 +6966,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6792,7 +7078,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6803,7 +7089,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6821,10 +7107,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.IntervalFamily: @@ -6846,21 +7137,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -6935,7 +7228,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -6946,7 +7239,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -6964,10 +7257,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case types.JsonFamily: @@ -6989,21 +7287,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7096,7 +7396,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7107,7 +7407,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -7125,10 +7425,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } case typeconv.DatumVecCanonicalTypeFamily: @@ -7150,21 +7455,23 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(curLIdx) rNull := rNulls.NullAt(curRIdx) - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } @@ -7245,7 +7552,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -7256,7 +7563,7 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { o.groups.addRightSemiGroup(beginRIdx, rGroupLength) @@ -7274,10 +7581,15 @@ func (o *mergeJoinRightSemiOp) probeBodyLSelfalseRSelfalse() { } } } - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } } default: diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go index 620994a13745..ff29ce6837f9 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go @@ -129,11 +129,10 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / curRIdx := rGroup.rowStartIdx curLEndIdx := lGroup.rowEndIdx curREndIdx := rGroup.rowEndIdx - areGroupsProcessed := false _LEFT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE) _RIGHT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE) // Expand or filter each group based on the current equality column. - for curLIdx < curLEndIdx && curRIdx < curREndIdx && !areGroupsProcessed { + for curLIdx < curLEndIdx && curRIdx < curREndIdx { cmp = 0 lNull := lNulls.NullAt(_L_SEL_IND) rNull := rNulls.NullAt(_R_SEL_IND) @@ -163,16 +162,19 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / // so if either value is NULL, the tuples are not // matches. // */}} - // TODO(yuzefovich): we can advance both sides if both are - // NULL. + curLIdxInc := 0 if lNull { _NULL_FROM_LEFT_SWITCH(_JOIN_TYPE) - curLIdx++ - continue + curLIdxInc = 1 } + curRIdxInc := 0 if rNull { _NULL_FROM_RIGHT_SWITCH(_JOIN_TYPE) - curRIdx++ + curRIdxInc = 1 + } + if lNull || rNull { + curLIdx += curLIdxInc + curRIdx += curRIdxInc continue } // {{end}} @@ -268,7 +270,7 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / } // Last equality column and either group is incomplete. - if eqColIdx == len(o.left.eqCols)-1 && (!lComplete || !rComplete) { + if lastEqCol && (!lComplete || !rComplete) { // Store the state about the buffered group. o.startLeftBufferedGroup(lSel, beginLIdx, lGroupLength) o.bufferedGroup.leftGroupStartIdx = beginLIdx @@ -279,7 +281,7 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / return } - if eqColIdx < len(o.left.eqCols)-1 { + if !lastEqCol { o.groups.addGroupsToNextCol(beginLIdx, lGroupLength, beginRIdx, rGroupLength) } else { // {{if _JOIN_TYPE.IsLeftSemi}} @@ -331,10 +333,15 @@ func _PROBE_SWITCH(_JOIN_TYPE joinTypeInfo, _SEL_PERMUTATION selPermutation) { / } } _PROCESS_NOT_LAST_GROUP_IN_COLUMN_SWITCH(_JOIN_TYPE) - // Both o.proberState.lIdx and o.proberState.rIdx should point to the - // last elements processed in their respective batches. - o.proberState.lIdx = curLIdx - o.proberState.rIdx = curRIdx + // Both o.proberState.lIdx and o.proberState.rIdx should point + // to the last tuples that have been fully processed in their + // respective batches. This is the case when we've just finished + // the last equality column or the current column is such that + // all tuples were filtered out. + if lastEqCol || !o.groups.hasGroupForNextCol() { + o.proberState.lIdx = curLIdx + o.proberState.rIdx = curRIdx + } } // {{end}} } @@ -361,14 +368,13 @@ func _LEFT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{end}} // {{if or $.JoinType.IsLeftOuter $.JoinType.IsLeftAnti}} if lGroup.unmatched { - if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addLeftUnmatchedGroup(curLIdx, curRIdx) - curLIdx++ - areGroupsProcessed = true + if lastEqCol && curLIdx >= o.proberState.lIdx { + o.proberState.lIdx = curLIdx + 1 + } + continue } // {{end}} // {{if or $.JoinType.IsRightOuter $.JoinType.IsRightAnti}} @@ -403,14 +409,13 @@ func _RIGHT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{end}} // {{if or $.JoinType.IsRightOuter $.JoinType.IsRightAnti}} if rGroup.unmatched { - if curRIdx+1 != curREndIdx { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) - } // The row already does not have a match, so we don't need to do any // additional processing. o.groups.addRightUnmatchedGroup(curLIdx, curRIdx) - curRIdx++ - areGroupsProcessed = true + if lastEqCol && curRIdx >= o.proberState.rIdx { + o.proberState.rIdx = curRIdx + 1 + } + continue } // {{end}} // {{end}} @@ -605,7 +610,7 @@ func _PROCESS_NOT_LAST_GROUP_IN_COLUMN_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // */}} // {{end}} // {{if or $.JoinType.IsLeftOuter $.JoinType.IsLeftAnti}} - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the left group will not get a match, so each one of @@ -617,7 +622,7 @@ func _PROCESS_NOT_LAST_GROUP_IN_COLUMN_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} } // {{end}} // {{if or $.JoinType.IsRightOuter $.JoinType.IsRightAnti}} - if !o.groups.isLastGroupInCol() && !areGroupsProcessed { + if !o.groups.isLastGroupInCol() { // The current group is not the last one within the column, so it cannot be // extended into the next batch, and we need to process it right now. Any // unprocessed row in the right group will not get a match, so each one of @@ -644,6 +649,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) probeBodyLSel_IS_L_SELRSel_IS_R_SEL() { lVec := o.proberState.lBatch.ColVec(int(leftColIdx)) rVec := o.proberState.rBatch.ColVec(int(rightColIdx)) colType := o.left.sourceTypes[leftColIdx] + lastEqCol := eqColIdx == len(o.left.eqCols)-1 _PROBE_SWITCH(_JOIN_TYPE, _SEL_ARG) // Look at the groups associated with the next equality column by moving // the circular buffer pointer up. diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_util.go b/pkg/sql/colexec/colexecjoin/mergejoiner_util.go index 74df84023034..10dae4754e7f 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_util.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_util.go @@ -148,6 +148,12 @@ func (b *circularGroupsBuffer) isLastGroupInCol() bool { return b.startIdx == b.nextColStartIdx } +// hasGroupForNextCol returns true if at least one group has already been added +// for the next column. +func (b *circularGroupsBuffer) hasGroupForNextCol() bool { + return b.nextColStartIdx != b.endIdx +} + // ensureCapacityForNewGroup makes sure that groups slices have enough space to // add another group to the buffer, reallocating the slices if necessary. func (b *circularGroupsBuffer) ensureCapacityForNewGroup() { diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 81f7c1c3c940..40780fa33c93 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -1639,6 +1639,19 @@ func getMJTestCases() []*joinTestCase { rightDirections: []execinfrapb.Ordering_Column_Direction{execinfrapb.Ordering_Column_DESC}, expected: colexectestutils.Tuples{{0, 4, nil, nil}, {nil, nil, 0, 1}, {nil, 0, nil, 0}, {3, 0, nil, 0}, {nil, nil, 4, nil}}, }, + { + description: "RIGHT OUTER join with a single match in the middle", + joinType: descpb.RightOuterJoin, + leftTypes: []*types.T{types.Int, types.Int}, + rightTypes: []*types.T{types.Int, types.Int}, + leftTuples: colexectestutils.Tuples{{nil, 4}, {nil, 4}, {2, nil}, {2, nil}, {2, 4}, {3, 0}}, + rightTuples: colexectestutils.Tuples{{nil, 1}, {1, nil}, {1, 0}, {2, 4}, {3, nil}, {3, 2}}, + leftOutCols: []uint32{0, 1}, + rightOutCols: []uint32{0, 1}, + leftEqCols: []uint32{0, 1}, + rightEqCols: []uint32{0, 1}, + expected: colexectestutils.Tuples{{nil, nil, nil, 1}, {nil, nil, 1, nil}, {nil, nil, 1, 0}, {2, 4, 2, 4}, {nil, nil, 3, nil}, {nil, nil, 3, 2}}, + }, } return withMirrors(mjTestCases) } From 74a0e8e52d5aad1de1ed07e8a4290c08326679b9 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 16 Aug 2021 20:53:44 -0700 Subject: [PATCH 3/4] colexecjoin: avoid buffering tuples from the right in merge joiner Depending on the join type, we don't need to fully buffer the tuples from the right input in order to produce the output. Namely, for set-operation joins we only need to know the number of right tuples whereas for LEFT SEMI and RIGHT ANTI we know exactly the behavior of the builder for the buffered group. Release note: None --- pkg/sql/colexec/colexecjoin/mergejoiner.go | 23 ++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner.go b/pkg/sql/colexec/colexecjoin/mergejoiner.go index b8593dcd39b6..0abb41d31ac4 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner.go @@ -665,14 +665,6 @@ func (o *mergeJoinBase) appendToRightBufferedGroup(sel []int, groupStartIdx int, sourceTypes := o.right.sourceTypes numBufferedTuples := o.bufferedGroup.helper.numRightTuples o.bufferedGroup.helper.numRightTuples += groupLength - // TODO(yuzefovich): for LEFT/RIGHT ANTI joins we only need to store the - // first tuple (in order to find the boundaries of the groups) since all - // of the buffered tuples do have a match and, thus, don't contribute to - // the output. - // TODO(yuzefovich): for INTERSECT/EXCEPT ALL joins we can buffer only - // tuples from the left side and count the number of tuples on the right. - // TODO(yuzefovich): for LEFT/RIGHT SEMI joins we only need to buffer tuples - // from one side (left/right respectively). if numBufferedTuples == 0 && groupStartIdx+groupLength == o.proberState.rLength { // Set the right first tuple only if this is the first call to this // method for the current right buffered group and if the group doesn't @@ -692,6 +684,21 @@ func (o *mergeJoinBase) appendToRightBufferedGroup(sel []int, groupStartIdx int, }) } + // TODO(yuzefovich): check whether it's worth templating this method out as + // well as having join-type-specific crossJoinerBase. + switch o.joinType { + case descpb.LeftSemiJoin, descpb.RightAntiJoin: + // For LEFT SEMI and RIGHT ANTI joins we only need to store the first + // tuple (in order to find the boundaries of the groups) since all of + // the buffered tuples don't/do have a match and, thus, do/don't + // contribute to the output. + return + case descpb.IntersectAllJoin, descpb.ExceptAllJoin: + // For INTERSECT/EXCEPT ALL joins we only need the number of tuples on + // the right side (which we have already updated above). + return + } + // We don't impose any memory limits on the scratch batch because we rely on // the inputs to the merge joiner to produce reasonably sized batches. const maxBatchMemSize = math.MaxInt64 From 4999ac5c2edee2f94a91dbe338c87aa190be3cc4 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 16 Aug 2021 21:12:53 -0700 Subject: [PATCH 4/4] colexecjoin: remove a copy when buffering the right group Previously, before enqueueing the tuples from the right buffered group into the spiling queue we would perform a deep-copy. This is an overkill because the spilling queue itself performs the deep copy. This commit refactors the enqueueing code to modify the right batch directly to include only the tuples from the group. Release note: None --- pkg/sql/colexec/colexecjoin/mergejoiner.go | 64 +++++++++++++--------- 1 file changed, 39 insertions(+), 25 deletions(-) diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner.go b/pkg/sql/colexec/colexecjoin/mergejoiner.go index 0abb41d31ac4..07f8af436e49 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner.go @@ -12,7 +12,6 @@ package colexecjoin import ( "context" - "math" "unsafe" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -248,9 +247,8 @@ type mjBufferedGroupState struct { // rightFirstTuple is the first tuple of the right buffered group. It is set // only in case the right buffered group spans more than one input batch. rightFirstTuple []coldata.Vec - // rightScratchBatch is a scratch space for copying the tuples out of the - // right input batches before enqueueing them into the spilling queue. - rightScratchBatch coldata.Batch + // scratchSel is a scratch selection vector initialized only when needed. + scratchSel []int // helper is the building facility for the cross join of the buffered group. helper *crossJoinerBase @@ -699,29 +697,45 @@ func (o *mergeJoinBase) appendToRightBufferedGroup(sel []int, groupStartIdx int, return } - // We don't impose any memory limits on the scratch batch because we rely on - // the inputs to the merge joiner to produce reasonably sized batches. - const maxBatchMemSize = math.MaxInt64 - o.bufferedGroup.rightScratchBatch, _ = o.unlimitedAllocator.ResetMaybeReallocate( - sourceTypes, o.bufferedGroup.rightScratchBatch, groupLength, maxBatchMemSize, - ) - // TODO(yuzefovich): SpillingQueue.Enqueue deep-copies the batch too. Think - // through whether the copy here can be avoided altogether. - o.unlimitedAllocator.PerformOperation(o.bufferedGroup.rightScratchBatch.ColVecs(), func() { - for colIdx := range sourceTypes { - o.bufferedGroup.rightScratchBatch.ColVec(colIdx).Copy( - coldata.SliceArgs{ - Src: o.proberState.rBatch.ColVec(colIdx), - Sel: sel, - DestIdx: 0, - SrcStartIdx: groupStartIdx, - SrcEndIdx: groupStartIdx + groupLength, - }, + // Update the selection on the probing batch to only include tuples from the + // buffered group. + rBatch, rLength := o.proberState.rBatch, o.proberState.rLength + rSel := rBatch.Selection() + rBatchHasSel := rSel != nil + // No need to modify the batch if the whole batch is part of the buffered + // group. + needToModify := groupStartIdx != 0 || groupLength != rLength + if needToModify { + if rBatchHasSel { + // Since rBatch already has a selection vector which we'll be + // modifying, we need to copy the original. + o.bufferedGroup.scratchSel = colexecutils.EnsureSelectionVectorLength(o.bufferedGroup.scratchSel, rLength) + copy(o.bufferedGroup.scratchSel, rSel) + // Now we need to shift elements in range + // [groupStartIdx; groupStartIdx+groupLength) to the beginning of + // the selection vector and then update the length of the batch + // accordingly. + copy(rSel[:groupLength], rSel[groupStartIdx:groupStartIdx+groupLength]) + rBatch.SetLength(groupLength) + } else { + // Since rBatch doesn't have a selection vector, we will set the + // selection vector to include tuples in range + // [groupStartIdx; groupStartIdx+groupLength). + colexecutils.UpdateBatchState( + rBatch, groupLength, true, /* usesSel */ + colexecutils.DefaultSelectionVector[groupStartIdx:groupStartIdx+groupLength], ) } - o.bufferedGroup.rightScratchBatch.SetLength(groupLength) - }) - bufferedTuples.Enqueue(o.Ctx, o.bufferedGroup.rightScratchBatch) + } + + bufferedTuples.Enqueue(o.Ctx, rBatch) + + // If we had to modify the batch, then restore the original state now. + if needToModify { + colexecutils.UpdateBatchState( + rBatch, rLength, rBatchHasSel, o.bufferedGroup.scratchSel, + ) + } } // sourceFinished returns true if either of input sources has no more rows.