Skip to content

Commit e0c79c6

Browse files
Hisoka-Xcloud-fan
andcommitted
[SPARK-43838][SQL] Fix subquery on single table with having clause can't be optimized
### What changes were proposed in this pull request? Eg: ```scala sql("create view t(c1, c2) as values (0, 1), (0, 2), (1, 2)") sql("select c1, c2, (select count(*) cnt from t t2 where t1.c1 = t2.c1 " + "having cnt = 0) from t t1").show() ``` The error will throw: ``` [PLAN_VALIDATION_FAILED_RULE_IN_BATCH] Rule org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery in batch Operator Optimization before Inferring Filters generated an invalid plan: The plan becomes unresolved: 'Project [toprettystring(c1#224, Some(America/Los_Angeles)) AS toprettystring(c1)#238, toprettystring(c2#225, Some(America/Los_Angeles)) AS toprettystring(c2)#239, toprettystring(cnt#246L, Some(America/Los_Angeles)) AS toprettystring(scalarsubquery(c1))#240] +- 'Project [c1#224, c2#225, CASE WHEN isnull(alwaysTrue#245) THEN 0 WHEN NOT (cnt#222L = 0) THEN null ELSE cnt#222L END AS cnt#246L]    +- 'Join LeftOuter, (c1#224 = c1#224#244)       :- Project [col1#226 AS c1#224, col2#227 AS c2#225]       :  +- LocalRelation [col1#226, col2#227]       +- Project [cnt#222L, c1#224#244, cnt#222L, c1#224, true AS alwaysTrue#245]          +- Project [cnt#222L, c1#224 AS c1#224#244, cnt#222L, c1#224]             +- Aggregate [c1#224], [count(1) AS cnt#222L, c1#224]                +- Project [col1#228 AS c1#224]                   +- LocalRelation [col1#228, col2#229]The previous plan: Project [toprettystring(c1#224, Some(America/Los_Angeles)) AS toprettystring(c1)#238, toprettystring(c2#225, Some(America/Los_Angeles)) AS toprettystring(c2)#239, toprettystring(scalar-subquery#223 [c1#224 && (c1#224 = c1#224#244)], Some(America/Los_Angeles)) AS toprettystring(scalarsubquery(c1))#240] :  +- Project [cnt#222L, c1#224 AS c1#224#244] :     +- Filter (cnt#222L = 0) :        +- Aggregate [c1#224], [count(1) AS cnt#222L, c1#224] :           +- Project [col1#228 AS c1#224] :              +- LocalRelation [col1#228, col2#229] +- Project [col1#226 AS c1#224, col2#227 AS c2#225]    +- LocalRelation [col1#226, col2#227] ``` The reason of error is the unresolved expression in `Join` node which generate by subquery decorrelation. The `duplicateResolved` in `Join` node are false. That's meaning the `Join` left and right have same `Attribute`, in this eg is `c1#224`. The right `c1#224` `Attribute` generated by having Inputs, because there are wrong having Inputs. This problem only occurs when there contain having clause. also do some code format fix. ### Why are the changes needed? Fix subquery bug on single table when use having clause ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add new test Closes #41347 from Hisoka-X/SPARK-43838_subquery_having. Lead-authored-by: Jia Fan <[email protected]> Co-authored-by: Wenchen Fan <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 11e2e42 commit e0c79c6

File tree

45 files changed

+2168
-2041
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

45 files changed

+2168
-2041
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -979,10 +979,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB
979979
case ScalarSubquery(query, outerAttrs, _, _, _, _) =>
980980
// Scalar subquery must return one column as output.
981981
if (query.output.size != 1) {
982-
expr.failAnalysis(
983-
errorClass = "INVALID_SUBQUERY_EXPRESSION." +
984-
"SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN",
985-
messageParameters = Map("number" -> query.output.size.toString))
982+
throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(query.output.size,
983+
expr.origin)
986984
}
987985

988986
if (outerAttrs.nonEmpty) {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala

Lines changed: 149 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -82,6 +82,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
8282
}
8383
}
8484

85+
private def existDuplicatedExprId(
86+
existingRelations: mutable.HashSet[RelationWrapper],
87+
plan: RelationWrapper): Boolean = {
88+
existingRelations.filter(_.cls == plan.cls)
89+
.exists(_.outputAttrIds.intersect(plan.outputAttrIds).nonEmpty)
90+
}
91+
8592
/**
8693
* Deduplicate any duplicated relations of a LogicalPlan
8794
* @param existingRelations the known unique relations for a LogicalPlan
@@ -95,59 +102,161 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
95102
case p: LogicalPlan if p.isStreaming => (plan, false)
96103

97104
case m: MultiInstanceRelation =>
98-
val planWrapper = RelationWrapper(m.getClass, m.output.map(_.exprId.id))
99-
if (existingRelations.contains(planWrapper)) {
100-
val newNode = m.newInstance()
101-
newNode.copyTagsFrom(m)
102-
(newNode, true)
103-
} else {
104-
existingRelations.add(planWrapper)
105-
(m, false)
106-
}
105+
deduplicateAndRenew[LogicalPlan with MultiInstanceRelation](
106+
existingRelations,
107+
m,
108+
_.output.map(_.exprId.id),
109+
node => node.newInstance().asInstanceOf[LogicalPlan with MultiInstanceRelation])
110+
111+
case p: Project =>
112+
deduplicateAndRenew[Project](
113+
existingRelations,
114+
p,
115+
newProject => findAliases(newProject.projectList).map(_.exprId.id).toSeq,
116+
newProject => newProject.copy(newAliases(newProject.projectList)))
117+
118+
case s: SerializeFromObject =>
119+
deduplicateAndRenew[SerializeFromObject](
120+
existingRelations,
121+
s,
122+
_.serializer.map(_.exprId.id),
123+
newSer => newSer.copy(newSer.serializer.map(_.newInstance())))
124+
125+
case f: FlatMapGroupsInPandas =>
126+
deduplicateAndRenew[FlatMapGroupsInPandas](
127+
existingRelations,
128+
f,
129+
_.output.map(_.exprId.id),
130+
newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance())))
131+
132+
case f: FlatMapCoGroupsInPandas =>
133+
deduplicateAndRenew[FlatMapCoGroupsInPandas](
134+
existingRelations,
135+
f,
136+
_.output.map(_.exprId.id),
137+
newFlatMap => newFlatMap.copy(output = newFlatMap.output.map(_.newInstance())))
138+
139+
case m: MapInPandas =>
140+
deduplicateAndRenew[MapInPandas](
141+
existingRelations,
142+
m,
143+
_.output.map(_.exprId.id),
144+
newMap => newMap.copy(output = newMap.output.map(_.newInstance())))
145+
146+
case p: PythonMapInArrow =>
147+
deduplicateAndRenew[PythonMapInArrow](
148+
existingRelations,
149+
p,
150+
_.output.map(_.exprId.id),
151+
newMap => newMap.copy(output = newMap.output.map(_.newInstance())))
152+
153+
case a: AttachDistributedSequence =>
154+
deduplicateAndRenew[AttachDistributedSequence](
155+
existingRelations,
156+
a,
157+
_.producedAttributes.map(_.exprId.id).toSeq,
158+
newAttach => newAttach.copy(sequenceAttr = newAttach.producedAttributes
159+
.map(_.newInstance()).head))
160+
161+
case g: Generate =>
162+
deduplicateAndRenew[Generate](
163+
existingRelations,
164+
g,
165+
_.generatorOutput.map(_.exprId.id), newGenerate =>
166+
newGenerate.copy(generatorOutput = newGenerate.generatorOutput.map(_.newInstance())))
167+
168+
case e: Expand =>
169+
deduplicateAndRenew[Expand](
170+
existingRelations,
171+
e,
172+
_.producedAttributes.map(_.exprId.id).toSeq,
173+
newExpand => newExpand.copy(output = newExpand.output.map(_.newInstance())))
174+
175+
case w: Window =>
176+
deduplicateAndRenew[Window](
177+
existingRelations,
178+
w,
179+
_.windowExpressions.map(_.exprId.id),
180+
newWindow => newWindow.copy(windowExpressions =
181+
newWindow.windowExpressions.map(_.newInstance())))
182+
183+
case s: ScriptTransformation =>
184+
deduplicateAndRenew[ScriptTransformation](
185+
existingRelations,
186+
s,
187+
_.output.map(_.exprId.id),
188+
newScript => newScript.copy(output = newScript.output.map(_.newInstance())))
107189

108190
case plan: LogicalPlan =>
109-
var planChanged = false
110-
val newPlan = if (plan.children.nonEmpty) {
111-
val newChildren = mutable.ArrayBuffer.empty[LogicalPlan]
112-
for (c <- plan.children) {
113-
val (renewed, changed) = renewDuplicatedRelations(existingRelations, c)
114-
newChildren += renewed
115-
if (changed) {
116-
planChanged = true
117-
}
118-
}
191+
deduplicate(existingRelations, plan)
192+
}
119193

120-
val planWithNewSubquery = plan.transformExpressions {
121-
case subquery: SubqueryExpression =>
122-
val (renewed, changed) = renewDuplicatedRelations(existingRelations, subquery.plan)
123-
if (changed) planChanged = true
124-
subquery.withNewPlan(renewed)
194+
private def deduplicate(
195+
existingRelations: mutable.HashSet[RelationWrapper],
196+
plan: LogicalPlan): (LogicalPlan, Boolean) = {
197+
var planChanged = false
198+
val newPlan = if (plan.children.nonEmpty) {
199+
val newChildren = mutable.ArrayBuffer.empty[LogicalPlan]
200+
for (c <- plan.children) {
201+
val (renewed, changed) = renewDuplicatedRelations(existingRelations, c)
202+
newChildren += renewed
203+
if (changed) {
204+
planChanged = true
125205
}
206+
}
207+
208+
val planWithNewSubquery = plan.transformExpressions {
209+
case subquery: SubqueryExpression =>
210+
val (renewed, changed) = renewDuplicatedRelations(existingRelations, subquery.plan)
211+
if (changed) planChanged = true
212+
subquery.withNewPlan(renewed)
213+
}
126214

127-
if (planChanged) {
128-
if (planWithNewSubquery.childrenResolved) {
129-
val planWithNewChildren = planWithNewSubquery.withNewChildren(newChildren.toSeq)
130-
val attrMap = AttributeMap(
131-
plan
132-
.children
133-
.flatMap(_.output).zip(newChildren.flatMap(_.output))
134-
.filter { case (a1, a2) => a1.exprId != a2.exprId }
135-
)
136-
if (attrMap.isEmpty) {
137-
planWithNewChildren
138-
} else {
139-
planWithNewChildren.rewriteAttrs(attrMap)
140-
}
215+
if (planChanged) {
216+
if (planWithNewSubquery.childrenResolved) {
217+
val planWithNewChildren = planWithNewSubquery.withNewChildren(newChildren.toSeq)
218+
val attrMap = AttributeMap(plan.children.flatMap(_.output)
219+
.zip(newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId })
220+
if (attrMap.isEmpty) {
221+
planWithNewChildren
141222
} else {
142-
planWithNewSubquery.withNewChildren(newChildren.toSeq)
223+
planWithNewChildren.rewriteAttrs(attrMap)
143224
}
144225
} else {
145-
plan
226+
planWithNewSubquery.withNewChildren(newChildren.toSeq)
146227
}
147228
} else {
148229
plan
149230
}
231+
} else {
232+
plan
233+
}
234+
(newPlan, planChanged)
235+
}
236+
237+
private def deduplicateAndRenew[T <: LogicalPlan](
238+
existingRelations: mutable.HashSet[RelationWrapper], plan: T,
239+
getExprIds: T => Seq[Long],
240+
copyNewPlan: T => T): (LogicalPlan, Boolean) = {
241+
var (newPlan, planChanged) = deduplicate(existingRelations, plan)
242+
if (newPlan.resolved) {
243+
val exprIds = getExprIds(newPlan.asInstanceOf[T])
244+
if (exprIds.nonEmpty) {
245+
val planWrapper = RelationWrapper(newPlan.getClass, exprIds)
246+
if (existDuplicatedExprId(existingRelations, planWrapper)) {
247+
newPlan = copyNewPlan(newPlan.asInstanceOf[T])
248+
newPlan.copyTagsFrom(plan)
249+
(newPlan, true)
250+
} else {
251+
existingRelations.add(planWrapper)
252+
(newPlan, planChanged)
253+
}
254+
} else {
255+
(newPlan, planChanged)
256+
}
257+
} else {
150258
(newPlan, planChanged)
259+
}
151260
}
152261

153262
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -271,7 +271,10 @@ case class ScalarSubquery(
271271
mayHaveCountBug: Option[Boolean] = None)
272272
extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable {
273273
override def dataType: DataType = {
274-
assert(plan.schema.fields.nonEmpty, "Scalar subquery should have only one column")
274+
if (!plan.schema.fields.nonEmpty) {
275+
throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length,
276+
origin)
277+
}
275278
plan.schema.fields.head.dataType
276279
}
277280
override def nullable: Boolean = true

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2072,6 +2072,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
20722072
messageParameters = Map("function" -> funcStr))
20732073
}
20742074

2075+
def subqueryReturnMoreThanOneColumn(number: Int, origin: Origin): Throwable = {
2076+
new AnalysisException(
2077+
errorClass = "INVALID_SUBQUERY_EXPRESSION." +
2078+
"SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN",
2079+
origin = origin,
2080+
messageParameters = Map("number" -> number.toString))
2081+
}
2082+
20752083
def unsupportedCorrelatedReferenceDataTypeError(
20762084
expr: Expression,
20772085
dataType: DataType,

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala

Lines changed: 0 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -437,25 +437,6 @@ class LeftSemiAntiJoinPushDownSuite extends PlanTest {
437437
}
438438
}
439439

440-
Seq(LeftSemi, LeftAnti).foreach { case jt =>
441-
test(s"Aggregate: $jt join no pushdown - join condition refers left leg and right leg child") {
442-
val aggregation = testRelation
443-
.select($"b".as("id"), $"c")
444-
.groupBy($"id")($"id", sum($"c").as("sum"))
445-
446-
// reference "b" exists in left leg, and the children of the right leg of the join
447-
val originalQuery = aggregation.select(($"id" + 1).as("id_plus_1"), $"sum")
448-
.join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1"))
449-
val optimized = Optimize.execute(originalQuery.analyze)
450-
val correctAnswer = testRelation
451-
.select($"b".as("id"), $"c")
452-
.groupBy($"id")(($"id" + 1).as("id_plus_1"), sum($"c").as("sum"))
453-
.join(aggregation, joinType = jt, condition = Some($"id" === $"id_plus_1"))
454-
.analyze
455-
comparePlans(optimized, correctAnswer)
456-
}
457-
}
458-
459440
Seq(LeftSemi, LeftAnti).foreach { case outerJT =>
460441
Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT =>
461442
test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") {

sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out

Lines changed: 30 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -1977,17 +1977,16 @@ Union false, false
19771977
: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
19781978
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
19791979
: +- LocalRelation [col1#x, col2#x]
1980-
+- Project [c1#x AS c1#x, c2#x AS c2#x, c1#x AS c1#x, c2#x AS c2#x]
1981-
+- Project [c1#x, c2#x, c1#x, c2#x]
1982-
+- Join Inner
1983-
:- SubqueryAlias spark_catalog.default.t1
1984-
: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
1985-
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1986-
: +- LocalRelation [col1#x, col2#x]
1987-
+- SubqueryAlias spark_catalog.default.t4
1988-
+- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x])
1989-
+- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1990-
+- LocalRelation [col1#x, col2#x]
1980+
+- Project [c1#x, c2#x, c1#x, c2#x]
1981+
+- Join Inner
1982+
:- SubqueryAlias spark_catalog.default.t1
1983+
: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
1984+
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1985+
: +- LocalRelation [col1#x, col2#x]
1986+
+- SubqueryAlias spark_catalog.default.t4
1987+
+- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x])
1988+
+- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1989+
+- LocalRelation [col1#x, col2#x]
19911990

19921991

19931992
-- !query
@@ -2030,27 +2029,26 @@ Union false, false
20302029
: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
20312030
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
20322031
: +- LocalRelation [col1#x, col2#x]
2033-
+- Project [c1#x AS c1#x, c2#x AS c2#x, c2#x AS c2#x]
2034-
+- Project [c1#x, c2#x, c2#x]
2035-
+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner
2036-
: +- SubqueryAlias __auto_generated_subquery_name
2037-
: +- Union false, false
2038-
: :- Project [c2#x]
2039-
: : +- Filter (outer(c1#x) <= c1#x)
2040-
: : +- SubqueryAlias spark_catalog.default.t1
2041-
: : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
2042-
: : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2043-
: : +- LocalRelation [col1#x, col2#x]
2044-
: +- Project [c2#x]
2045-
: +- Filter (c1#x < outer(c1#x))
2046-
: +- SubqueryAlias spark_catalog.default.t4
2047-
: +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x])
2048-
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2049-
: +- LocalRelation [col1#x, col2#x]
2050-
+- SubqueryAlias spark_catalog.default.t2
2051-
+- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])
2052-
+- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2053-
+- LocalRelation [col1#x, col2#x]
2032+
+- Project [c1#x, c2#x, c2#x]
2033+
+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner
2034+
: +- SubqueryAlias __auto_generated_subquery_name
2035+
: +- Union false, false
2036+
: :- Project [c2#x]
2037+
: : +- Filter (outer(c1#x) <= c1#x)
2038+
: : +- SubqueryAlias spark_catalog.default.t1
2039+
: : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])
2040+
: : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2041+
: : +- LocalRelation [col1#x, col2#x]
2042+
: +- Project [c2#x]
2043+
: +- Filter (c1#x < outer(c1#x))
2044+
: +- SubqueryAlias spark_catalog.default.t4
2045+
: +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x])
2046+
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2047+
: +- LocalRelation [col1#x, col2#x]
2048+
+- SubqueryAlias spark_catalog.default.t2
2049+
+- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])
2050+
+- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
2051+
+- LocalRelation [col1#x, col2#x]
20542052

20552053

20562054
-- !query

sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1057,3 +1057,21 @@ Project [c1#xL, c2#xL]
10571057
: +- Range (1, 2, step=1, splits=None)
10581058
+- SubqueryAlias t1
10591059
+- Range (1, 3, step=1, splits=None)
1060+
1061+
1062+
-- !query
1063+
SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1
1064+
-- !query analysis
1065+
Project [c1#x, c2#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL]
1066+
: +- Filter (cnt#xL = cast(0 as bigint))
1067+
: +- Aggregate [count(1) AS cnt#xL]
1068+
: +- Filter (outer(c1#x) = c1#x)
1069+
: +- SubqueryAlias t2
1070+
: +- SubqueryAlias t1
1071+
: +- View (`t1`, [c1#x,c2#x])
1072+
: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1073+
: +- LocalRelation [col1#x, col2#x]
1074+
+- SubqueryAlias t1
1075+
+- View (`t1`, [c1#x,c2#x])
1076+
+- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
1077+
+- LocalRelation [col1#x, col2#x]

sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -255,3 +255,6 @@ select * from (
255255
where t1.id = t2.id ) c2
256256
from range (1, 3) t1 ) t
257257
where t.c2 is not null;
258+
259+
-- SPARK-43838: Subquery on single table with having clause
260+
SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1

sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -598,3 +598,12 @@ where t.c2 is not null
598598
struct<c1:bigint,c2:bigint>
599599
-- !query output
600600
1 1
601+
602+
603+
-- !query
604+
SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1
605+
-- !query schema
606+
struct<c1:int,c2:int,scalarsubquery(c1):bigint>
607+
-- !query output
608+
0 1 NULL
609+
1 2 NULL

0 commit comments

Comments
 (0)