From 01f26f765209de16ae8b77baa81a0818283b21d6 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Fri, 30 Jun 2023 14:03:23 -0600 Subject: [PATCH] colexec: don't infinite loop cross-join with zero-column left input Previously, the cross-joiner wouldn't advance its internal state when its left input projected no columns. This would result in an infinite loop as the right rows were repeatedly emitted. This patch advances the state when there are no left columns as if values from the left side were emitted. Fixes #105882 Release note (bug fix): Fixed a bug introduced in v22.1 that could cause a join to infinite-loop in rare cases when (1) the join filter is not an equality and (2) no columns from the left input are returned. --- pkg/sql/colexec/colexecjoin/crossjoiner.eg.go | 50 +++++++++++++++++++ .../colexec/colexecjoin/crossjoiner_tmpl.go | 26 ++++++++++ .../logictest/testdata/logic_test/cross_join | 15 ++++++ 3 files changed, 91 insertions(+) diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go b/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go index 183f72a9a08d..e29ab82f1652 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner.eg.go @@ -691,6 +691,31 @@ func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx i colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) } } + // If there are no columns projected from the left input, simply advance the + // cross-joiner state according to the number of input rows. + if len(b.left.types) == 0 { + outStartIdx := destStartIdx + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + outStartIdx += toAppend + } + } } } else { { @@ -1339,6 +1364,31 @@ func (b *crossJoinerBase) buildFromLeftInput(ctx context.Context, destStartIdx i colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) } } + // If there are no columns projected from the left input, simply advance the + // cross-joiner state according to the number of input rows. + if len(b.left.types) == 0 { + outStartIdx := destStartIdx + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + outStartIdx += toAppend + } + } } } }, diff --git a/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go b/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go index d465e80945cb..8f5be4325cac 100644 --- a/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go +++ b/pkg/sql/colexec/colexecjoin/crossjoiner_tmpl.go @@ -168,6 +168,32 @@ func buildFromLeftBatch(b *crossJoinerBase, currentBatch coldata.Batch, sel []in colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", b.left.types[colIdx].String())) } } + // If there are no columns projected from the left input, simply advance the + // cross-joiner state according to the number of input rows. + if len(b.left.types) == 0 { + outStartIdx := destStartIdx + for bs.curSrcStartIdx < leftSrcEndIdx && outStartIdx < outputCapacity { + // Repeat each row leftNumRepeats times. + // {{/* toAppend will always be positive. */}} + toAppend := leftNumRepeats - bs.numRepeatsIdx + if outStartIdx+toAppend > outputCapacity { + // We don't have enough space to repeat the current + // value the required number of times, so we'll have + // to continue from here on the next call. + toAppend = outputCapacity - outStartIdx + bs.numRepeatsIdx += toAppend + } else { + // We fully processed the current tuple for the + // current column, and before moving on to the next + // one, we need to reset numRepeatsIdx (so that the + // next tuple would be repeated leftNumRepeats + // times). + bs.curSrcStartIdx++ + bs.numRepeatsIdx = 0 + } + outStartIdx += toAppend + } + } } // buildFromLeftInput builds part of the output of a cross join that comes from diff --git a/pkg/sql/logictest/testdata/logic_test/cross_join b/pkg/sql/logictest/testdata/logic_test/cross_join index 789090488cc7..df4e8d5a8daf 100644 --- a/pkg/sql/logictest/testdata/logic_test/cross_join +++ b/pkg/sql/logictest/testdata/logic_test/cross_join @@ -81,3 +81,18 @@ SELECT * ) WHERE r < .01 LIMIT 1 + +# Regression test for #105882 - don't infinite loop when the left input has +# no columns. +statement ok +CREATE TABLE t105882 (c0 INT); +UPSERT INTO t105882 (c0) VALUES(1); + +query I +SELECT ( + SELECT count(t2.rowid) FROM t105882 t2 + WHERE ((t1.rowid) IN (SELECT max(t3.rowid) FROM t105882 t3)) +) +FROM t105882 t1; +---- +1