From d82affec8be142708cc4ffde19d144be36caee80 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 18 Mar 2022 12:35:29 -0700 Subject: [PATCH] colexecjoin: optimize building output on the left in merge joiner MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit updates the way we're building output in the merge joiner from the left input when building directly from the left batch (i.e. not from the buffered group). There, we need to repeat a single tuple `toAppend` times, so we do it in a loop. This commit adds the optimization of using `Bytes.Copy` for the bytes-like types as well as BCE for sliceable types. ``` MergeJoiner/rows=32-24 29.3MB/s ± 3% 29.5MB/s ± 3% ~ (p=0.684 n=10+10) MergeJoiner/rows=512-24 79.4MB/s ± 2% 77.8MB/s ± 3% -1.91% (p=0.043 n=10+10) MergeJoiner/rows=4096-24 192MB/s ± 2% 189MB/s ± 1% -1.36% (p=0.029 n=10+10) MergeJoiner/rows=32768-24 278MB/s ± 1% 275MB/s ± 0% -1.30% (p=0.000 n=10+10) MergeJoiner/oneSideRepeat-rows=32-24 37.3MB/s ± 3% 38.0MB/s ± 2% +1.78% (p=0.029 n=10+10) MergeJoiner/oneSideRepeat-rows=512-24 212MB/s ± 1% 215MB/s ± 2% +1.42% (p=0.003 n=9+10) MergeJoiner/oneSideRepeat-rows=4096-24 765MB/s ± 4% 770MB/s ± 3% ~ (p=0.436 n=10+10) MergeJoiner/oneSideRepeat-rows=32768-24 1.22GB/s ± 2% 1.23GB/s ± 2% ~ (p=0.393 n=10+10) MergeJoiner/bothSidesRepeat-rows=32-24 22.7MB/s ± 2% 22.9MB/s ± 2% ~ (p=0.203 n=9+10) MergeJoiner/bothSidesRepeat-rows=512-24 102MB/s ± 4% 104MB/s ± 2% +2.38% (p=0.011 n=10+10) MergeJoiner/bothSidesRepeat-rows=4096-24 117MB/s ± 1% 127MB/s ± 1% +9.11% (p=0.000 n=10+9) MergeJoiner/bothSidesRepeat-rows=32768-24 59.2MB/s ± 1% 67.1MB/s ± 1% +13.48% (p=0.000 n=10+10) ``` Release note: None --- pkg/col/coldata/bytes.go | 4 +- pkg/col/coldata/bytes_test.go | 2 +- pkg/col/coldata/json.go | 6 +- pkg/col/coldata/vec.eg.go | 8 +- pkg/col/coldata/vec_tmpl.go | 6 +- .../colexecjoin/mergejoiner_exceptall.eg.go | 541 ++++++++++------- .../colexecjoin/mergejoiner_fullouter.eg.go | 563 +++++++++++------- .../colexecjoin/mergejoiner_inner.eg.go | 541 ++++++++++------- .../mergejoiner_intersectall.eg.go | 541 ++++++++++------- .../colexecjoin/mergejoiner_leftanti.eg.go | 541 ++++++++++------- .../colexecjoin/mergejoiner_leftouter.eg.go | 541 ++++++++++------- .../colexecjoin/mergejoiner_leftsemi.eg.go | 541 ++++++++++------- .../colexecjoin/mergejoiner_rightanti.eg.go | 563 +++++++++++------- .../colexecjoin/mergejoiner_rightouter.eg.go | 563 +++++++++++------- .../colexecjoin/mergejoiner_rightsemi.eg.go | 541 ++++++++++------- .../colexec/colexecjoin/mergejoiner_tmpl.go | 60 +- 16 files changed, 3382 insertions(+), 2180 deletions(-) diff --git a/pkg/col/coldata/bytes.go b/pkg/col/coldata/bytes.go index 5b97e358af21..1f1d523c01ea 100644 --- a/pkg/col/coldata/bytes.go +++ b/pkg/col/coldata/bytes.go @@ -211,9 +211,9 @@ func (b *Bytes) Window(start, end int) *Bytes { } } -// copy copies a single value from src at position srcIdx into position destIdx +// Copy copies a single value from src at position srcIdx into position destIdx // of the receiver. It is faster than b.Set(destIdx, src.Get(srcIdx)). -func (b *Bytes) copy(src *Bytes, destIdx, srcIdx int) { +func (b *Bytes) Copy(src *Bytes, destIdx, srcIdx int) { if buildutil.CrdbTestBuild { if b.isWindow { panic("copy is called on a window into Bytes") diff --git a/pkg/col/coldata/bytes_test.go b/pkg/col/coldata/bytes_test.go index 0d904aefa59f..fe265b136e09 100644 --- a/pkg/col/coldata/bytes_test.go +++ b/pkg/col/coldata/bytes_test.go @@ -122,7 +122,7 @@ func applyMethodsAndVerify( destIdx := rng.Intn(n) srcIdx := rng.Intn(sourceN) debugString += fmt.Sprintf("(%d, %d)", destIdx, srcIdx) - b1.copy(b1Source, destIdx, srcIdx) + b1.Copy(b1Source, destIdx, srcIdx) b2[destIdx] = append([]byte(nil), b2Source[srcIdx]...) case copySlice, appendSlice: // Generate a length-inclusive destIdx. diff --git a/pkg/col/coldata/json.go b/pkg/col/coldata/json.go index a7b2f42c4aaa..c62ed767b142 100644 --- a/pkg/col/coldata/json.go +++ b/pkg/col/coldata/json.go @@ -70,10 +70,10 @@ func (js *JSONs) Window(start, end int) *JSONs { } } -// copy copies a single value from src at position srcIdx into position destIdx +// Copy copies a single value from src at position srcIdx into position destIdx // of the receiver. -func (js *JSONs) copy(src *JSONs, destIdx, srcIdx int) { - js.Bytes.copy(&src.Bytes, destIdx, srcIdx) +func (js *JSONs) Copy(src *JSONs, destIdx, srcIdx int) { + js.Bytes.Copy(&src.Bytes, destIdx, srcIdx) } // CopySlice copies srcStartIdx inclusive and srcEndIdx exclusive []byte values diff --git a/pkg/col/coldata/vec.eg.go b/pkg/col/coldata/vec.eg.go index 2c9c33b120ee..dd9a03071c4b 100644 --- a/pkg/col/coldata/vec.eg.go +++ b/pkg/col/coldata/vec.eg.go @@ -544,7 +544,7 @@ func (m *memColumn) Copy(args SliceArgs) { if nulls.NullAt(selIdx) { m.nulls.SetNull(i + args.DestIdx) } else { - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) } } return @@ -553,7 +553,7 @@ func (m *memColumn) Copy(args SliceArgs) { for i := 0; i < n; i++ { //gcassert:bce selIdx := sel[i] - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) } return } @@ -850,7 +850,7 @@ func (m *memColumn) Copy(args SliceArgs) { if nulls.NullAt(selIdx) { m.nulls.SetNull(i + args.DestIdx) } else { - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) } } return @@ -859,7 +859,7 @@ func (m *memColumn) Copy(args SliceArgs) { for i := 0; i < n; i++ { //gcassert:bce selIdx := sel[i] - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) } return } diff --git a/pkg/col/coldata/vec_tmpl.go b/pkg/col/coldata/vec_tmpl.go index dd460a9c6d88..5c88d4b7e3f3 100644 --- a/pkg/col/coldata/vec_tmpl.go +++ b/pkg/col/coldata/vec_tmpl.go @@ -210,7 +210,7 @@ func (m *memColumn) Copy(args SliceArgs) { m.nulls.SetNull(i + args.DestIdx) } else { // {{if .IsBytesLike}} - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) // {{else}} v := fromCol.Get(selIdx) // {{if .Sliceable}} @@ -238,7 +238,7 @@ func (m *memColumn) Copy(args SliceArgs) { //gcassert:bce selIdx := sel[i] // {{if .IsBytesLike}} - toCol.copy(fromCol, i+args.DestIdx, selIdx) + toCol.Copy(fromCol, i+args.DestIdx, selIdx) // {{else}} v := fromCol.Get(selIdx) // {{if .Sliceable}} @@ -285,7 +285,7 @@ func _COPY_WITH_REORDERED_SOURCE(_SRC_HAS_NULLS bool) { // */}} // {{end}} { // {{if .IsBytesLike}} - toCol.copy(fromCol, destIdx, srcIdx) + toCol.Copy(fromCol, destIdx, srcIdx) // {{else}} v := fromCol.Get(srcIdx) toCol.Set(destIdx, v) diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go index 2f8792c93372..1cf9e4098498 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_exceptall.eg.go @@ -10925,6 +10925,7 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -10947,8 +10948,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10964,33 +10963,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11012,8 +11019,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11029,33 +11034,37 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11077,8 +11086,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11094,33 +11101,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11141,8 +11156,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11158,33 +11171,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11202,8 +11223,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11219,33 +11238,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11264,8 +11291,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11281,33 +11306,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11329,8 +11362,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11346,33 +11377,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11394,8 +11433,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11411,33 +11448,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11459,8 +11504,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11476,33 +11519,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11524,8 +11575,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11541,33 +11590,37 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11589,8 +11642,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11606,33 +11657,38 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11664,8 +11720,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11681,32 +11735,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11728,8 +11791,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11745,32 +11806,37 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11792,8 +11858,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11809,32 +11873,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11855,8 +11928,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11872,32 +11943,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11915,8 +11995,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11932,32 +12010,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11976,8 +12063,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11993,32 +12078,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12040,8 +12134,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12057,32 +12149,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12104,8 +12205,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12121,32 +12220,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12168,8 +12276,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12185,32 +12291,41 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12232,8 +12347,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12249,32 +12362,37 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12296,8 +12414,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12313,32 +12429,38 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12392,6 +12514,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -12458,7 +12581,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12525,7 +12648,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12592,7 +12715,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12658,7 +12781,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12721,7 +12844,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12785,7 +12908,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12852,7 +12975,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12919,7 +13042,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12986,7 +13109,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13053,7 +13176,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13120,7 +13243,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13197,7 +13320,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13264,7 +13387,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13331,7 +13454,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13397,7 +13520,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13460,7 +13583,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13524,7 +13647,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13591,7 +13714,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13658,7 +13781,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13725,7 +13848,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13792,7 +13915,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13859,7 +13982,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go index 163164a24f04..acd7c0d46884 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_fullouter.eg.go @@ -12061,6 +12061,7 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -12083,8 +12084,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12097,36 +12096,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12148,8 +12154,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12162,36 +12166,39 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12213,8 +12220,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12227,36 +12232,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12277,8 +12289,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12291,36 +12301,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12338,8 +12355,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12352,36 +12367,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12400,8 +12422,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12414,36 +12434,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12465,8 +12492,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12479,36 +12504,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12530,8 +12562,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12544,36 +12574,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12595,8 +12632,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12609,36 +12644,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12660,8 +12702,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12674,36 +12714,39 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12725,8 +12768,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12739,36 +12780,40 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12800,8 +12845,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12814,35 +12857,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12864,8 +12915,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12878,35 +12927,39 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12928,8 +12981,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -12942,35 +12993,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -12991,8 +13050,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13005,35 +13062,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13051,8 +13116,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13065,35 +13128,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13112,8 +13183,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13126,35 +13195,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13176,8 +13253,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13190,35 +13265,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13240,8 +13323,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13254,35 +13335,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13304,8 +13393,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13318,35 +13405,43 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13368,8 +13463,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13382,35 +13475,39 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13432,8 +13529,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -13446,35 +13541,40 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -13528,6 +13628,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -13596,7 +13697,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13665,7 +13766,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13734,7 +13835,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13802,7 +13903,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13867,7 +13968,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -13933,7 +14034,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14002,7 +14103,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14071,7 +14172,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14140,7 +14241,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14209,7 +14310,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14278,7 +14379,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14357,7 +14458,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14426,7 +14527,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14495,7 +14596,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14563,7 +14664,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14628,7 +14729,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14694,7 +14795,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14763,7 +14864,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14832,7 +14933,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14901,7 +15002,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -14970,7 +15071,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -15039,7 +15140,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go index dd29c473db22..04610f220618 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_inner.eg.go @@ -7721,6 +7721,7 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -7743,8 +7744,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7757,33 +7756,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7805,8 +7812,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7819,33 +7824,37 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7867,8 +7876,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7881,33 +7888,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7928,8 +7943,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7942,33 +7955,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7986,8 +8007,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8000,33 +8019,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8045,8 +8072,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8059,33 +8084,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8107,8 +8140,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8121,33 +8152,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8169,8 +8208,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8183,33 +8220,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8231,8 +8276,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8245,33 +8288,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8293,8 +8344,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8307,33 +8356,37 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8355,8 +8408,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8369,33 +8420,38 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8427,8 +8483,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8441,32 +8495,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8488,8 +8551,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8502,32 +8563,37 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8549,8 +8615,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8563,32 +8627,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8609,8 +8682,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8623,32 +8694,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8666,8 +8746,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8680,32 +8758,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8724,8 +8811,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8738,32 +8823,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8785,8 +8879,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8799,32 +8891,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8846,8 +8947,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8860,32 +8959,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8907,8 +9015,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8921,32 +9027,41 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8968,8 +9083,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8982,32 +9095,37 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9029,8 +9147,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9043,32 +9159,38 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9122,6 +9244,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -9188,7 +9311,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9255,7 +9378,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9322,7 +9445,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9388,7 +9511,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9451,7 +9574,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9515,7 +9638,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9582,7 +9705,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9649,7 +9772,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9716,7 +9839,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9783,7 +9906,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9850,7 +9973,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9927,7 +10050,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9994,7 +10117,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10061,7 +10184,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10127,7 +10250,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10190,7 +10313,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10254,7 +10377,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10321,7 +10444,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10388,7 +10511,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10455,7 +10578,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10522,7 +10645,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10589,7 +10712,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go index 6392b9ee2121..99e21ae44e5a 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_intersectall.eg.go @@ -8425,6 +8425,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -8447,8 +8448,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8461,33 +8460,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8509,8 +8516,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8523,33 +8528,37 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8571,8 +8580,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8585,33 +8592,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8632,8 +8647,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8646,33 +8659,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8690,8 +8711,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8704,33 +8723,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8749,8 +8776,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8763,33 +8788,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8811,8 +8844,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8825,33 +8856,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8873,8 +8912,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8887,33 +8924,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8935,8 +8980,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8949,33 +8992,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8997,8 +9048,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9011,33 +9060,37 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9059,8 +9112,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9073,33 +9124,38 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9131,8 +9187,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9145,32 +9199,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9192,8 +9255,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9206,32 +9267,37 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9253,8 +9319,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9267,32 +9331,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9313,8 +9386,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9327,32 +9398,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9370,8 +9450,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9384,32 +9462,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9428,8 +9515,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9442,32 +9527,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9489,8 +9583,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9503,32 +9595,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9550,8 +9651,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9564,32 +9663,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9611,8 +9719,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9625,32 +9731,41 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9672,8 +9787,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9686,32 +9799,37 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9733,8 +9851,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9747,32 +9863,38 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9826,6 +9948,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -9892,7 +10015,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9959,7 +10082,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10026,7 +10149,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10092,7 +10215,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10155,7 +10278,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10219,7 +10342,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10286,7 +10409,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10353,7 +10476,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10420,7 +10543,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10487,7 +10610,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10554,7 +10677,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10631,7 +10754,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10698,7 +10821,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10765,7 +10888,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10831,7 +10954,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10894,7 +11017,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10958,7 +11081,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11025,7 +11148,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11092,7 +11215,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11159,7 +11282,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11226,7 +11349,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11293,7 +11416,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go index 4cb0a7e8779e..892f0368dda2 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftanti.eg.go @@ -9869,6 +9869,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -9891,8 +9892,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9908,33 +9907,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9956,8 +9963,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9973,33 +9978,37 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10021,8 +10030,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10038,33 +10045,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10085,8 +10100,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10102,33 +10115,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10146,8 +10167,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10163,33 +10182,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10208,8 +10235,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10225,33 +10250,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10273,8 +10306,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10290,33 +10321,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10338,8 +10377,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10355,33 +10392,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10403,8 +10448,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10420,33 +10463,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10468,8 +10519,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10485,33 +10534,37 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10533,8 +10586,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10550,33 +10601,38 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10608,8 +10664,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10625,32 +10679,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10672,8 +10735,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10689,32 +10750,37 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10736,8 +10802,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10753,32 +10817,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10799,8 +10872,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10816,32 +10887,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10859,8 +10939,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10876,32 +10954,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10920,8 +11007,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10937,32 +11022,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10984,8 +11078,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11001,32 +11093,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11048,8 +11149,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11065,32 +11164,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11112,8 +11220,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11129,32 +11235,41 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11176,8 +11291,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11193,32 +11306,37 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11240,8 +11358,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11257,32 +11373,38 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11336,6 +11458,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -11402,7 +11525,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11469,7 +11592,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11536,7 +11659,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11602,7 +11725,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11665,7 +11788,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11729,7 +11852,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11796,7 +11919,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11863,7 +11986,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11930,7 +12053,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11997,7 +12120,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12064,7 +12187,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12141,7 +12264,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12208,7 +12331,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12275,7 +12398,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12341,7 +12464,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12404,7 +12527,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12468,7 +12591,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12535,7 +12658,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12602,7 +12725,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12669,7 +12792,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12736,7 +12859,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12803,7 +12926,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go index d464c352540d..cf6c89fd787f 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftouter.eg.go @@ -9913,6 +9913,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -9935,8 +9936,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9949,33 +9948,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9997,8 +10004,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10011,33 +10016,37 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10059,8 +10068,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10073,33 +10080,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10120,8 +10135,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10134,33 +10147,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10178,8 +10199,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10192,33 +10211,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10237,8 +10264,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10251,33 +10276,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10299,8 +10332,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10313,33 +10344,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10361,8 +10400,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10375,33 +10412,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10423,8 +10468,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10437,33 +10480,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10485,8 +10536,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10499,33 +10548,37 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10547,8 +10600,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10561,33 +10612,38 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10619,8 +10675,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10633,32 +10687,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10680,8 +10743,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10694,32 +10755,37 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10741,8 +10807,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10755,32 +10819,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10801,8 +10874,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10815,32 +10886,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10858,8 +10938,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10872,32 +10950,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10916,8 +11003,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10930,32 +11015,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10977,8 +11071,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10991,32 +11083,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11038,8 +11139,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11052,32 +11151,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11099,8 +11207,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11113,32 +11219,41 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11160,8 +11275,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11174,32 +11287,37 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11221,8 +11339,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11235,32 +11351,38 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11314,6 +11436,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -11382,7 +11505,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11451,7 +11574,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11520,7 +11643,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11588,7 +11711,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11653,7 +11776,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11719,7 +11842,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11788,7 +11911,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11857,7 +11980,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11926,7 +12049,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11995,7 +12118,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12064,7 +12187,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12143,7 +12266,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12212,7 +12335,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12281,7 +12404,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12349,7 +12472,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12414,7 +12537,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12480,7 +12603,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12549,7 +12672,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12618,7 +12741,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12687,7 +12810,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12756,7 +12879,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12825,7 +12948,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go index b079269d9269..0a32a201ba3f 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_leftsemi.eg.go @@ -7677,6 +7677,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -7699,8 +7700,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7713,33 +7712,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7761,8 +7768,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7775,33 +7780,37 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7823,8 +7832,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7837,33 +7844,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7884,8 +7899,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7898,33 +7911,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7942,8 +7963,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7956,33 +7975,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8001,8 +8028,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8015,33 +8040,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8063,8 +8096,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8077,33 +8108,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8125,8 +8164,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8139,33 +8176,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8187,8 +8232,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8201,33 +8244,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8249,8 +8300,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8263,33 +8312,37 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8311,8 +8364,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8325,33 +8376,38 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8383,8 +8439,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8397,32 +8451,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8444,8 +8507,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8458,32 +8519,37 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8505,8 +8571,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8519,32 +8583,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8565,8 +8638,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8579,32 +8650,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8622,8 +8702,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8636,32 +8714,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8680,8 +8767,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8694,32 +8779,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8741,8 +8835,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8755,32 +8847,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8802,8 +8903,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8816,32 +8915,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8863,8 +8971,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8877,32 +8983,41 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8924,8 +9039,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8938,32 +9051,37 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8985,8 +9103,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8999,32 +9115,38 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9078,6 +9200,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -9144,7 +9267,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9211,7 +9334,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9278,7 +9401,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9344,7 +9467,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9407,7 +9530,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9471,7 +9594,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9538,7 +9661,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9605,7 +9728,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9672,7 +9795,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9739,7 +9862,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9806,7 +9929,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9883,7 +10006,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9950,7 +10073,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10017,7 +10140,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10083,7 +10206,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10146,7 +10269,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10210,7 +10333,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10277,7 +10400,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10344,7 +10467,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10411,7 +10534,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10478,7 +10601,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10545,7 +10668,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go index fe5326e059f5..e0bb89c01df7 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go @@ -9825,6 +9825,7 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -9847,8 +9848,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9861,36 +9860,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9912,8 +9918,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9926,36 +9930,39 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9977,8 +9984,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9991,36 +9996,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10041,8 +10053,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10055,36 +10065,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10102,8 +10119,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10116,36 +10131,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10164,8 +10186,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10178,36 +10198,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10229,8 +10256,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10243,36 +10268,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10294,8 +10326,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10308,36 +10338,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10359,8 +10396,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10373,36 +10408,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10424,8 +10466,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10438,36 +10478,39 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10489,8 +10532,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10503,36 +10544,40 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10564,8 +10609,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10578,35 +10621,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10628,8 +10679,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10642,35 +10691,39 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10692,8 +10745,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10706,35 +10757,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10755,8 +10814,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10769,35 +10826,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10815,8 +10880,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10829,35 +10892,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10876,8 +10947,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10890,35 +10959,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10940,8 +11017,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10954,35 +11029,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11004,8 +11087,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11018,35 +11099,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11068,8 +11157,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11082,35 +11169,43 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11132,8 +11227,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11146,35 +11239,39 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11196,8 +11293,6 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11210,35 +11305,40 @@ func (o *mergeJoinRightAntiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11292,6 +11392,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -11358,7 +11459,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11425,7 +11526,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11492,7 +11593,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11558,7 +11659,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11621,7 +11722,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11685,7 +11786,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11752,7 +11853,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11819,7 +11920,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11886,7 +11987,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11953,7 +12054,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12020,7 +12121,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12097,7 +12198,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12164,7 +12265,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12231,7 +12332,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12297,7 +12398,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12360,7 +12461,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12424,7 +12525,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12491,7 +12592,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12558,7 +12659,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12625,7 +12726,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12692,7 +12793,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12759,7 +12860,7 @@ func (o *mergeJoinRightAntiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go index b2c189dd5242..c66c465b5fbd 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go @@ -9869,6 +9869,7 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -9891,8 +9892,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9905,36 +9904,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9956,8 +9962,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -9970,36 +9974,39 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10021,8 +10028,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10035,36 +10040,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10085,8 +10097,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10099,36 +10109,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10146,8 +10163,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10160,36 +10175,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10208,8 +10230,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10222,36 +10242,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10273,8 +10300,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10287,36 +10312,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10338,8 +10370,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10352,36 +10382,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10403,8 +10440,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10417,36 +10452,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10468,8 +10510,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10482,36 +10522,39 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10533,8 +10576,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10547,36 +10588,40 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10608,8 +10653,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10622,35 +10665,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10672,8 +10723,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10686,35 +10735,39 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10736,8 +10789,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10750,35 +10801,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10799,8 +10858,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10813,35 +10870,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10859,8 +10924,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10873,35 +10936,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10920,8 +10991,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10934,35 +11003,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -10984,8 +11061,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -10998,35 +11073,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11048,8 +11131,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11062,35 +11143,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11112,8 +11201,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11126,35 +11213,43 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11176,8 +11271,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11190,35 +11283,39 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11240,8 +11337,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -11254,35 +11349,40 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -11336,6 +11436,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -11402,7 +11503,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11469,7 +11570,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11536,7 +11637,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11602,7 +11703,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11665,7 +11766,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11729,7 +11830,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11796,7 +11897,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11863,7 +11964,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11930,7 +12031,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -11997,7 +12098,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12064,7 +12165,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12141,7 +12242,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12208,7 +12309,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12275,7 +12376,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12341,7 +12442,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12404,7 +12505,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12468,7 +12569,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12535,7 +12636,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12602,7 +12703,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12669,7 +12770,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12736,7 +12837,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -12803,7 +12904,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go index 086bda45a3f9..1d1258496b5d 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go @@ -7633,6 +7633,7 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -7655,8 +7656,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7669,33 +7668,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7717,8 +7724,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7731,33 +7736,37 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7779,8 +7788,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7793,33 +7800,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7840,8 +7855,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7854,33 +7867,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7898,8 +7919,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7912,33 +7931,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -7957,8 +7984,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -7971,33 +7996,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8019,8 +8052,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8033,33 +8064,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8081,8 +8120,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8095,33 +8132,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8143,8 +8188,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8157,33 +8200,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8205,8 +8256,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8219,33 +8268,37 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8267,8 +8320,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8281,33 +8332,38 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8339,8 +8395,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bool() } outCol := out.Bool() - var val bool - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8353,32 +8407,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8400,8 +8463,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Bytes() } outCol := out.Bytes() - var val []byte - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8414,32 +8475,37 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8461,8 +8527,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Decimal() } outCol := out.Decimal() - var val apd.Decimal - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8475,32 +8539,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8521,8 +8594,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int16() } outCol := out.Int16() - var val int16 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8535,32 +8606,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8578,8 +8658,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int32() } outCol := out.Int32() - var val int32 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8592,32 +8670,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8636,8 +8723,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Int64() } outCol := out.Int64() - var val int64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8650,32 +8735,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8697,8 +8791,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Float64() } outCol := out.Float64() - var val float64 - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8711,32 +8803,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8758,8 +8859,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Timestamp() } outCol := out.Timestamp() - var val time.Time - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8772,32 +8871,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8819,8 +8927,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Interval() } outCol := out.Interval() - var val duration.Duration - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8833,32 +8939,41 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + //gcassert:bce + outCol.Set(i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8880,8 +8995,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.JSON() } outCol := out.JSON() - var val json.JSON - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8894,32 +9007,37 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { 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++ + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -8941,8 +9059,6 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( srcCol = src.Datum() } outCol := out.Datum() - var val interface{} - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -8955,32 +9071,38 @@ func (o *mergeJoinRightSemiOp) buildLeftGroupsFromBatch( // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx + srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + val := srcCol.Get(srcStartIdx) for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + outCol.Set(outStartIdx+i, val) } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -9034,6 +9156,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec @@ -9100,7 +9223,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9167,7 +9290,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9234,7 +9357,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9300,7 +9423,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9363,7 +9486,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9427,7 +9550,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9494,7 +9617,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9561,7 +9684,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9628,7 +9751,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9695,7 +9818,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9762,7 +9885,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9839,7 +9962,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9906,7 +10029,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -9973,7 +10096,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10039,7 +10162,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10102,7 +10225,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10166,7 +10289,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10233,7 +10356,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10300,7 +10423,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10367,7 +10490,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10434,7 +10557,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -10501,7 +10624,7 @@ func (o *mergeJoinRightSemiOp) buildRightGroupsFromBatch( // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } diff --git a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go index f4707f75ccf7..b0ab33f95f0d 100644 --- a/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/mergejoiner_tmpl.go @@ -680,8 +680,6 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} srcCol = src.TemplateType() } outCol := out.TemplateType() - var val _GOTYPE - var srcStartIdx int // Loop over every group. for ; o.builderState.left.groupsIdx < len(leftGroups); o.builderState.left.groupsIdx++ { @@ -704,15 +702,28 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} // Loop over every row in the group. for ; o.builderState.left.curSrcStartIdx < leftGroup.rowEndIdx; o.builderState.left.curSrcStartIdx++ { // Repeat each row numRepeats times. - srcStartIdx = o.builderState.left.curSrcStartIdx // {{if _HAS_SELECTION}} - srcStartIdx = sel[srcStartIdx] + srcStartIdx := sel[o.builderState.left.curSrcStartIdx] + // {{else}} + srcStartIdx := o.builderState.left.curSrcStartIdx // {{end}} + // {{/* repeatsLeft will always be positive. */}} repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft if outStartIdx+toAppend > o.outputCapacity { toAppend = o.outputCapacity - outStartIdx + if toAppend == 0 { + // {{/* + // We reached the capacity of the output, so + // exit or move onto the next column. + // */}} + if lastSrcCol { + return + } + o.builderState.left.setBuilderColumnState(initialBuilderState) + continue LeftColLoop + } } // {{if or _JOIN_TYPE.IsRightOuter _JOIN_TYPE.IsRightAnti}} @@ -723,27 +734,52 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} // */}} if leftGroup.nullGroup { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else // {{end}} { if srcNulls.NullAt(srcStartIdx) { outNulls.SetNullRange(outStartIdx, outStartIdx+toAppend) - outStartIdx += toAppend } else { - val = srcCol.Get(srcStartIdx) + // {{if not .IsBytesLike}} + // {{if .Sliceable}} + outCol := outCol[outStartIdx:] + _ = outCol[toAppend-1] + // {{end}} + val := srcCol.Get(srcStartIdx) + // {{end}} for i := 0; i < toAppend; i++ { - outCol.Set(outStartIdx, val) - outStartIdx++ + // {{if .IsBytesLike}} + outCol.Copy(srcCol, outStartIdx+i, srcStartIdx) + // {{else}} + // {{if .Sliceable}} + // {{/* + // For the sliceable types, we sliced outCol + // to start at outStartIdx, so we use index + // i directly. + // */}} + //gcassert:bce + outCol.Set(i, val) + // {{else}} + // {{/* + // For the non-sliceable types, outCol + // vector is the original one (i.e. without + // an adjustment), so we need to add + // outStartIdx to set the element at the + // correct index. + // */}} + outCol.Set(outStartIdx+i, val) + // {{end}} + // {{end}} } } } + outStartIdx += toAppend if toAppend < repeatsLeft { // We didn't materialize all the rows in the group so save state and // move to the next column. o.builderState.left.numRepeatsIdx += toAppend - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { return } o.builderState.left.setBuilderColumnState(initialBuilderState) @@ -799,6 +835,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftGroupsFromBatch( // Loop over every column. LeftColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx) var src coldata.Vec @@ -896,7 +933,7 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool) { // */}} // done with the current column. if toAppend < rightGroup.rowEndIdx-o.builderState.right.curSrcStartIdx { // If it's the last column, save state and return. - if colIdx == len(input.sourceTypes)-1 { + if lastSrcCol { o.builderState.right.curSrcStartIdx += toAppend return } @@ -951,6 +988,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightGroupsFromBatch( // Loop over every column. RightColLoop: for colIdx := range input.sourceTypes { + lastSrcCol := colIdx == len(input.sourceTypes)-1 outStartIdx := destStartIdx out := o.output.ColVec(colIdx + colOffset) var src coldata.Vec