From 9a6f9826a737277f802e125645a04dbe11547c9c Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 14 Mar 2025 17:22:23 -0700 Subject: [PATCH 01/48] add unresolved outer attrs --- .../sql/catalyst/analysis/Analyzer.scala | 18 +++---- .../catalyst/expressions/DynamicPruning.scala | 8 +++- ...ctionTableSubqueryArgumentExpression.scala | 9 +++- .../sql/catalyst/expressions/subquery.scala | 47 ++++++++++++++++--- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../sql/catalyst/optimizer/expressions.scala | 2 +- .../sql/catalyst/optimizer/subquery.scala | 20 ++++---- .../adaptive/PlanAdaptiveSubqueries.scala | 4 +- 8 files changed, 79 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0dcb38a97077b..cf065969767a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2319,21 +2319,21 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor */ private def resolveSubQueries(plan: LogicalPlan, outer: LogicalPlan): LogicalPlan = { plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { - case s @ ScalarSubquery(sub, _, exprId, _, _, _, _) if !sub.resolved => - resolveSubQuery(s, outer)(ScalarSubquery(_, _, exprId)) - case e @ Exists(sub, _, exprId, _, _) if !sub.resolved => - resolveSubQuery(e, outer)(Exists(_, _, exprId)) - case InSubquery(values, l @ ListQuery(_, _, exprId, _, _, _)) + case s @ ScalarSubquery(sub, _, _, exprId, _, _, _, _) if !sub.resolved => + resolveSubQuery(s, outer)(ScalarSubquery(_, _, Seq.empty, exprId)) + case e @ Exists(sub, _, _, exprId, _, _) if !sub.resolved => + resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) + case InSubquery(values, l @ ListQuery(_, _, _, exprId, _, _)) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { - ListQuery(plan, exprs, exprId, plan.output.length) + ListQuery(plan, exprs, Seq.empty, exprId, plan.output.length) }) InSubquery(values, expr.asInstanceOf[ListQuery]) - case s @ LateralSubquery(sub, _, exprId, _, _) if !sub.resolved => - resolveSubQuery(s, outer)(LateralSubquery(_, _, exprId)) + case s @ LateralSubquery(sub, _, _, exprId, _, _) if !sub.resolved => + resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) case a: FunctionTableSubqueryArgumentExpression if !a.plan.resolved => resolveSubQuery(a, outer)( - (plan, outerAttrs) => a.copy(plan = plan, outerAttrs = outerAttrs)) + (plan, outerAttrs) => a.copy(plan = plan, outerAttrs = outerAttrs, unresolvedOuterAttrs = Seq.empty)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index b65576403e9d8..52d0c6e081927 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -47,7 +47,7 @@ case class DynamicPruningSubquery( onlyInBroadcast: Boolean, exprId: ExprId = NamedExpression.newExprId, hint: Option[HintInfo] = None) - extends SubqueryExpression(buildQuery, Seq(pruningKey), exprId, Seq.empty, hint) + extends SubqueryExpression(buildQuery, Seq(pruningKey), Seq.empty, exprId, Seq.empty, hint) with DynamicPruning with Unevaluable with UnaryLike[Expression] { @@ -67,6 +67,12 @@ case class DynamicPruningSubquery( copy() } + override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): DynamicPruningSubquery = { + // DynamicPruningSubquery should not have unresolved outer attrs + assert(unresolvedOuterAttrs.isEmpty) + copy() + } + override def withNewHint(hint: Option[HintInfo]): SubqueryExpression = copy(hint = hint) override lazy val resolved: Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index bfd3bc8051dff..9029ccaf1b121 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -67,12 +67,13 @@ import org.apache.spark.sql.types.DataType case class FunctionTableSubqueryArgumentExpression( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, + unresolvedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, partitionByExpressions: Seq[Expression] = Seq.empty, withSinglePartition: Boolean = false, orderByExpressions: Seq[SortOrder] = Seq.empty, selectedInputExpressions: Seq[PythonUDTFSelectedExpression] = Seq.empty) - extends SubqueryExpression(plan, outerAttrs, exprId, Seq.empty, None) with Unevaluable { + extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { assert(!(withSinglePartition && partitionByExpressions.nonEmpty), "WITH SINGLE PARTITION is mutually exclusive with PARTITION BY") @@ -86,11 +87,17 @@ case class FunctionTableSubqueryArgumentExpression( override def hint: Option[HintInfo] = None override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() + override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): FunctionTableSubqueryArgumentExpression = { + assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + } override def toString: String = s"table-argument#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { FunctionTableSubqueryArgumentExpression( plan.canonicalized, outerAttrs.map(_.canonicalized), + unresolvedOuterAttrs.map(_.canonicalized), ExprId(0), partitionByExpressions, withSinglePartition, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 210b7f8fb5306..d76a39962f202 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -67,6 +67,8 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { * * @param plan: the subquery plan * @param outerAttrs: the outer references in the subquery plan + * @param unresolvedOuterAttrs: the outer references in the subquery plan that cannot be resolved + * in its immediate parent plan * @param exprId: ID of the expression * @param joinCond: the join conditions with the outer query. It contains both inner and outer * query references. @@ -76,6 +78,7 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { abstract class SubqueryExpression( plan: LogicalPlan, outerAttrs: Seq[Expression], + unresolvedOuterAttrs: Seq[Expression], exprId: ExprId, joinCond: Seq[Expression], hint: Option[HintInfo]) extends PlanExpression[LogicalPlan] { @@ -85,6 +88,10 @@ abstract class SubqueryExpression( override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression def withNewOuterAttrs(outerAttrs: Seq[Expression]): SubqueryExpression + def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): SubqueryExpression + def getUnresolvedOuterAttrs: Seq[Expression] = unresolvedOuterAttrs + def getOuterAttrs: Seq[Expression] = outerAttrs + def getJoinCond: Seq[Expression] = joinCond def isCorrelated: Boolean = outerAttrs.nonEmpty def hint: Option[HintInfo] def withNewHint(hint: Option[HintInfo]): SubqueryExpression @@ -395,12 +402,13 @@ object SubExprUtils extends PredicateHelper { case class ScalarSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, + unresolvedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None, mayHaveCountBug: Option[Boolean] = None, needSingleJoin: Option[Boolean] = None) - extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, @@ -410,14 +418,20 @@ case class ScalarSubquery( } override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) - override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ScalarSubquery = copy( - outerAttrs = outerAttrs) + override def withNewUnresolvedOuterAttrs( + unresolvedOuterAttrs: Seq[Expression] + ): ScalarSubquery = { + assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + } override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { ScalarSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), + unresolvedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -474,21 +488,28 @@ case class UnresolvedTableArgPlanId( case class LateralSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, + unresolvedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = plan.output.toStructType override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) + override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): LateralSubquery = { + assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { LateralSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), + unresolvedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -517,13 +538,14 @@ case class LateralSubquery( case class ListQuery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, + unresolvedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, // The plan of list query may have more columns after de-correlation, and we need to track the // number of the columns of the original plan, to report the data type properly. numCols: Int = -1, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { def childOutputs: Seq[Attribute] = plan.output.take(numCols) override def dataType: DataType = if (numCols > 1) { childOutputs.toStructType @@ -543,12 +565,18 @@ case class ListQuery( override def withNewPlan(plan: LogicalPlan): ListQuery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) + override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): ListQuery = { + assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + } override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) override def toString: String = s"list#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { ListQuery( plan.canonicalized, outerAttrs.map(_.canonicalized), + unresolvedOuterAttrs.map(_.canonicalized), ExprId(0), numCols, joinCond.map(_.canonicalized)) @@ -591,22 +619,29 @@ case class ListQuery( case class Exists( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, + unresolvedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, exprId, joinCond, hint) + extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Predicate with Unevaluable { override def nullable: Boolean = false override def withNewPlan(plan: LogicalPlan): Exists = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) + override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): Exists = { + assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + } override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) override def toString: String = s"exists#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { Exists( plan.canonicalized, outerAttrs.map(_.canonicalized), + unresolvedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 9d269f37e58b9..c2738b736c4c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -364,7 +364,7 @@ abstract class Optimizer(catalogManager: CatalogManager) case d: DynamicPruningSubquery => d case s @ ScalarSubquery( PhysicalOperation(projections, predicates, a @ Aggregate(group, _, child, _)), - _, _, _, _, mayHaveCountBug, _) + _, _, _, _, _, mayHaveCountBug, _) if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && mayHaveCountBug.nonEmpty && mayHaveCountBug.get => // This is a subquery with an aggregate that may suffer from a COUNT bug. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index e867953bcf282..4bd81ff28f280 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -90,7 +90,7 @@ object ConstantFolding extends Rule[LogicalPlan] { } // Don't replace ScalarSubquery if its plan is an aggregate that may suffer from a COUNT bug. - case s @ ScalarSubquery(_, _, _, _, _, mayHaveCountBug, _) + case s @ ScalarSubquery(_, _, _, _, _, _, mayHaveCountBug, _) if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && mayHaveCountBug.nonEmpty && mayHaveCountBug.get => s diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 378081221c8c1..3523008e48bb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -587,7 +587,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case ScalarSubquery(sub, children, exprId, conditions, hint, + case ScalarSubquery(sub, children, unresolvedOuterAttrs, exprId, conditions, hint, mayHaveCountBugOld, needSingleJoinOld) if children.nonEmpty => @@ -639,26 +639,26 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } else { conf.getConf(SQLConf.SCALAR_SUBQUERY_USE_SINGLE_JOIN) && !guaranteedToReturnOneRow(sub) } - ScalarSubquery(newPlan, children, exprId, getJoinCondition(newCond, conditions), + ScalarSubquery(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint, Some(mayHaveCountBug), Some(needSingleJoin)) - case Exists(sub, children, exprId, conditions, hint) if children.nonEmpty => + case Exists(sub, children, unresolvedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } - Exists(newPlan, children, exprId, getJoinCondition(newCond, conditions), hint) - case ListQuery(sub, children, exprId, numCols, conditions, hint) if children.nonEmpty => + Exists(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) + case ListQuery(sub, children, unresolvedOuterAttrs, exprId, numCols, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } val joinCond = getJoinCondition(newCond, conditions) - ListQuery(newPlan, children, exprId, numCols, joinCond, hint) - case LateralSubquery(sub, children, exprId, conditions, hint) if children.nonEmpty => + ListQuery(newPlan, children, unresolvedOuterAttrs, exprId, numCols, joinCond, hint) + case LateralSubquery(sub, children, unresolvedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = decorrelate(sub, plan, handleCountBug = true) - LateralSubquery(newPlan, children, exprId, getJoinCondition(newCond, conditions), hint) + LateralSubquery(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) } } @@ -1122,7 +1122,7 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { */ private def rewrite(plan: LogicalPlan): LogicalPlan = plan.transformUpWithSubqueries { case LateralJoin( - left, right @ LateralSubquery(OneRowSubquery(plan), _, _, _, _), _, None) + left, right @ LateralSubquery(OneRowSubquery(plan), _, _, _, _, _), _, None) if !hasCorrelatedSubquery(right.plan) && right.joinCond.isEmpty => plan match { case Project(projectList, _: OneRowRelation) => @@ -1145,7 +1145,7 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { case p: LogicalPlan => p.transformExpressionsUpWithPruning( _.containsPattern(SCALAR_SUBQUERY)) { - case s @ ScalarSubquery(OneRowSubquery(p @ Project(_, _: OneRowRelation)), _, _, _, _, _, _) + case s @ ScalarSubquery(OneRowSubquery(p @ Project(_, _: OneRowRelation)), _, _, _, _, _, _, _) if !hasCorrelatedSubquery(s.plan) && s.joinCond.isEmpty => assert(p.projectList.size == 1) stripOuterReferences(p.projectList).head diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala index 5f2638655c37c..dd38c1e94e481 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala @@ -30,11 +30,11 @@ case class PlanAdaptiveSubqueries( def apply(plan: SparkPlan): SparkPlan = { plan.transformAllExpressionsWithPruning( _.containsAnyPattern(SCALAR_SUBQUERY, IN_SUBQUERY, DYNAMIC_PRUNING_SUBQUERY)) { - case expressions.ScalarSubquery(_, _, exprId, _, _, _, _) => + case expressions.ScalarSubquery(_, _, _, exprId, _, _, _, _) => val subquery = SubqueryExec.createForScalarSubquery( s"subquery#${exprId.id}", subqueryMap(exprId.id)) execution.ScalarSubquery(subquery, exprId) - case expressions.InSubquery(values, ListQuery(_, _, exprId, _, _, _)) => + case expressions.InSubquery(values, ListQuery(_, _, _, exprId, _, _, _)) => val expr = if (values.length == 1) { values.head } else { From 52d5ce70d3ff0ccd7fca371e7d3572e2d3500997 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 14 Mar 2025 17:41:34 -0700 Subject: [PATCH 02/48] fix wrong number of arguments error; fix assertions --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../apache/spark/sql/catalyst/catalog/SQLFunction.scala | 2 +- .../FunctionTableSubqueryArgumentExpression.scala | 2 +- .../apache/spark/sql/catalyst/expressions/subquery.scala | 8 ++++---- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cf065969767a1..47647ee73b528 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2323,7 +2323,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor resolveSubQuery(s, outer)(ScalarSubquery(_, _, Seq.empty, exprId)) case e @ Exists(sub, _, _, exprId, _, _) if !sub.resolved => resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) - case InSubquery(values, l @ ListQuery(_, _, _, exprId, _, _)) + case InSubquery(values, l @ ListQuery(_, _, _, exprId, _, _, _)) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { ListQuery(plan, exprs, Seq.empty, exprId, plan.output.length) @@ -2333,7 +2333,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) case a: FunctionTableSubqueryArgumentExpression if !a.plan.resolved => resolveSubQuery(a, outer)( - (plan, outerAttrs) => a.copy(plan = plan, outerAttrs = outerAttrs, unresolvedOuterAttrs = Seq.empty)) + (plan, outerAttrs) => a.copy(plan = plan, outerAttrs = outerAttrs)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index 923373c1856a9..72f0b7c3376b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -85,7 +85,7 @@ case class SQLFunction( case (None, Some(Project(expr :: Nil, _: OneRowRelation))) if !isTableFunc => (Some(expr), None) - case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), _, _, _, _, _, _)), None) + case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), _, _, _, _, _, _, _)), None) if !isTableFunc => (Some(expr), None) case (_, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index 9029ccaf1b121..eb9d3e758e505 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -88,7 +88,7 @@ case class FunctionTableSubqueryArgumentExpression( override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): FunctionTableSubqueryArgumentExpression = { - assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index d76a39962f202..decf21f7511de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -421,7 +421,7 @@ case class ScalarSubquery( override def withNewUnresolvedOuterAttrs( unresolvedOuterAttrs: Seq[Expression] ): ScalarSubquery = { - assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } @@ -499,7 +499,7 @@ case class LateralSubquery( override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): LateralSubquery = { - assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } @@ -566,7 +566,7 @@ case class ListQuery( override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): ListQuery = { - assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } @@ -631,7 +631,7 @@ case class Exists( override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): Exists = { - assert(unresolvedOuterAttrs.subsetOf(outerAttrs), + assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } From e3bfef4cbedbc68acd5d570b11f1c9a508538966 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 14 Mar 2025 23:35:06 -0700 Subject: [PATCH 03/48] fix wrong number of arguments error --- .../sql/catalyst/optimizer/expressions.scala | 2 +- .../sql/catalyst/optimizer/subquery.scala | 18 +++++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 4bd81ff28f280..e6d553e80d4eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -891,7 +891,7 @@ object NullPropagation extends Rule[LogicalPlan] { case InSubquery(Seq(Literal(null, _)), _) if SQLConf.get.legacyNullInEmptyBehavior => Literal.create(null, BooleanType) - case InSubquery(Seq(Literal(null, _)), ListQuery(sub, _, _, _, conditions, _)) + case InSubquery(Seq(Literal(null, _)), ListQuery(sub, _, _, _, _, conditions, _)) if !SQLConf.get.legacyNullInEmptyBehavior && conditions.isEmpty => If(Exists(sub), Literal(null, BooleanType), FalseLiteral) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 3523008e48bb1..298210762f88d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -152,17 +152,17 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { - case (p, Exists(sub, _, _, conditions, subHint)) => + case (p, Exists(sub, _, _, _, conditions, subHint)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) val join = buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond), LeftSemi, joinCond, subHint) Project(p.output, join) - case (p, Not(Exists(sub, _, _, conditions, subHint))) => + case (p, Not(Exists(sub, _, _, _, conditions, subHint))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) val join = buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond), LeftAnti, joinCond, subHint) Project(p.output, join) - case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint))) => + case (p, InSubquery(values, ListQuery(sub, _, _, _, _, conditions, subHint))) => // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) val inConditions = values.zip(newSub.output).map(EqualTo.tupled) @@ -170,7 +170,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val join = Join(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, newSub, joinCond), LeftSemi, joinCond, JoinHint(None, subHint)) Project(p.output, join) - case (p, Not(InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint)))) => + case (p, Not(InSubquery(values, ListQuery(sub, _, _, _, _, conditions, subHint)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive // result; such a row will be filtered out by the Anti-Join operator. @@ -400,7 +400,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val introducedAttrs = ArrayBuffer.empty[Attribute] val newExprs = exprs.map { e => e.transformDownWithPruning(_.containsAnyPattern(EXISTS_SUBQUERY, IN_SUBQUERY)) { - case Exists(sub, _, _, conditions, subHint) => + case Exists(sub, _, _, _, conditions, subHint) => val exists = AttributeReference("exists", BooleanType, nullable = false)() val existenceJoin = ExistenceJoin(exists) val newCondition = conditions.reduceLeftOption(And) @@ -409,7 +409,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { existenceJoin, newCondition, subHint) introducedAttrs += exists exists - case Not(InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint))) => + case Not(InSubquery(values, ListQuery(sub, _, _, _, _, conditions, subHint))) => val exists = AttributeReference("exists", BooleanType, nullable = false)() // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) @@ -434,7 +434,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { ExistenceJoin(exists), Some(finalJoinCond), joinHint) introducedAttrs += exists Not(exists) - case InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint)) => + case InSubquery(values, ListQuery(sub, _, _, _, _, conditions, subHint)) => val exists = AttributeReference("exists", BooleanType, nullable = false)() // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) @@ -898,7 +898,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe subqueries: ArrayBuffer[ScalarSubquery]): (LogicalPlan, AttributeMap[Attribute]) = { val subqueryAttrMapping = ArrayBuffer[(Attribute, Attribute)]() val newChild = subqueries.foldLeft(child) { - case (currentChild, ScalarSubquery(sub, _, _, conditions, subHint, mayHaveCountBug, + case (currentChild, ScalarSubquery(sub, _, _, _, conditions, subHint, mayHaveCountBug, needSingleJoin)) => val query = DecorrelateInnerQuery.rewriteDomainJoins(currentChild, sub, conditions) val origOutput = query.output.head @@ -1085,7 +1085,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe object RewriteLateralSubquery extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning( _.containsPattern(LATERAL_JOIN)) { - case LateralJoin(left, LateralSubquery(sub, _, _, joinCond, subHint), joinType, condition) => + case LateralJoin(left, LateralSubquery(sub, _, _, _, joinCond, subHint), joinType, condition) => val newRight = DecorrelateInnerQuery.rewriteDomainJoins(left, sub, joinCond) val newCond = (condition ++ joinCond).reduceOption(And) // The subquery appears on the right side of the join, hence add the hint to the right side From 995ffdd19331a87132b3c2f470ce08bb5e5204da Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 14 Mar 2025 23:47:44 -0700 Subject: [PATCH 04/48] fix wrong number of arguments error --- .../sql/catalyst/analysis/ValidateSubqueryExpression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index d6b7a4dccb907..24018c454c1b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -129,7 +129,7 @@ object ValidateSubqueryExpression checkOuterReference(plan, expr) expr match { - case ScalarSubquery(query, outerAttrs, _, _, _, _, _) => + case ScalarSubquery(query, outerAttrs, _, _, _, _, _, _) => // Scalar subquery must return one column as output. if (query.output.size != 1) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(query.output.size, From 08d3cce201815bfb013cb03d9d85a36bb688707c Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Sat, 15 Mar 2025 00:04:41 -0700 Subject: [PATCH 05/48] fix for mis-deleting ScalarSubquery.withNewOuterAttrs --- .../org/apache/spark/sql/catalyst/expressions/subquery.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index decf21f7511de..f94b1b2f16a29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -418,6 +418,8 @@ case class ScalarSubquery( } override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ScalarSubquery = copy( + outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs( unresolvedOuterAttrs: Seq[Expression] ): ScalarSubquery = { From 471f0849e4bbd7a84558a180b16e1e45afcac4b0 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Sat, 15 Mar 2025 00:19:23 -0700 Subject: [PATCH 06/48] fmt --- .../sql/catalyst/catalog/SQLFunction.scala | 4 +-- .../catalyst/expressions/DynamicPruning.scala | 4 ++- ...ctionTableSubqueryArgumentExpression.scala | 10 +++++--- .../sql/catalyst/expressions/subquery.scala | 25 +++++++++++++------ 4 files changed, 29 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index 72f0b7c3376b5..d65c90099b62c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -85,8 +85,8 @@ case class SQLFunction( case (None, Some(Project(expr :: Nil, _: OneRowRelation))) if !isTableFunc => (Some(expr), None) - case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), _, _, _, _, _, _, _)), None) - if !isTableFunc => + case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), + _, _, _, _, _, _, _)), None) if !isTableFunc => (Some(expr), None) case (_, _) => (parsedExpression, parsedQuery) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index 52d0c6e081927..bb51546a2fdc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -67,7 +67,9 @@ case class DynamicPruningSubquery( copy() } - override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): DynamicPruningSubquery = { + override def withNewUnresolvedOuterAttrs( + unresolvedOuterAttrs: Seq[Expression] + ): DynamicPruningSubquery = { // DynamicPruningSubquery should not have unresolved outer attrs assert(unresolvedOuterAttrs.isEmpty) copy() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index eb9d3e758e505..544532f818e25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -73,7 +73,8 @@ case class FunctionTableSubqueryArgumentExpression( withSinglePartition: Boolean = false, orderByExpressions: Seq[SortOrder] = Seq.empty, selectedInputExpressions: Seq[PythonUDTFSelectedExpression] = Seq.empty) - extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { + extends SubqueryExpression( + plan, outerAttrs, unresolvedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { assert(!(withSinglePartition && partitionByExpressions.nonEmpty), "WITH SINGLE PARTITION is mutually exclusive with PARTITION BY") @@ -87,9 +88,12 @@ case class FunctionTableSubqueryArgumentExpression( override def hint: Option[HintInfo] = None override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() - override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): FunctionTableSubqueryArgumentExpression = { + override def withNewUnresolvedOuterAttrs( + unresolvedOuterAttrs: Seq[Expression] + ): FunctionTableSubqueryArgumentExpression = { assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + s"unresolvedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } override def toString: String = s"table-argument#${exprId.id} $conditionString" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index f94b1b2f16a29..79c2777d74faa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -408,7 +408,8 @@ case class ScalarSubquery( hint: Option[HintInfo] = None, mayHaveCountBug: Option[Boolean] = None, needSingleJoin: Option[Boolean] = None) - extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression( + plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, @@ -424,7 +425,8 @@ case class ScalarSubquery( unresolvedOuterAttrs: Seq[Expression] ): ScalarSubquery = { assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + s"unresolvedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) @@ -494,15 +496,19 @@ case class LateralSubquery( exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression( + plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = plan.output.toStructType override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) - override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): LateralSubquery = { + override def withNewUnresolvedOuterAttrs( + unresolvedOuterAttrs: Seq[Expression] + ): LateralSubquery = { assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + s"unresolvedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) @@ -547,7 +553,8 @@ case class ListQuery( numCols: Int = -1, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + extends SubqueryExpression( + plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { def childOutputs: Seq[Attribute] = plan.output.take(numCols) override def dataType: DataType = if (numCols > 1) { childOutputs.toStructType @@ -569,7 +576,8 @@ case class ListQuery( outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): ListQuery = { assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + s"unresolvedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) @@ -634,7 +642,8 @@ case class Exists( outerAttrs = outerAttrs) override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): Exists = { assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, but got ${unresolvedOuterAttrs.mkString(", ")}") + s"unresolvedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${unresolvedOuterAttrs.mkString(", ")}") copy(unresolvedOuterAttrs = unresolvedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) From 4e0bf7409f6c7d892123742168b156c3d4b2049d Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Sat, 15 Mar 2025 00:28:41 -0700 Subject: [PATCH 07/48] fix wrong number of arguments error --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index d5f258a8084be..e07dcc1b5e46d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -187,7 +187,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { SubqueryExec.createForScalarSubquery( s"scalar-subquery#${subquery.exprId.id}", executedPlan), subquery.exprId) - case expressions.InSubquery(values, ListQuery(query, _, exprId, _, _, _)) => + case expressions.InSubquery(values, ListQuery(query, _, _, exprId, _, _, _)) => val expr = if (values.length == 1) { values.head } else { From bc9179e695139286afc1fbf71471987dee84b2c5 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Sat, 15 Mar 2025 00:47:46 -0700 Subject: [PATCH 08/48] fix wrong number of arguments error --- .../spark/sql/execution/command/PlanResolutionSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index b4b3bff86471e..2a4cd889c0ccf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1012,7 +1012,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { query match { case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", Seq()), UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), - _, _, _, _, _) => + _, _, _, _, _, _) => assert(projects.size == 1 && projects.head.name == "s.name") assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") case o => fail("Unexpected subquery: \n" + o.treeString) @@ -1093,7 +1093,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { query match { case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", Seq()), UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), - _, _, _, _, _) => + _, _, _, _, _, _) => assert(projects.size == 1 && projects.head.name == "s.name") assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") case o => fail("Unexpected subquery: \n" + o.treeString) From 9559dbc6cbf6583a2f1e76e03e4f2aa925f7ca37 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 9 Apr 2025 12:01:04 -0700 Subject: [PATCH 09/48] rename unresolved outer attrs to nested outer attrs --- .../catalyst/expressions/DynamicPruning.scala | 8 +- ...ctionTableSubqueryArgumentExpression.scala | 18 ++--- .../sql/catalyst/expressions/subquery.scala | 76 +++++++++---------- .../sql/catalyst/optimizer/subquery.scala | 16 ++-- 4 files changed, 59 insertions(+), 59 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index bb51546a2fdc4..a632e29ccdb26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -67,11 +67,11 @@ case class DynamicPruningSubquery( copy() } - override def withNewUnresolvedOuterAttrs( - unresolvedOuterAttrs: Seq[Expression] + override def withNewNestedOuterAttrs( + nestedOuterAttrs: Seq[Expression] ): DynamicPruningSubquery = { - // DynamicPruningSubquery should not have unresolved outer attrs - assert(unresolvedOuterAttrs.isEmpty) + // DynamicPruningSubquery should not have nested outer attrs + assert(nestedOuterAttrs.isEmpty) copy() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index 544532f818e25..bfe8d1ab8f27d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -67,14 +67,14 @@ import org.apache.spark.sql.types.DataType case class FunctionTableSubqueryArgumentExpression( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - unresolvedOuterAttrs: Seq[Expression] = Seq.empty, + nestedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, partitionByExpressions: Seq[Expression] = Seq.empty, withSinglePartition: Boolean = false, orderByExpressions: Seq[SortOrder] = Seq.empty, selectedInputExpressions: Seq[PythonUDTFSelectedExpression] = Seq.empty) extends SubqueryExpression( - plan, outerAttrs, unresolvedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { + plan, outerAttrs, nestedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { assert(!(withSinglePartition && partitionByExpressions.nonEmpty), "WITH SINGLE PARTITION is mutually exclusive with PARTITION BY") @@ -88,20 +88,20 @@ case class FunctionTableSubqueryArgumentExpression( override def hint: Option[HintInfo] = None override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() - override def withNewUnresolvedOuterAttrs( - unresolvedOuterAttrs: Seq[Expression] + override def withNewNestedOuterAttrs( + nestedOuterAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${unresolvedOuterAttrs.mkString(", ")}") - copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def toString: String = s"table-argument#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { FunctionTableSubqueryArgumentExpression( plan.canonicalized, outerAttrs.map(_.canonicalized), - unresolvedOuterAttrs.map(_.canonicalized), + nestedOuterAttrs.map(_.canonicalized), ExprId(0), partitionByExpressions, withSinglePartition, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 79c2777d74faa..cfb9811ff94e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -67,7 +67,7 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { * * @param plan: the subquery plan * @param outerAttrs: the outer references in the subquery plan - * @param unresolvedOuterAttrs: the outer references in the subquery plan that cannot be resolved + * @param nestedOuterAttrs: the outer references in the subquery plan that cannot be resolved * in its immediate parent plan * @param exprId: ID of the expression * @param joinCond: the join conditions with the outer query. It contains both inner and outer @@ -78,7 +78,7 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { abstract class SubqueryExpression( plan: LogicalPlan, outerAttrs: Seq[Expression], - unresolvedOuterAttrs: Seq[Expression], + nestedOuterAttrs: Seq[Expression], exprId: ExprId, joinCond: Seq[Expression], hint: Option[HintInfo]) extends PlanExpression[LogicalPlan] { @@ -88,8 +88,8 @@ abstract class SubqueryExpression( override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression def withNewOuterAttrs(outerAttrs: Seq[Expression]): SubqueryExpression - def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): SubqueryExpression - def getUnresolvedOuterAttrs: Seq[Expression] = unresolvedOuterAttrs + def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): SubqueryExpression + def getNestedOuterAttrs: Seq[Expression] = nestedOuterAttrs def getOuterAttrs: Seq[Expression] = outerAttrs def getJoinCond: Seq[Expression] = joinCond def isCorrelated: Boolean = outerAttrs.nonEmpty @@ -402,14 +402,14 @@ object SubExprUtils extends PredicateHelper { case class ScalarSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - unresolvedOuterAttrs: Seq[Expression] = Seq.empty, + nestedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None, mayHaveCountBug: Option[Boolean] = None, needSingleJoin: Option[Boolean] = None) extends SubqueryExpression( - plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, @@ -421,13 +421,13 @@ case class ScalarSubquery( override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ScalarSubquery = copy( outerAttrs = outerAttrs) - override def withNewUnresolvedOuterAttrs( - unresolvedOuterAttrs: Seq[Expression] + override def withNewNestedOuterAttrs( + nestedOuterAttrs: Seq[Expression] ): ScalarSubquery = { - assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${unresolvedOuterAttrs.mkString(", ")}") - copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" @@ -435,7 +435,7 @@ case class ScalarSubquery( ScalarSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), - unresolvedOuterAttrs.map(_.canonicalized), + nestedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -492,24 +492,24 @@ case class UnresolvedTableArgPlanId( case class LateralSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - unresolvedOuterAttrs: Seq[Expression] = Seq.empty, + nestedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) extends SubqueryExpression( - plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = plan.output.toStructType override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) - override def withNewUnresolvedOuterAttrs( - unresolvedOuterAttrs: Seq[Expression] + override def withNewNestedOuterAttrs( + nestedOuterAttrs: Seq[Expression] ): LateralSubquery = { - assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${unresolvedOuterAttrs.mkString(", ")}") - copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" @@ -517,7 +517,7 @@ case class LateralSubquery( LateralSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), - unresolvedOuterAttrs.map(_.canonicalized), + nestedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -546,7 +546,7 @@ case class LateralSubquery( case class ListQuery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - unresolvedOuterAttrs: Seq[Expression] = Seq.empty, + nestedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, // The plan of list query may have more columns after de-correlation, and we need to track the // number of the columns of the original plan, to report the data type properly. @@ -554,7 +554,7 @@ case class ListQuery( joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) extends SubqueryExpression( - plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { def childOutputs: Seq[Attribute] = plan.output.take(numCols) override def dataType: DataType = if (numCols > 1) { childOutputs.toStructType @@ -574,11 +574,11 @@ case class ListQuery( override def withNewPlan(plan: LogicalPlan): ListQuery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) - override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): ListQuery = { - assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${unresolvedOuterAttrs.mkString(", ")}") - copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + override def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): ListQuery = { + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) override def toString: String = s"list#${exprId.id} $conditionString" @@ -586,7 +586,7 @@ case class ListQuery( ListQuery( plan.canonicalized, outerAttrs.map(_.canonicalized), - unresolvedOuterAttrs.map(_.canonicalized), + nestedOuterAttrs.map(_.canonicalized), ExprId(0), numCols, joinCond.map(_.canonicalized)) @@ -629,22 +629,22 @@ case class ListQuery( case class Exists( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - unresolvedOuterAttrs: Seq[Expression] = Seq.empty, + nestedOuterAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) + extends SubqueryExpression(plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Predicate with Unevaluable { override def nullable: Boolean = false override def withNewPlan(plan: LogicalPlan): Exists = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) - override def withNewUnresolvedOuterAttrs(unresolvedOuterAttrs: Seq[Expression]): Exists = { - assert(unresolvedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"unresolvedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${unresolvedOuterAttrs.mkString(", ")}") - copy(unresolvedOuterAttrs = unresolvedOuterAttrs) + override def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): Exists = { + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) override def toString: String = s"exists#${exprId.id} $conditionString" @@ -652,7 +652,7 @@ case class Exists( Exists( plan.canonicalized, outerAttrs.map(_.canonicalized), - unresolvedOuterAttrs.map(_.canonicalized), + nestedOuterAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 298210762f88d..99111db8b3122 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -587,7 +587,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case ScalarSubquery(sub, children, unresolvedOuterAttrs, exprId, conditions, hint, + case ScalarSubquery(sub, children, nestedOuterAttrs, exprId, conditions, hint, mayHaveCountBugOld, needSingleJoinOld) if children.nonEmpty => @@ -639,26 +639,26 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } else { conf.getConf(SQLConf.SCALAR_SUBQUERY_USE_SINGLE_JOIN) && !guaranteedToReturnOneRow(sub) } - ScalarSubquery(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), + ScalarSubquery(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint, Some(mayHaveCountBug), Some(needSingleJoin)) - case Exists(sub, children, unresolvedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => + case Exists(sub, children, nestedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } - Exists(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) - case ListQuery(sub, children, unresolvedOuterAttrs, exprId, numCols, conditions, hint) if children.nonEmpty => + Exists(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) + case ListQuery(sub, children, nestedOuterAttrs, exprId, numCols, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } val joinCond = getJoinCondition(newCond, conditions) - ListQuery(newPlan, children, unresolvedOuterAttrs, exprId, numCols, joinCond, hint) - case LateralSubquery(sub, children, unresolvedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => + ListQuery(newPlan, children, nestedOuterAttrs, exprId, numCols, joinCond, hint) + case LateralSubquery(sub, children, nestedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = decorrelate(sub, plan, handleCountBug = true) - LateralSubquery(newPlan, children, unresolvedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) + LateralSubquery(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) } } From edd682881a91c257d4a0dadf581a9c23667e4970 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 9 Apr 2025 13:29:39 -0700 Subject: [PATCH 10/48] Analyzer support nested correlated subqueries --- .../sql/catalyst/analysis/Analyzer.scala | 161 ++++++++++++++---- .../sql/catalyst/analysis/CheckAnalysis.scala | 25 +++ .../analysis/ColumnResolutionHelper.scala | 30 ++-- .../analysis/ValidateSubqueryExpression.scala | 13 ++ ...ctionTableSubqueryArgumentExpression.scala | 7 +- .../apache/spark/sql/internal/SQLConf.scala | 8 + 6 files changed, 197 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 47647ee73b528..5647cf5edec35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -25,16 +25,10 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.{Failure, Random, Success, Try} -import org.apache.spark.{SparkException, SparkThrowable, SparkUnsupportedOperationException} +import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkThrowable, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.analysis.resolver.{ - AnalyzerBridgeState, - HybridAnalyzer, - Resolver => OperatorResolver, - ResolverExtension, - ResolverGuard -} +import org.apache.spark.sql.catalyst.analysis.resolver.{AnalyzerBridgeState, HybridAnalyzer, Resolver => OperatorResolver, ResolverExtension, ResolverGuard} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ @@ -156,7 +150,7 @@ case class AnalysisContext( // lookup a temporary function. And export to the view metadata. referredTempFunctionNames: mutable.Set[String] = mutable.Set.empty, referredTempVariableNames: Seq[Seq[String]] = Seq.empty, - outerPlan: Option[LogicalPlan] = None, + outerPlans: Option[Seq[LogicalPlan]] = None, isExecuteImmediate: Boolean = false, /** @@ -232,9 +226,9 @@ object AnalysisContext { try f finally { set(originContext) } } - def withOuterPlan[A](outerPlan: LogicalPlan)(f: => A): A = { + def withOuterPlans[A](outerPlans: Seq[LogicalPlan])(f: => A): A = { val originContext = value.get() - val context = originContext.copy(outerPlan = Some(outerPlan)) + val context = originContext.copy(outerPlans = Some(outerPlans)) set(context) try f finally { set(originContext) } } @@ -1762,17 +1756,30 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor s.expand(plan, resolver) } catch { case e: AnalysisException => - AnalysisContext.get.outerPlan.map { - // Only Project, Aggregate, CollectMetrics can host star expressions. - case u @ (_: Project | _: Aggregate | _: CollectMetrics) => - Try(s.expand(u.children.head, resolver)) match { - case Success(expanded) => expanded.map(wrapOuterReference) - case Failure(_) => throw e - } - // Do not use the outer plan to resolve the star expression - // since the star usage is invalid. - case _ => throw e - }.getOrElse { throw e } + val outerPlans = + if (AnalysisContext.get.outerPlans.isDefined) { + AnalysisContext.get.outerPlans.get + } else { + Seq.empty[LogicalPlan] + } + val success = outerPlans.flatMap { plan => + plan match { + // Only Project, Aggregate, CollectMetrics can host star expressions. + case u @ (_: Project | _: Aggregate | _: CollectMetrics) => + Try(s.expand(u.children.head, resolver)) match { + case Success(expanded) => expanded.map(wrapOuterReference) + case Failure(_) => Seq[NamedExpression]() + } + // Do not use the outer plan to resolve the star expression + // since the star usage is invalid. + case _ => Seq[NamedExpression]() + } + } + if (success.nonEmpty) { + return success + } else { + throw e + } } } } @@ -2283,6 +2290,27 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor * Note: CTEs are handled in CTESubstitution. */ object ResolveSubquery extends Rule[LogicalPlan] { + + /** + * Returns the nested outer attributes referenced in the subquery expressions + * in current plan and the children of the current plan. + */ + private def getOuterAttrsNeedToBePropagated(plan: LogicalPlan): Seq[Expression] = { + plan.expressions.flatMap { + case subExpr: SubqueryExpression => subExpr.getNestedOuterAttrs + case in: InSubquery => in.query.getNestedOuterAttrs + case expr if expr.containsPattern(PLAN_EXPRESSION) => + expr.collect { + case subExpr: SubqueryExpression => subExpr.getNestedOuterAttrs + }.flatten + case _ => Seq.empty + } ++ plan.children.flatMap{ + case p if p.containsPattern(PLAN_EXPRESSION) => + getOuterAttrsNeedToBePropagated(p) + case _ => Seq.empty + } + } + /** * Resolves the subquery plan that is referenced in a subquery expression, by invoking the * entire analyzer recursively. We set outer plan in `AnalysisContext`, so that the analyzer @@ -2294,20 +2322,53 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor e: SubqueryExpression, outer: LogicalPlan)( f: (LogicalPlan, Seq[Expression]) => SubqueryExpression): SubqueryExpression = { - val newSubqueryPlan = AnalysisContext.withOuterPlan(outer) { - executeSameContext(e.plan) + val outerPlanContext = AnalysisContext.get.outerPlans + val newSubqueryPlan = if (outerPlanContext.isDefined) { + // The previous outerPlanContext contains resolved nested outer plans + // and unresolved direct outer plan. Append the current outer plan into + // new outerPlanContext as current outer is guaranteed to be resolved. + val updatedOuterPlan = outerPlanContext.get ++ Seq(outer) + AnalysisContext.withOuterPlans(updatedOuterPlan) { + executeSameContext(e.plan) + } + } else { + AnalysisContext.withOuterPlans(Seq(outer)) { + executeSameContext(e.plan) + } } // If the subquery plan is fully resolved, pull the outer references and record // them as children of SubqueryExpression. if (newSubqueryPlan.resolved) { // Record the outer references as children of subquery expression. - f(newSubqueryPlan, SubExprUtils.getOuterReferences(newSubqueryPlan)) + f(newSubqueryPlan, SubExprUtils.getOuterReferences(newSubqueryPlan) ++ + getOuterAttrsNeedToBePropagated(newSubqueryPlan)) } else { e.withNewPlan(newSubqueryPlan) } } + /** + * Returns the outer references that are not resolved in the current plan {{p}}. + * These outer references are nested outer references which can be resolved + * in nested outer plans. + * If these references cannot be resolved in the whole query plan, an analysis + * exception will be thrown in checkAnalysis or ColumnResolutionHelper$resolve. + */ + private def getNestedOuterReferences( + s: SubqueryExpression, p: LogicalPlan + ): Seq[Expression] = { + val outerReferencesInSubquery = s.getOuterAttrs + + // return outer references cannot be resolved in current plan + outerReferencesInSubquery.filter( + _ match { + case a: AttributeReference => !p.inputSet.contains(a) + case _ => false + } + ) + } + /** * Resolves the subquery. Apart of resolving the subquery and outer references (if any) * in the subquery plan, the children of subquery expression are updated to record the @@ -2319,18 +2380,56 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor */ private def resolveSubQueries(plan: LogicalPlan, outer: LogicalPlan): LogicalPlan = { plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { + // There are four kinds of outer references here: + // 1. Outer references which are newly introduced in the subquery `res` + // which can be resolved in current `plan`. + // It is extracted by `SubExprUtils.getOuterReferences(res.plan)` and + // stored among res.outerAttrs + // 2. Outer references which are newly introduced in the subquery `res` + // which cannot be resolved in current `plan` + // It is extracted by `SubExprUtils.getOuterReferences(res.plan)` with + // `getNestedOuterReferences(res, plan)` filter and stored in + // res.nestedOuterAttrs + // 3. Outer references which are introduced by nested subquery within `res.plan` + // which can be resolved in current `plan` + // It is extracted by `getOuterAttrsNeedToBePropagated(res.plan)`, filtered + // by `plan.inputSet.contains(_)`, need to be stored in res.outerAttrs + // 4. Outer references which are introduced by nested subquery within `res.plan` + // which cannot be resolved in current `plan` + // It is extracted by `getOuterAttrsNeedToBePropagated(res.plan)`, filtered + // by `!plan.inputSet.contains(_)`, need to be stored in + // res.outerAttrs and res.nestedOuterAttrs case s @ ScalarSubquery(sub, _, _, exprId, _, _, _, _) if !sub.resolved => - resolveSubQuery(s, outer)(ScalarSubquery(_, _, Seq.empty, exprId)) + val res = resolveSubQuery(s, outer)(ScalarSubquery(_, _, Seq.empty, exprId)) + val nestedOuterReferences = getNestedOuterReferences(res, plan) + res.withNewNestedOuterAttrs(nestedOuterReferences) case e @ Exists(sub, _, _, exprId, _, _) if !sub.resolved => - resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) + val res = resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) + val nestedOuterReferences = getNestedOuterReferences(res, plan) + res.withNewNestedOuterAttrs(nestedOuterReferences) case InSubquery(values, l @ ListQuery(_, _, _, exprId, _, _, _)) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { - ListQuery(plan, exprs, Seq.empty, exprId, plan.output.length) - }) - InSubquery(values, expr.asInstanceOf[ListQuery]) + ListQuery(plan, exprs, Seq.empty, i.query.exprId, plan.output.length) + }).asInstanceOf[ListQuery] + val nestedOuterReferences = getNestedOuterReferences(expr, plan) + val newExpr = expr.withNewNestedOuterAttrs(nestedOuterReferences) + InSubquery(values, newExpr) case s @ LateralSubquery(sub, _, _, exprId, _, _) if !sub.resolved => - resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) + val res = resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) + val nestedOuterReferences = getNestedOuterReferences(res, plan) + if (nestedOuterReferences.nonEmpty) { + // Outer references within lateral subquery can only refer to attributes + // in the containing plan of lateral subquery. Nested outer references + // are not allowed. + throw new SparkIllegalArgumentException( + errorClass = "FIELD_NOT_FOUND", + messageParameters = Map( + "fieldName" -> toSQLId(nestedOuterReferences.head.prettyName), + "fields" -> plan.inputSet.map(f => toSQLId(f.name)).mkString(", ")) + ) + } + res case a: FunctionTableSubqueryArgumentExpression if !a.plan.resolved => resolveSubQuery(a, outer)( (plan, outerAttrs) => a.copy(plan = plan, outerAttrs = outerAttrs)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c8678d994ba80..ad40eff87d946 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -228,6 +228,30 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } } + def checkNoNestedOuterReferencesInMainQuery(plan: LogicalPlan): Unit = { + plan.expressions.foreach { + case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => + subExpr.failAnalysis( + errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + messageParameters = Map("objectName" -> toSQLId( + subExpr.getNestedOuterAttrs.head.prettyName))) + case expr if expr.containsPattern(PLAN_EXPRESSION) => + expr.collect { + case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => + subExpr.failAnalysis( + errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + messageParameters = Map("objectName" -> toSQLId( + subExpr.getNestedOuterAttrs.head.prettyName))) + } + case _ => + } + plan.children.foreach { + case p: LogicalPlan if p.containsPattern(PLAN_EXPRESSION) => + checkNoNestedOuterReferencesInMainQuery(p) + case _ => + } + } + def checkAnalysis(plan: LogicalPlan): Unit = { // We should inline all CTE relations to restore the original plan shape, as the analysis check // may need to match certain plan shapes. For dangling CTE relations, they will still be kept @@ -241,6 +265,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } preemptedError.clear() try { + checkNoNestedOuterReferencesInMainQuery(inlinedPlan) checkAnalysis0(inlinedPlan) preemptedError.getErrorOpt().foreach(throw _) // throw preempted error if any } catch { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index e778342d08374..08bc2d426628d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -199,21 +199,27 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { // Resolves `UnresolvedAttribute` to `OuterReference`. protected def resolveOuterRef(e: Expression): Expression = { - val outerPlan = AnalysisContext.get.outerPlan - if (outerPlan.isEmpty) return e + val outerPlanContext = AnalysisContext.get.outerPlans + if (outerPlanContext.isEmpty) return e def resolve(nameParts: Seq[String]): Option[Expression] = try { - outerPlan.get match { - // Subqueries in UnresolvedHaving can host grouping expressions and aggregate functions. - // We should resolve columns with `agg.output` and the rule `ResolveAggregateFunctions` will - // push them down to Aggregate later. This is similar to what we do in `resolveColumns`. - case u @ UnresolvedHaving(_, agg: Aggregate) => - agg.resolveChildren(nameParts, conf.resolver) - .orElse(u.resolveChildren(nameParts, conf.resolver)) - .map(wrapOuterReference) - case other => - other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference) + val outerPlans = outerPlanContext.get + val resolvedExpressions = outerPlans.flatMap { + _ match { + // Subqueries in UnresolvedHaving can host grouping + // expressions and aggregate functions. We should resolve + // columns with `agg.output` and the rule `ResolveAggregateFunctions` will + // push them down to Aggregate later. This is similar to what we do in `resolveColumns`. + case u @ UnresolvedHaving(_, agg: Aggregate) => + agg.resolveChildren(nameParts, conf.resolver) + .orElse(u.resolveChildren(nameParts, conf.resolver)) + .map(wrapOuterReference) + case other => + other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference) + } } + assert(resolvedExpressions.size <= 1) + resolvedExpressions.headOption } catch { case ae: AnalysisException => logDebug(ae.getMessage) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 24018c454c1b1..4379b27b4c268 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -125,6 +125,19 @@ object ValidateSubqueryExpression case _ => } + def checkNestedOuterReferences(expr: SubqueryExpression): Unit = { + if ((!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) && + expr.getNestedOuterAttrs.nonEmpty) { + expr.failAnalysis( + errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + messageParameters = Map("objectName" -> toSQLId( + expr.getNestedOuterAttrs.head.prettyName))) + } + } + + // Check if there are nested correlated subqueries in the plan. + checkNestedOuterReferences(expr) + // Check if there is outer attribute that cannot be found from the plan. checkOuterReference(plan, expr) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index bfe8d1ab8f27d..f8002063fcda6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -91,10 +91,9 @@ case class FunctionTableSubqueryArgumentExpression( override def withNewNestedOuterAttrs( nestedOuterAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + // FunctionTableSubquery should not have nested outer attrs + assert(nestedOuterAttrs.isEmpty) + copy() } override def toString: String = s"table-argument#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7a2bd12c868a0..9e6dc1b4c4b65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4062,6 +4062,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val SUPPORT_NESTED_CORRELATED_SUBQUERIES = + buildConf("spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled") + .internal() + .doc("If enabled, support nested correlated subqueries") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + val PULL_OUT_NESTED_DATA_OUTER_REF_EXPRESSIONS_ENABLED = buildConf("spark.sql.optimizer.pullOutNestedDataOuterRefExpressions.enabled") .internal() From bbfdd7bdcfbf4eecfb35bc5af5f24515fa9b69eb Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 9 Apr 2025 13:43:34 -0700 Subject: [PATCH 11/48] fix compilation error --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5647cf5edec35..dc6a7f290afb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2407,10 +2407,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val res = resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) val nestedOuterReferences = getNestedOuterReferences(res, plan) res.withNewNestedOuterAttrs(nestedOuterReferences) - case InSubquery(values, l @ ListQuery(_, _, _, exprId, _, _, _)) + case InSubquery(values, l) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { - ListQuery(plan, exprs, Seq.empty, i.query.exprId, plan.output.length) + ListQuery(plan, exprs, Seq.empty, l.exprId, plan.output.length) }).asInstanceOf[ListQuery] val nestedOuterReferences = getNestedOuterReferences(expr, plan) val newExpr = expr.withNewNestedOuterAttrs(nestedOuterReferences) From dbb2dd1fccc34174e044c8f34602b923bca22fdc Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 15 Apr 2025 11:10:39 -0700 Subject: [PATCH 12/48] throw internalErrors and format --- .../spark/sql/catalyst/expressions/DynamicPruning.scala | 5 ++++- .../org/apache/spark/sql/catalyst/expressions/subquery.scala | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index a632e29ccdb26..97ef3fd30d2a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -71,7 +71,10 @@ case class DynamicPruningSubquery( nestedOuterAttrs: Seq[Expression] ): DynamicPruningSubquery = { // DynamicPruningSubquery should not have nested outer attrs - assert(nestedOuterAttrs.isEmpty) + if (nestedOuterAttrs.nonEmpty) { + throw SparkException.internalError( + "DynamicPruningSubquery should not have nested outer attributes.") + } copy() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index cfb9811ff94e5..f4c39b7c7c3e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -501,6 +501,7 @@ case class LateralSubquery( override def dataType: DataType = plan.output.toStructType override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) override def withNewNestedOuterAttrs( @@ -511,6 +512,7 @@ case class LateralSubquery( s"but got ${nestedOuterAttrs.mkString(", ")}") copy(nestedOuterAttrs = nestedOuterAttrs) } + override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { From 4500892a1205c0098bb548a3abf323206b09a631 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 15 Apr 2025 12:58:47 -0700 Subject: [PATCH 13/48] compile and format --- .../apache/spark/sql/catalyst/expressions/DynamicPruning.scala | 1 + .../org/apache/spark/sql/catalyst/expressions/subquery.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index 97ef3fd30d2a6..ebb4e4591a006 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, LogicalPlan} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index f4c39b7c7c3e2..17aab1db08f6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -512,7 +512,7 @@ case class LateralSubquery( s"but got ${nestedOuterAttrs.mkString(", ")}") copy(nestedOuterAttrs = nestedOuterAttrs) } - + override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { From 588627339acbed5db5fa6236c99feb9d0ba9c73f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 15 Apr 2025 13:58:46 -0700 Subject: [PATCH 14/48] testing --- .../scala/org/apache/spark/sql/SubquerySuite.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index e7e41f6570d3c..ce81b71017264 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2830,4 +2830,17 @@ class SubquerySuite extends QueryTest Row(1, false) :: Row(2, false) :: Row(3, true) :: Nil) } } + + test("analyzer same error class") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2);") + sql("CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3);") + sql("CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1))," + + " (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4));") + sql("CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2)," + + " (1, 1), (1, 3);") + + val query = "SELECT * FROM t1, LATERAL" + + " (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*, t3.* FROM t2 AS t3))" + val res = sql(query).collect() + } } From 31937b6cde1956ad5cdc092944781a29d3b8f359 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 11:39:43 -0700 Subject: [PATCH 15/48] try to align errors --- .../resources/error/error-conditions.json | 6 +++++ .../sql/catalyst/analysis/Analyzer.scala | 12 ++++------ .../analysis/ValidateSubqueryExpression.scala | 23 ++++++++++++++----- .../org/apache/spark/sql/SubquerySuite.scala | 16 ++++++------- 4 files changed, 35 insertions(+), 22 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 2ae54ef5f305d..1efc7bd521d08 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3974,6 +3974,12 @@ ], "sqlState" : "42607" }, + "NESTED_REFERENCES_IN_LATERAL_SUBQUERY" : { + "message" : [ + "Lateral subqueries can only reference attributes from their own scope or their immediate containing query. Invalid external references are not allowed." + ], + "sqlState" : "42703" + }, "NESTED_EXECUTE_IMMEDIATE" : { "message" : [ "Nested EXECUTE IMMEDIATE commands are not allowed. Please ensure that the SQL query provided () does not contain another EXECUTE IMMEDIATE command." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index dc6a7f290afb4..463db62ad4f22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.{Failure, Random, Success, Try} -import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkThrowable, SparkUnsupportedOperationException} +import org.apache.spark.{SparkException, SparkThrowable, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis.resolver.{AnalyzerBridgeState, HybridAnalyzer, Resolver => OperatorResolver, ResolverExtension, ResolverGuard} @@ -2327,7 +2327,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // The previous outerPlanContext contains resolved nested outer plans // and unresolved direct outer plan. Append the current outer plan into // new outerPlanContext as current outer is guaranteed to be resolved. - val updatedOuterPlan = outerPlanContext.get ++ Seq(outer) + val updatedOuterPlan = Seq(outer) ++ outerPlanContext.get AnalysisContext.withOuterPlans(updatedOuterPlan) { executeSameContext(e.plan) } @@ -2422,11 +2422,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // Outer references within lateral subquery can only refer to attributes // in the containing plan of lateral subquery. Nested outer references // are not allowed. - throw new SparkIllegalArgumentException( - errorClass = "FIELD_NOT_FOUND", - messageParameters = Map( - "fieldName" -> toSQLId(nestedOuterReferences.head.prettyName), - "fields" -> plan.inputSet.map(f => toSQLId(f.name)).mkString(", ")) + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + messageParameters = Map.empty ) } res diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 4379b27b4c268..5953dbd980556 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -20,10 +20,12 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression.hasSubquery import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -125,18 +127,27 @@ object ValidateSubqueryExpression case _ => } - def checkNestedOuterReferences(expr: SubqueryExpression): Unit = { + def checkNestedOuterReferences(plan: LogicalPlan, expr: SubqueryExpression): Unit = { if ((!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) && expr.getNestedOuterAttrs.nonEmpty) { - expr.failAnalysis( - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - messageParameters = Map("objectName" -> toSQLId( - expr.getNestedOuterAttrs.head.prettyName))) + val nestedOuterAttrName = expr.getNestedOuterAttrs.flatMap{ + o => o.collect { + case a: AttributeReference => a.name + } + }.head + val output = plan.inputSet ++ expr.plan.inputSet + throw QueryCompilationErrors.unresolvedColumnError( + nestedOuterAttrName, + proposal = orderSuggestedIdentifiersBySimilarity( + nestedOuterAttrName, + candidates = output.map(attribute => attribute.qualifier :+ attribute.name).toSeq + ) + ) } } // Check if there are nested correlated subqueries in the plan. - checkNestedOuterReferences(expr) + checkNestedOuterReferences(plan, expr) // Check if there is outer attribute that cannot be found from the plan. checkOuterReference(plan, expr) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index ce81b71017264..f9d223e2b3e0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2832,15 +2832,13 @@ class SubquerySuite extends QueryTest } test("analyzer same error class") { - sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2);") - sql("CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3);") - sql("CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1))," + - " (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4));") - sql("CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2)," + - " (1, 1), (1, 3);") - - val query = "SELECT * FROM t1, LATERAL" + - " (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*, t3.* FROM t2 AS t3))" + sql("CREATE OR REPLACE TEMP VIEW tenk1 AS SELECT 1 AS unique1, 1 AS unique2") + val query = + """ + |select + | (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) + |from tenk1 o; + |""".stripMargin val res = sql(query).collect() } } From 488081399b0a59a7d0d7c086c7661a0c62ddeb4e Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 11:46:59 -0700 Subject: [PATCH 16/48] remove temporary test first --- .../scala/org/apache/spark/sql/SubquerySuite.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index f9d223e2b3e0c..e7e41f6570d3c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2830,15 +2830,4 @@ class SubquerySuite extends QueryTest Row(1, false) :: Row(2, false) :: Row(3, true) :: Nil) } } - - test("analyzer same error class") { - sql("CREATE OR REPLACE TEMP VIEW tenk1 AS SELECT 1 AS unique1, 1 AS unique2") - val query = - """ - |select - | (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) - |from tenk1 o; - |""".stripMargin - val res = sql(query).collect() - } } From fc37a5e17eed7310918b4c4e6ef8c76305896966 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 11:59:32 -0700 Subject: [PATCH 17/48] restore FunctionTableSubqueryArgumentExpression, output UNRESOLVED_COLUMN.WITH_SUGGESTION error for main query with nested outer attrs --- .../sql/catalyst/analysis/CheckAnalysis.scala | 28 +++++++++++++------ ...ctionTableSubqueryArgumentExpression.scala | 7 +++-- 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index ad40eff87d946..a5c54814244b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable - import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException @@ -28,6 +27,7 @@ import org.apache.spark.sql.catalyst.optimizer.InlineCTE import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, PLAN_EXPRESSION, UNRESOLVED_WINDOW_EXPRESSION} +import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} @@ -229,19 +229,29 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } def checkNoNestedOuterReferencesInMainQuery(plan: LogicalPlan): Unit = { + def throwUnresolvedColumnErrorForNestedOuterAttrs( + plan: LogicalPlan, expr: SubqueryExpression): Unit = { + val nestedOuterAttrName = expr.getNestedOuterAttrs.flatMap{ + o => o.collect { + case a: AttributeReference => a.name + } + }.head + val output = plan.inputSet ++ expr.plan.inputSet + throw QueryCompilationErrors.unresolvedColumnError( + nestedOuterAttrName, + proposal = orderSuggestedIdentifiersBySimilarity( + nestedOuterAttrName, + candidates = output.map(attribute => attribute.qualifier :+ attribute.name).toSeq + ) + ) + } plan.expressions.foreach { case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => - subExpr.failAnalysis( - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - messageParameters = Map("objectName" -> toSQLId( - subExpr.getNestedOuterAttrs.head.prettyName))) + throwUnresolvedColumnErrorForNestedOuterAttrs(plan, subExpr) case expr if expr.containsPattern(PLAN_EXPRESSION) => expr.collect { case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => - subExpr.failAnalysis( - errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - messageParameters = Map("objectName" -> toSQLId( - subExpr.getNestedOuterAttrs.head.prettyName))) + throwUnresolvedColumnErrorForNestedOuterAttrs(plan, subExpr) } case _ => } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index f8002063fcda6..bfe8d1ab8f27d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -91,9 +91,10 @@ case class FunctionTableSubqueryArgumentExpression( override def withNewNestedOuterAttrs( nestedOuterAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - // FunctionTableSubquery should not have nested outer attrs - assert(nestedOuterAttrs.isEmpty) - copy() + assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), + s"nestedOuterAttrs must be a subset of outerAttrs, " + + s"but got ${nestedOuterAttrs.mkString(", ")}") + copy(nestedOuterAttrs = nestedOuterAttrs) } override def toString: String = s"table-argument#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { From 9457a27d26b06f8f26129f406d7114c643e79453 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 12:33:08 -0700 Subject: [PATCH 18/48] update updateOuterReferences for nested correlation --- .../spark/sql/catalyst/analysis/Analyzer.scala | 17 ++++++++++++++++- .../analysis/ValidateSubqueryExpression.scala | 1 - 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 463db62ad4f22..a382861c6f958 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -4222,14 +4222,29 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { } } + def updateOuterReferenceInAllSubqueries( + s: SubqueryExpression, outerAliases: Seq[Alias]): SubqueryExpression = { + val planWithNestedSubqueriesRewritten = + s.plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { + // Only update the nested subqueries if they have nested outer references + // And we don't collect new outerAliases along s.plan because this rule + // will be fired multiple times for each subquery plan in the Analyzer, + // we only collect outerAliases in the outer plan each time. + case s: SubqueryExpression if s.getNestedOuterAttrs.nonEmpty => + updateOuterReferenceInAllSubqueries(s, outerAliases) + } + val newPlan = updateOuterReferenceInSubquery(planWithNestedSubqueriesRewritten, outerAliases) + s.withNewPlan(newPlan) + } + def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsWithPruning( _.containsAllPatterns(PLAN_EXPRESSION, FILTER, AGGREGATE), ruleId) { case f @ Filter(_, a: Aggregate) if f.resolved => + val outerAliases = a.aggregateExpressions collect { case a: Alias => a } f.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { case s: SubqueryExpression if s.children.nonEmpty => // Collect the aliases from output of aggregate. - val outerAliases = a.aggregateExpressions collect { case a: Alias => a } // Update the subquery plan to record the OuterReference to point to outer query plan. s.withNewPlan(updateOuterReferenceInSubquery(s.plan, outerAliases)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 5953dbd980556..21de1f7718d41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ -import org.apache.spark.sql.catalyst.expressions.SubqueryExpression.hasSubquery import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery} import org.apache.spark.sql.catalyst.plans._ From 3acaafd5e973c45c81859442beb6f0ca4a339cc9 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 13:47:55 -0700 Subject: [PATCH 19/48] scalafmt --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index a5c54814244b2..3818fce0fe53c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable + import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException @@ -27,7 +28,6 @@ import org.apache.spark.sql.catalyst.optimizer.InlineCTE import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, PLAN_EXPRESSION, UNRESOLVED_WINDOW_EXPRESSION} -import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} @@ -239,7 +239,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString val output = plan.inputSet ++ expr.plan.inputSet throw QueryCompilationErrors.unresolvedColumnError( nestedOuterAttrName, - proposal = orderSuggestedIdentifiersBySimilarity( + proposal = StringUtils.orderSuggestedIdentifiersBySimilarity( nestedOuterAttrName, candidates = output.map(attribute => attribute.qualifier :+ attribute.name).toSeq ) From 5de70ee9a6a37dadd68bf4570e318d4b1239de8f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 14:48:08 -0700 Subject: [PATCH 20/48] remove assertion as we might have duplicate column identifiers --- .../spark/sql/catalyst/analysis/ColumnResolutionHelper.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 08bc2d426628d..ef01ce3d6f6c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -218,7 +218,9 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference) } } - assert(resolvedExpressions.size <= 1) + // We use the first resolved expression here + // as the outerPlans are ordered by their depth and the + // first one is the closest to the subquery scope. resolvedExpressions.headOption } catch { case ae: AnalysisException => From 1f6f00018349bb4f610e7c1de4dd214546f26d12 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 17:17:28 -0700 Subject: [PATCH 21/48] new error type --- .../resources/error/error-conditions.json | 6 +++++ .../analysis/ValidateSubqueryExpression.scala | 22 ++++++------------- 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 1efc7bd521d08..f2829c95ea97c 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3980,6 +3980,12 @@ ], "sqlState" : "42703" }, + "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED" : { + "message" : [ + "Detected nested outer references in the subquery. Please set 'spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled' to true to enable this feature." + ], + "sqlState" : "0A000" + }, "NESTED_EXECUTE_IMMEDIATE" : { "message" : [ "Nested EXECUTE IMMEDIATE commands are not allowed. Please ensure that the SQL query provided () does not contain another EXECUTE IMMEDIATE command." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 21de1f7718d41..4dc8f246744ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -18,13 +18,13 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.StringUtils.orderSuggestedIdentifiersBySimilarity import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -126,27 +126,19 @@ object ValidateSubqueryExpression case _ => } - def checkNestedOuterReferences(plan: LogicalPlan, expr: SubqueryExpression): Unit = { + def checkNestedOuterReferences(expr: SubqueryExpression): Unit = { if ((!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) && expr.getNestedOuterAttrs.nonEmpty) { - val nestedOuterAttrName = expr.getNestedOuterAttrs.flatMap{ - o => o.collect { - case a: AttributeReference => a.name - } - }.head - val output = plan.inputSet ++ expr.plan.inputSet - throw QueryCompilationErrors.unresolvedColumnError( - nestedOuterAttrName, - proposal = orderSuggestedIdentifiersBySimilarity( - nestedOuterAttrName, - candidates = output.map(attribute => attribute.qualifier :+ attribute.name).toSeq - ) + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + messageParameters = Map( + "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) ) } } // Check if there are nested correlated subqueries in the plan. - checkNestedOuterReferences(plan, expr) + checkNestedOuterReferences(expr) // Check if there is outer attribute that cannot be found from the plan. checkOuterReference(plan, expr) From aba5e81d361515f615bdcd94296a3c32c4bd5e58 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 17:20:45 -0700 Subject: [PATCH 22/48] format new error type --- .../src/main/resources/error/error-conditions.json | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index f2829c95ea97c..2995baf536817 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3974,6 +3974,12 @@ ], "sqlState" : "42607" }, + "NESTED_EXECUTE_IMMEDIATE" : { + "message" : [ + "Nested EXECUTE IMMEDIATE commands are not allowed. Please ensure that the SQL query provided () does not contain another EXECUTE IMMEDIATE command." + ], + "sqlState" : "07501" + }, "NESTED_REFERENCES_IN_LATERAL_SUBQUERY" : { "message" : [ "Lateral subqueries can only reference attributes from their own scope or their immediate containing query. Invalid external references are not allowed." @@ -3986,12 +3992,6 @@ ], "sqlState" : "0A000" }, - "NESTED_EXECUTE_IMMEDIATE" : { - "message" : [ - "Nested EXECUTE IMMEDIATE commands are not allowed. Please ensure that the SQL query provided () does not contain another EXECUTE IMMEDIATE command." - ], - "sqlState" : "07501" - }, "NONEXISTENT_FIELD_NAME_IN_LIST" : { "message" : [ "Field(s) do(es) not exist. Available fields: " From 278777782b6c9e6ef29c65d5cfca083b6788e114 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 16 Apr 2025 17:38:26 -0700 Subject: [PATCH 23/48] try regenerate golden files --- .../analyzer-results/join-lateral.sql.out | 63 ++++--------------- .../postgreSQL/aggregates_part1.sql.out | 16 ++--- .../analyzer-results/sql-udf.sql.out | 30 +++------ .../invalid-correlation.sql.out | 18 ++---- .../scalar-subquery-predicate.sql.out | 26 +++----- .../postgreSQL/udf-aggregates_part1.sql.out | 16 ++--- .../sql-tests/results/join-lateral.sql.out | 63 ++++--------------- .../postgreSQL/aggregates_part1.sql.out | 16 ++--- .../sql-tests/results/sql-udf.sql.out | 30 +++------ .../invalid-correlation.sql.out | 18 ++---- .../scalar-subquery-predicate.sql.out | 13 +++- .../postgreSQL/udf-aggregates_part1.sql.out | 16 ++--- 12 files changed, 85 insertions(+), 240 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 390fcf5e30152..afc623d53c513 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -177,19 +177,8 @@ SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.* -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", - "sqlState" : "42704", - "messageParameters" : { - "columns" : "`c1`, `c2`", - "targetString" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 74, - "fragment" : "t1.*" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } @@ -656,40 +645,20 @@ Project [c1#x, c2#x, c1#x, m#x, m#x] -- !query SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t1`.`c1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 62, - "stopIndex" : 66, - "fragment" : "t1.c1" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } -- !query SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`c2`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 72, - "fragment" : "c2" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } @@ -739,19 +708,11 @@ SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`t1`.`c1`", - "proposal" : "`t2`.`c1`, `t2`.`c2`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 74, - "stopIndex" : 78, - "fragment" : "t1.c1" - } ] + "expression" : "spark_catalog.default.t1.c1" + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out index 0577d73ea6a3c..f6683cbca32dc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out @@ -505,17 +505,9 @@ from tenk1 o -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`o`.`unique1`", - "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 79, - "fragment" : "o.unique1" - } ] + "expression" : "o.unique1" + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out index 8a110190bb020..08a19b10fd35d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out @@ -408,18 +408,11 @@ CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`a`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 21, - "stopIndex" : 21, - "fragment" : "a" - } ] + "expression" : "foo2_2c.a" + } } @@ -428,18 +421,11 @@ CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (S -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`a`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 37, - "stopIndex" : 37, - "fragment" : "a" - } ] + "expression" : "foo2_2d.a" + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out index 95b38e1c7e0f5..ba79902831901 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out @@ -76,20 +76,12 @@ WHERE t1a IN (SELECT min(t2a) -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "input" : "\"min(t2a)\", \"t2c\"", - "missingAttributes" : "\"t2b\"", - "operator" : "!Filter t2c#x IN (list#x [t2b#x])" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 35, - "stopIndex" : 298, - "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" - } ] + "objectName" : "`t2b`", + "proposal" : "`t1`.`t1b`, `min(t2a)`, `t2`.`t2c`, `t1`.`t1a`, `t1`.`t1c`" + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 0414782fb7e67..f1acee3de386d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -861,23 +861,15 @@ HAVING max(t1b) <= (SELECT max(t2b) WHERE t2c = t1c GROUP BY t2c) -- !query analysis -Project [t1a#x] -+- Filter (max(t1b)#x <= scalar-subquery#x [t1c#x]) - : +- Aggregate [t2c#x], [max(t2b#x) AS max(t2b)#x] - : +- Filter (t2c#x = outer(t1c#x)) - : +- SubqueryAlias t2 - : +- View (`t2`, [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]) - : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] - : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] - : +- SubqueryAlias t2 - : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] - +- Aggregate [t1a#x, t1c#x], [t1a#x, max(t1b#x) AS max(t1b)#x, t1c#x] - +- SubqueryAlias t1 - +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) - +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] - +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] - +- SubqueryAlias t1 - +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1c`", + "proposal" : "`t1`.`t1c`, `t1`.`t1a`, `max(t1b)`, `t2`.`t2c`, `t2`.`t2a`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out index ec5fb2058447e..eafdf4c2ae70d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -496,17 +496,9 @@ from tenk1 o -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`o`.`unique1`", - "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 75, - "stopIndex" : 83, - "fragment" : "o.unique1" - } ] + "expression" : "o.unique1" + } } diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index 11bafb2cf63c9..ec46ac9a1b5b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -131,19 +131,8 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", - "sqlState" : "42704", - "messageParameters" : { - "columns" : "`c1`, `c2`", - "targetString" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 74, - "fragment" : "t1.*" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } @@ -477,20 +466,10 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t1`.`c1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 62, - "stopIndex" : 66, - "fragment" : "t1.c1" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } @@ -499,20 +478,10 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`c2`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 72, - "fragment" : "c2" - } ] + "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + "sqlState" : "42703" } @@ -541,19 +510,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`t1`.`c1`", - "proposal" : "`t2`.`c1`, `t2`.`c2`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 74, - "stopIndex" : 78, - "fragment" : "t1.c1" - } ] + "expression" : "spark_catalog.default.t1.c1" + } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index 42cf942add486..ec25271e6b54e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -509,17 +509,9 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`o`.`unique1`", - "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 71, - "stopIndex" : 79, - "fragment" : "o.unique1" - } ] + "expression" : "o.unique1" + } } diff --git a/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out index 184489190d0e5..79d612d972ac0 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out @@ -327,18 +327,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`a`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 21, - "stopIndex" : 21, - "fragment" : "a" - } ] + "expression" : "foo2_2c.a" + } } @@ -349,18 +342,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`a`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 37, - "stopIndex" : 37, - "fragment" : "a" - } ] + "expression" : "foo2_2d.a" + } } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 93c60fd49c582..56e4dab2e7cbc 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -71,20 +71,12 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "input" : "\"min(t2a)\", \"t2c\"", - "missingAttributes" : "\"t2b\"", - "operator" : "!Filter t2c#x IN (list#x [t2b#x])" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 35, - "stopIndex" : 298, - "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" - } ] + "objectName" : "`t2b`", + "proposal" : "`t1`.`t1b`, `min(t2a)`, `t2`.`t2c`, `t1`.`t1a`, `t1`.`t1c`" + } } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 2460c2452ea56..9990452b4393c 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -437,10 +437,17 @@ HAVING max(t1b) <= (SELECT max(t2b) WHERE t2c = t1c GROUP BY t2c) -- !query schema -struct +struct<> -- !query output -val1b -val1c +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1c`", + "proposal" : "`t1`.`t1c`, `t1`.`t1a`, `max(t1b)`, `t2`.`t2c`, `t2`.`t2a`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 6e955c1e4f1ee..a425846772d2f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -500,17 +500,9 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", "messageParameters" : { - "objectName" : "`o`.`unique1`", - "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 75, - "stopIndex" : 83, - "fragment" : "o.unique1" - } ] + "expression" : "o.unique1" + } } From e51ce6172b14152130b41769f4c360dd9e5e327f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Thu, 17 Apr 2025 18:33:33 -0700 Subject: [PATCH 24/48] update ResolveSubquerySuite --- .../sql/catalyst/analysis/ResolveSubquerySuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 86718ee434311..9b113275598f5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -125,8 +125,8 @@ class ResolveSubquerySuite extends AnalysisTest { // TODO: support accessing columns from outer outer query. assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), - "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - Map("objectName" -> "`a`") + "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + Map.empty ) } @@ -146,14 +146,14 @@ class ResolveSubquerySuite extends AnalysisTest { // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), - "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - Map("objectName" -> "`t1`.`a`") + "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + Map.empty ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), - "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - Map("objectName" -> "`a`") + "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", + Map.empty ) } From 27c909cd9abd367844872c0b4471ecaf2cfc0095 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 11:26:32 -0700 Subject: [PATCH 25/48] fix ResolveSubquerySuite.scala --- .../spark/sql/catalyst/analysis/ResolveSubquerySuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 9b113275598f5..349f152420fe0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -146,8 +146,8 @@ class ResolveSubquerySuite extends AnalysisTest { // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), - "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - Map.empty + "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + Map("objectName" -> "`t1`.`a`") ) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorCondition( From a318f1e0cacf0bbe7998e858242b18244a39869b Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 11:37:38 -0700 Subject: [PATCH 26/48] restore same behavior for lateral subqueyr --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 5 ++++- .../sql/catalyst/analysis/ResolveSubquerySuite.scala | 8 ++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a382861c6f958..7357204245aa5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2323,7 +2323,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor outer: LogicalPlan)( f: (LogicalPlan, Seq[Expression]) => SubqueryExpression): SubqueryExpression = { val outerPlanContext = AnalysisContext.get.outerPlans - val newSubqueryPlan = if (outerPlanContext.isDefined) { + val newSubqueryPlan = if (outerPlanContext.isDefined && + // We don't allow lateral subquery having nested correlation + !e.isInstanceOf[LateralSubquery] + ) { // The previous outerPlanContext contains resolved nested outer plans // and unresolved direct outer plan. Append the current outer plan into // new outerPlanContext as current outer is guaranteed to be resolved. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 349f152420fe0..86718ee434311 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -125,8 +125,8 @@ class ResolveSubquerySuite extends AnalysisTest { // TODO: support accessing columns from outer outer query. assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), - "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - Map.empty + "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + Map("objectName" -> "`a`") ) } @@ -152,8 +152,8 @@ class ResolveSubquerySuite extends AnalysisTest { // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorCondition( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), - "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - Map.empty + "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + Map("objectName" -> "`a`") ) } From 154b1dbbc0d4c695b79616c94ba779b40f69b297 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 11:56:14 -0700 Subject: [PATCH 27/48] restore error msg for lateral subquery --- .../resources/error/error-conditions.json | 6 --- .../sql/catalyst/analysis/Analyzer.scala | 10 +--- .../analyzer-results/join-lateral.sql.out | 47 +++++++++++++++---- .../sql-tests/results/join-lateral.sql.out | 47 +++++++++++++++---- 4 files changed, 79 insertions(+), 31 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 2995baf536817..30f56fa8d1035 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3980,12 +3980,6 @@ ], "sqlState" : "07501" }, - "NESTED_REFERENCES_IN_LATERAL_SUBQUERY" : { - "message" : [ - "Lateral subqueries can only reference attributes from their own scope or their immediate containing query. Invalid external references are not allowed." - ], - "sqlState" : "42703" - }, "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED" : { "message" : [ "Detected nested outer references in the subquery. Please set 'spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled' to true to enable this feature." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7357204245aa5..fb37ccc12dc38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2421,15 +2421,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case s @ LateralSubquery(sub, _, _, exprId, _, _) if !sub.resolved => val res = resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) val nestedOuterReferences = getNestedOuterReferences(res, plan) - if (nestedOuterReferences.nonEmpty) { - // Outer references within lateral subquery can only refer to attributes - // in the containing plan of lateral subquery. Nested outer references - // are not allowed. - throw new AnalysisException( - errorClass = "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - messageParameters = Map.empty - ) - } + assert(nestedOuterReferences.isEmpty) res case a: FunctionTableSubqueryArgumentExpression if !a.plan.resolved => resolveSubQuery(a, outer)( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index afc623d53c513..f8e88a57bf426 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -177,8 +177,19 @@ SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.* -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`c1`, `c2`", + "targetString" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 74, + "fragment" : "t1.*" + } ] } @@ -645,20 +656,40 @@ Project [c1#x, c2#x, c1#x, m#x, m#x] -- !query SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "t1.c1" + } ] } -- !query SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 72, + "fragment" : "c2" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index ec46ac9a1b5b7..f8074eb805b4f 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -131,8 +131,19 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`c1`, `c2`", + "targetString" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 74, + "fragment" : "t1.*" + } ] } @@ -466,10 +477,20 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "t1.c1" + } ] } @@ -478,10 +499,20 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "NESTED_REFERENCES_IN_LATERAL_SUBQUERY", - "sqlState" : "42703" + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 72, + "fragment" : "c2" + } ] } From f9e2b23c6f12936844e7c6d8aeebc1a18a85b326 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 13:44:28 -0700 Subject: [PATCH 28/48] update subquery's nested outer references in resolveAggregateFunction for subquery in having clause --- .../sql/catalyst/analysis/Analyzer.scala | 20 +++++- .../scalar-subquery-predicate.sql.out | 26 ++++--- .../scalar-subquery-predicate.sql.out | 13 +--- .../org/apache/spark/sql/SubquerySuite.scala | 70 +++++++++++++++++++ 4 files changed, 108 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fb37ccc12dc38..16a1c54b94258 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2872,11 +2872,27 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor * and group by expressions from them. */ object ResolveAggregateFunctions extends Rule[LogicalPlan] { + def UpdateSubqueryOuterReferences(expression: Expression, aggregate: Aggregate): Expression = { + expression.transformUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case sub: SubqueryExpression if sub.getNestedOuterAttrs.nonEmpty => + val newNestedOuterAttrs = + sub.getNestedOuterAttrs.filter( outerExpr => outerExpr match { + case a: AttributeReference => !aggregate.outputSet.contains(a) + case _ => true + }) + sub.withNewNestedOuterAttrs(newNestedOuterAttrs) + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning( _.containsPattern(AGGREGATE), ruleId) { case UnresolvedHaving(cond, agg: Aggregate) if agg.resolved && cond.resolved => resolveOperatorWithAggregate(Seq(cond), agg, (newExprs, newChild) => { - val newCond = newExprs.head + // Update the subquery in having clause as the aggregate output may be changed + // after the resolution. Some outer references being marked as nested outer + // references might be removed. + val headCond = newExprs.head + val newCond = UpdateSubqueryOuterReferences(headCond, newChild) if (newCond.resolved) { Filter(newCond, newChild) } else { @@ -4242,7 +4258,7 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { // Collect the aliases from output of aggregate. // Update the subquery plan to record the OuterReference to point to outer query plan. s.withNewPlan(updateOuterReferenceInSubquery(s.plan, outerAliases)) - } + } } } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index f1acee3de386d..0414782fb7e67 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -861,15 +861,23 @@ HAVING max(t1b) <= (SELECT max(t2b) WHERE t2c = t1c GROUP BY t2c) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t1c`", - "proposal" : "`t1`.`t1c`, `t1`.`t1a`, `max(t1b)`, `t2`.`t2c`, `t2`.`t2a`" - } -} +Project [t1a#x] ++- Filter (max(t1b)#x <= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [max(t2b#x) AS max(t2b)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Aggregate [t1a#x, t1c#x], [t1a#x, max(t1b#x) AS max(t1b)#x, t1c#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 9990452b4393c..2460c2452ea56 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -437,17 +437,10 @@ HAVING max(t1b) <= (SELECT max(t2b) WHERE t2c = t1c GROUP BY t2c) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`t1c`", - "proposal" : "`t1`.`t1c`, `t1`.`t1a`, `max(t1b)`, `t2`.`t2c`, `t2`.`t2a`" - } -} +val1b +val1c -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index e7e41f6570d3c..0689f860d0f88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2830,4 +2830,74 @@ class SubquerySuite extends QueryTest Row(1, false) :: Row(2, false) :: Row(3, true) :: Nil) } } + + test("doubt") { + // scalastyle:off + sql( + """ + |create temporary view t1 as select * from values + | ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + | ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + | ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + | ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + | ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + | ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + | ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + | ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + | ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + | ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + | ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + | ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + | as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + |""".stripMargin) + sql( + """ + |create temporary view t2 as select * from values + | ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + | ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + | ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + | ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + | ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + | ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + | ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + | ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + | ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + | ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + | ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + | as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + |""".stripMargin) + sql( + """ + |create temporary view t3 as select * from values + | ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + | ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + | ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + | ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + | ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + | ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + | ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + | ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + | ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + | ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + | ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + | as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + |""".stripMargin) + val query = + """ + |SELECT t1a + |FROM t1 + |GROUP BY t1a, t1c + |HAVING max(t1b) <= (SELECT max(t2b) + | FROM t2 + | WHERE t2c = t1c + | GROUP BY t2c) + |""".stripMargin + withSQLConf("spark.sql.planChangeLog.level" -> "info") { + sql(query).collect() + } + // scalastyle:on + } } From c428df8d338300882e38e9669251c84a86e16828 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 13:50:26 -0700 Subject: [PATCH 29/48] remove temporary tests --- .../org/apache/spark/sql/SubquerySuite.scala | 70 ------------------- 1 file changed, 70 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 0689f860d0f88..e7e41f6570d3c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2830,74 +2830,4 @@ class SubquerySuite extends QueryTest Row(1, false) :: Row(2, false) :: Row(3, true) :: Nil) } } - - test("doubt") { - // scalastyle:off - sql( - """ - |create temporary view t1 as select * from values - | ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), - | ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - | ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - | ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - | ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - | ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), - | ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), - | ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - | ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - | ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - | ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - | ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') - | as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); - |""".stripMargin) - sql( - """ - |create temporary view t2 as select * from values - | ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - | ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - | ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - | ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), - | ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - | ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - | ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - | ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - | ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - | ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - | ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) - | as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); - |""".stripMargin) - sql( - """ - |create temporary view t3 as select * from values - | ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - | ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - | ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - | ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - | ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - | ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - | ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - | ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - | ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), - | ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), - | ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - | ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') - | as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); - |""".stripMargin) - val query = - """ - |SELECT t1a - |FROM t1 - |GROUP BY t1a, t1c - |HAVING max(t1b) <= (SELECT max(t2b) - | FROM t2 - | WHERE t2c = t1c - | GROUP BY t2c) - |""".stripMargin - withSQLConf("spark.sql.planChangeLog.level" -> "info") { - sql(query).collect() - } - // scalastyle:on - } } From 9932efe2f6a6782c57e4e871f5c781d6755f83b2 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 16:13:05 -0700 Subject: [PATCH 30/48] restore missing_attributes error --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 77 +++++++++++++------ .../invalid-correlation.sql.out | 18 +++-- .../invalid-correlation.sql.out | 18 +++-- 4 files changed, 81 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 722ac1696fc75..27f949b5816cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2898,7 +2898,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor * and group by expressions from them. */ object ResolveAggregateFunctions extends Rule[LogicalPlan] { - def UpdateSubqueryOuterReferences(expression: Expression, aggregate: Aggregate): Expression = { + def updateSubqueryOuterReferences(expression: Expression, aggregate: Aggregate): Expression = { expression.transformUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) { case sub: SubqueryExpression if sub.getNestedOuterAttrs.nonEmpty => val newNestedOuterAttrs = @@ -2928,7 +2928,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // after the resolution. Some outer references being marked as nested outer // references might be removed. val headCond = newExprs.head - val newCond = UpdateSubqueryOuterReferences(headCond, newChild) + val newCond = updateSubqueryOuterReferences(headCond, newChild) if (newCond.resolved) { Filter(newCond, newChild) } else { @@ -4286,7 +4286,7 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { case s: SubqueryExpression if s.children.nonEmpty => // Collect the aliases from output of aggregate. // Update the subquery plan to record the OuterReference to point to outer query plan. - s.withNewPlan(updateOuterReferenceInSubquery(s.plan, outerAliases)) + updateOuterReferenceInAllSubqueries(s, outerAliases) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 0fe9d67861ff4..2e0c278db1402 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -229,35 +229,62 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } def checkNoNestedOuterReferencesInMainQuery(plan: LogicalPlan): Unit = { - def throwUnresolvedColumnErrorForNestedOuterAttrs( - plan: LogicalPlan, expr: SubqueryExpression): Unit = { - val nestedOuterAttrName = expr.getNestedOuterAttrs.flatMap{ - o => o.collect { - case a: AttributeReference => a.name - } - }.head - val output = plan.inputSet ++ expr.plan.inputSet - throw QueryCompilationErrors.unresolvedColumnError( - nestedOuterAttrName, - proposal = StringUtils.orderSuggestedIdentifiersBySimilarity( - nestedOuterAttrName, - candidates = output.map(attribute => attribute.qualifier :+ attribute.name).toSeq - ) - ) + def hasNestedOuterAttrsInSubqueryExpression(expr: Expression): Boolean = { + expr.exists { + case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => true + case _ => false + } } - plan.expressions.foreach { - case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => - throwUnresolvedColumnErrorForNestedOuterAttrs(plan, subExpr) - case expr if expr.containsPattern(PLAN_EXPRESSION) => - expr.collect { + + def getNestedOuterAttrsFromSubqueryExpression( + plan: LogicalPlan): Seq[(SubqueryExpression, AttributeSet)] = { + val res = plan.expressions.flatMap { + expr => expr.collect { case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => - throwUnresolvedColumnErrorForNestedOuterAttrs(plan, subExpr) + (subExpr, subExpr.getNestedOuterAttrs) } - case _ => + } + res.map { + case (subExpr, nestedOuterExprs) => + val attrs = nestedOuterExprs.collect { + case a: AttributeReference => a + } + (subExpr, AttributeSet(attrs)) + } } - plan.children.foreach { - case p: LogicalPlan if p.containsPattern(PLAN_EXPRESSION) => - checkNoNestedOuterReferencesInMainQuery(p) + def findFirstOccurence( + plan: LogicalPlan, + nestedOuterAttrs: AttributeSet, + operator: LogicalPlan): (LogicalPlan, AttributeSet) = { + val firstOccuredOperator = operator + plan.foreach { + case p if p.expressions.exists(hasNestedOuterAttrsInSubqueryExpression) => + val res = getNestedOuterAttrsFromSubqueryExpression(p) + res.find(_._2.intersect(nestedOuterAttrs).nonEmpty) match { + case Some((subExpr, nestedOuterAttrsInP)) => + return findFirstOccurence(subExpr.plan, + nestedOuterAttrsInP.intersect(nestedOuterAttrs), p) + case None => // Do nothing + } + case _ => // Do nothing + } + (firstOccuredOperator, nestedOuterAttrs) + } + def throwUnresolvedColumnErrorForNestedOuterAttrs(plan: LogicalPlan): Unit = { + val (subExpr, nestedOuterAttrs) = getNestedOuterAttrsFromSubqueryExpression(plan).head + val (operator, missingInput) = findFirstOccurence(subExpr.plan, nestedOuterAttrs, plan) + operator.failAnalysis( + errorClass = "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + messageParameters = Map( + "missingAttributes" -> missingInput.toSeq.map(attr => toSQLExpr(attr)).mkString(", "), + "input" -> operator.inputSet.map(attr => toSQLExpr(attr)).mkString(", "), + "operator" -> operator.simpleString(SQLConf.get.maxToStringFields) + ) + ) + } + plan.foreach { + case p: LogicalPlan if p.expressions.exists(hasNestedOuterAttrsInSubqueryExpression) => + throwUnresolvedColumnErrorForNestedOuterAttrs(p) case _ => } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out index ba79902831901..95b38e1c7e0f5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out @@ -76,12 +76,20 @@ WHERE t1a IN (SELECT min(t2a) -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", "messageParameters" : { - "objectName" : "`t2b`", - "proposal" : "`t1`.`t1b`, `min(t2a)`, `t2`.`t2c`, `t1`.`t1a`, `t1`.`t1c`" - } + "input" : "\"min(t2a)\", \"t2c\"", + "missingAttributes" : "\"t2b\"", + "operator" : "!Filter t2c#x IN (list#x [t2b#x])" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 298, + "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 56e4dab2e7cbc..93c60fd49c582 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -71,12 +71,20 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", "messageParameters" : { - "objectName" : "`t2b`", - "proposal" : "`t1`.`t1b`, `min(t2a)`, `t2`.`t2c`, `t1`.`t1a`, `t1`.`t1c`" - } + "input" : "\"min(t2a)\", \"t2c\"", + "missingAttributes" : "\"t2b\"", + "operator" : "!Filter t2c#x IN (list#x [t2b#x])" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 298, + "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" + } ] } From 508065de04e8802e531643971564fcf08e03677b Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 17:04:34 -0700 Subject: [PATCH 31/48] add test --- .../nestedcorrelation/combined-subquery.sql | 12 ++ .../nestedcorrelation/in-subquery.sql | 22 ++++ .../nestedcorrelation/lateral-subquery.sql | 23 ++++ .../nestedcorrelation/scalar-subquery.sql | 109 ++++++++++++++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 4 + 5 files changed, 170 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql new file mode 100644 index 0000000000000..58bd133e205d3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql @@ -0,0 +1,12 @@ +--ONLY_IF spark + +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; + +DROP TABLE IF EXISTS tbl; +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP) + +INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') + +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; + +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql new file mode 100644 index 0000000000000..d7c4af77ebf38 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql @@ -0,0 +1,22 @@ +--ONLY_IF spark + +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; + +DROP TABLE IF EXISTS tbl_ProductSales; +DROP TABLE IF EXISTS another_T; +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int); +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT); +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100); +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888); + +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; + +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1; + +SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; + +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1; + +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> ANY(SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1; + +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql new file mode 100644 index 0000000000000..b6b4d2f0a2668 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql @@ -0,0 +1,23 @@ +--ONLY_IF spark + +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(ps_supplycost INT, n_name INT); + +SELECT NULL +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4; + +SELECT * +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql new file mode 100644 index 0000000000000..e901a5227e4c4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -0,0 +1,109 @@ +--ONLY_IF spark +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; + +DROP TABLE IF EXISTS t0; +CREATE TABLE t0(c0 INT); + +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)); + +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)); + +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)); + +DROP TABLE IF EXISTS t; +CREATE TABLE t(ps_supplycost INT, n_name INT); + +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_2.n_name AS c1)))); + +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1)))); + +DROP TABLE IF EXISTS integers; +CREATE TABLE integers(i INTEGER); +INSERT INTO integers VALUES (1), (2), (3), (NULL); + +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM integers i2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM integers) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5)) FROM integers i4)) FROM integers i3)) FROM integers i2) AS j FROM integers i1 GROUP BY i ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i))) AS j FROM integers i1 ORDER BY i; + +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3) FROM integers i2) FROM integers i1 ORDER BY 1; + +SELECT i, (SELECT SUM(s1.i) FROM integers s1 INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i; + +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers) FROM integers i2) FROM integers i1 GROUP BY i ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s1.i) FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i=i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i<>i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT i=ANY(SELECT i FROM integers WHERE i=s1.i) FROM integers s1 WHERE i=i1.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM integers i1 ORDER BY i; + +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM integers i1 ORDER BY i; + +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM integers i3) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1; + +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM integers i3) FROM integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM integers i1 ORDER BY 1; \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 575a4ae69d1a9..ddc675e2f67e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -471,6 +471,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}cte.sql")) { CTETestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith( + s"$inputFilePath${File.separator}subquery${File.separator}nestedcorrelation" + )) { + AnalyzerTestCase(testCaseName, absPath, resultFile) :: Nil } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil } From 4faca483a5c4415ec3e361fedab0b7b27c49b7e5 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 18 Apr 2025 17:39:54 -0700 Subject: [PATCH 32/48] generate test --- .../nestedcorrelation/combined-subquery.sql | 4 +- .../nestedcorrelation/scalar-subquery.sql | 85 +- .../sql-tests/inputs/to_from_avro.sql | 1 + .../combined-subquery.sql.out | 50 ++ .../nestedcorrelation/in-subquery.sql.out | 160 ++++ .../lateral-subquery.sql.out | 70 ++ .../nestedcorrelation/scalar-subquery.sql.out | 840 ++++++++++++++++++ .../sql-tests/results/to_from_avro.sql.out | 72 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 2 + 9 files changed, 1208 insertions(+), 76 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql index 58bd133e205d3..01341d366c861 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql @@ -3,9 +3,9 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; DROP TABLE IF EXISTS tbl; -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP) +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP); -INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') +INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00'); SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql index e901a5227e4c4..1c6dd5bddb1c3 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -1,9 +1,6 @@ --ONLY_IF spark set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; -DROP TABLE IF EXISTS t0; -CREATE TABLE t0(c0 INT); - SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)); SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)); @@ -28,82 +25,82 @@ FROM (SELECT ref_5.ps_supplycost AS c0, ref_2.n_name AS c1)))); -DROP TABLE IF EXISTS integers; -CREATE TABLE integers(i INTEGER); -INSERT INTO integers VALUES (1), (2), (3), (NULL); +DROP TABLE IF EXISTS table_integers; +CREATE TABLE table_integers(i INTEGER); +INSERT INTO table_integers VALUES (1), (2), (3), (NULL); -SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM integers i2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM table_integers i2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM integers) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM table_integers) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5)) FROM integers i4)) FROM integers i3)) FROM integers i2) AS j FROM integers i1 GROUP BY i ORDER BY i; +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5)) FROM table_integers i4)) FROM table_integers i3)) FROM table_integers i2) AS j FROM table_integers i1 GROUP BY i ORDER BY i; -SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i))) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i; -SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3) FROM integers i2) FROM integers i1 ORDER BY 1; +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3) FROM table_integers i2) FROM table_integers i1 ORDER BY 1; -SELECT i, (SELECT SUM(s1.i) FROM integers s1 INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i; -SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers) FROM integers i2) FROM integers i1 GROUP BY i ORDER BY i; +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers) FROM table_integers i2) FROM table_integers i1 GROUP BY i ORDER BY i; -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s1.i) FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 LEFT OUTER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i=i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i<>i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT i=ANY(SELECT i FROM integers WHERE i=s1.i) FROM integers s1 WHERE i=i1.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT i=ANY(SELECT i FROM table_integers WHERE i=s1.i) FROM table_integers s1 WHERE i=i1.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM integers i1 ORDER BY i; +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i; -SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM integers i3) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1; +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM table_integers i3) FROM table_integers i2 LIMIT 1) FROM table_integers i1 ORDER BY 1; -SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM integers i3) FROM integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM integers i1 ORDER BY 1; \ No newline at end of file +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM table_integers i3) FROM table_integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM table_integers i1 ORDER BY 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/to_from_avro.sql b/sql/core/src/test/resources/sql-tests/inputs/to_from_avro.sql index 12541ff26e24e..8d6325906df5c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/to_from_avro.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/to_from_avro.sql @@ -1,4 +1,5 @@ -- Create some temporary test data. +drop table if exists t; create table t as select named_struct('u', named_struct('member0', member0, 'member1', member1)) as s from values (1, null), (null, 'a') tab(member0, member1); diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out new file mode 100644 index 0000000000000..ffc61aba4775b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -0,0 +1,50 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS tbl +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl + + +-- !query +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP) + +INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') + +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'INSERT'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`tbl`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 17, + "fragment" : "tbl" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out new file mode 100644 index 0000000000000..2d54d297e9ac3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out @@ -0,0 +1,160 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS tbl_ProductSales +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales + + +-- !query +DROP TABLE IF EXISTS another_T +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T + + +-- !query +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false + + +-- !query +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false + + +-- !query +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] ++- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] ++- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 129, + "stopIndex" : 144, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "max((outer(t1.col1) + t3.col4))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 116, + "fragment" : "MAX(t1.col1 + t3.col4)" + } ] +} + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> ANY(SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 58, + "fragment" : "SELECT MAX(col7)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out new file mode 100644 index 0000000000000..af38633f19c9a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out @@ -0,0 +1,70 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t(ps_supplycost INT, n_name INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT NULL +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 187, + "stopIndex" : 198, + "fragment" : "ref_2.n_name" + } ] +} + + +-- !query +SELECT * +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 184, + "stopIndex" : 195, + "fragment" : "ref_2.n_name" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out new file mode 100644 index 0000000000000..5cfd1b73ab9f0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -0,0 +1,840 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS t0 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t0 + + +-- !query +CREATE TABLE t0(c0 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t0`, false + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"c0\"", + "operator" : "!Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 57, + "fragment" : "SELECT (SELECT c0)" + } ] +} + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"c0 ASC NULLS FIRST\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 57, + "stopIndex" : 67, + "fragment" : "ORDER BY c0" + } ] +} + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"outer(t0.c0)\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 64, + "fragment" : "c0" + } ] +} + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t(ps_supplycost INT, n_name INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 124, + "fragment" : "ref_2.n_name" + } ] +} + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_5`.`ps_supplycost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 131, + "fragment" : "ref_5.ps_supplycost" + } ] +} + + +-- !query +DROP TABLE IF EXISTS integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.integers + + +-- !query +CREATE TABLE integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`integers`, false + + +-- !query +INSERT INTO integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/integers], Append, `spark_catalog`.`default`.`integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 42, + "fragment" : "SELECT (SELECT 42+i1.i)+42+i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 56, + "fragment" : "SELECT (SELECT 42+i1.i)++i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM integers i2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"(i + sum(i)) AS `(outer(i1.i) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 40, + "fragment" : "SELECT i1.i+SUM(i2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM integers) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(i)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 41, + "fragment" : "(SELECT 42+i1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5)) FROM integers i4)) FROM integers i3)) FROM integers i2) AS j FROM integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((((sum(i) + sum(i)) + sum(i)) + sum(i)) + sum(i)) AS `((((sum(i) + sum(outer(i4.i))) + sum(outer(i3.i))) + sum(outer(i2.i))) + sum(outer(i1.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 108, + "fragment" : "SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5" + } ] +} + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i))) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 106, + "fragment" : "SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i)" + } ] +} + + +-- !query +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3) FROM integers i2) FROM integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(outer(i1.i)) + sum(outer(i2.i))) + sum(i))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 69, + "fragment" : "SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM integers s1 INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Join Inner, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.integers\n: +- Relation spark_catalog.default.integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 108, + "fragment" : "INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" + } ] +} + + +-- !query +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers) FROM integers i2) FROM integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(i) + sum(outer(i1.i))) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 74, + "fragment" : "SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Join LeftOuter, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.integers\n: +- Relation spark_catalog.default.integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 113, + "fragment" : "LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i=i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.integers + : : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.integers + : +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.integers + +- Relation spark_catalog.default.integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i<>i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter NOT (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.integers + : : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.integers + : +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.integers + +- Relation spark_catalog.default.integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 97, + "fragment" : "SELECT i FROM integers WHERE i=i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter NOT (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 98, + "fragment" : "SELECT i FROM integers WHERE i<>i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'WHEN'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter (i#x = outer(i#x)) + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.integers + : +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.integers + +- Relation spark_catalog.default.integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT i=ANY(SELECT i FROM integers WHERE i=s1.i) FROM integers s1 WHERE i=i1.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'i'", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'WHEN'", + "hint" : ": missing ')'" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.integers + : : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.integers + : +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.integers + +- Relation spark_catalog.default.integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 49, + "fragment" : "SELECT (SELECT 42+i1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [scalar-subquery#x [i#x] AS k#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 62, + "fragment" : "SELECT (SELECT 42+i1.i) AS k" + } ] +} + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "", + "missingAttributes" : "\"i\"", + "operator" : "!Project [scalar-subquery#x [i#x] AS k#x]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 62, + "fragment" : "SELECT (SELECT 42+i1.i) AS k" + } ] +} + + +-- !query +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [outer(i#x) IN (1,2,3,4,5,6,7,8) AS (outer(i1.i) IN (1, 2, 3, 4, 5, 6, 7, 8))#x] + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.integers + +- Relation spark_catalog.default.integers[i#x] parquet + + +-- !query +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM integers i3) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "covar_pop(CAST(outer(i1.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 44, + "fragment" : "COVAR_POP(i1.i, i3.i)" + } ] +} + + +-- !query +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM integers i3) FROM integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "covar_pop(CAST(outer(i2.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 44, + "fragment" : "COVAR_POP(i2.i, i3.i)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out b/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out index a94175b1df39a..6d2aaaa8e0725 100644 --- a/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out @@ -6,7 +6,14 @@ create table t as -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t`" + } +} -- !query @@ -33,19 +40,18 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", - "sqlState" : "42K09", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "hint" : "", - "msg" : "The second argument of the FROM_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value from AVRO format", - "sqlExpr" : "\"from_avro(s, 42, map())\"" + "objectName" : "`s`", + "proposal" : "`n_name`, `ps_supplycost`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "from_avro(s, 42, map())" + "startIndex" : 18, + "stopIndex" : 18, + "fragment" : "s" } ] } @@ -57,19 +63,18 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", - "sqlState" : "42K09", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "hint" : "", - "msg" : "The third argument of the FROM_AVRO SQL function must be a constant map of strings to strings containing the options to use for converting the value from AVRO format", - "sqlExpr" : "\"from_avro(s, variablereference(system.session.avro_schema='{ \"type\": \"record\", \"name\": \"struct\", \"fields\": [{ \"name\": \"u\", \"type\": [\"int\",\"string\"] }] }'), 42)\"" + "objectName" : "`s`", + "proposal" : "`n_name`, `ps_supplycost`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 8, - "stopIndex" : 36, - "fragment" : "from_avro(s, avro_schema, 42)" + "startIndex" : 18, + "stopIndex" : 18, + "fragment" : "s" } ] } @@ -81,19 +86,18 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", - "sqlState" : "42K09", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "hint" : "", - "msg" : "The second argument of the TO_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value to AVRO format", - "sqlExpr" : "\"to_avro(s, 42)\"" + "objectName" : "`s`", + "proposal" : "`n_name`, `ps_supplycost`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 8, - "stopIndex" : 21, - "fragment" : "to_avro(s, 42)" + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "s" } ] } @@ -103,13 +107,21 @@ select to_avro(s, avro_schema) as result from t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "AVRO_NOT_LOADED_SQL_FUNCTIONS_UNUSABLE", - "sqlState" : "22KD3", + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "functionName" : "TO_AVRO" - } + "objectName" : "`s`", + "proposal" : "`n_name`, `ps_supplycost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 16, + "fragment" : "s" + } ] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index ddc675e2f67e8..49506f746eedb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -484,6 +484,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper case _: UDAFTestCase => // Skip creating analyzer test cases for UDAF tests as they are hard to update locally. Seq(test) + case _: AnalyzerTest => + Seq(test) case _ => Seq( test, From 9a31a2feaca5e333fdae108278cb4a86a2a55c05 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Sun, 20 Apr 2025 20:53:39 -0700 Subject: [PATCH 33/48] test --- .../analysis/ValidateSubqueryExpression.scala | 8 +- .../sql/catalyst/expressions/subquery.scala | 3 +- .../invalid-correlation.sql.out | 2 +- .../analyzer-results/to_from_avro.sql.out | 7 + .../nestedcorrelation/combined-subquery.sql | 6 +- .../nestedcorrelation/exists-subquery.sql | 94 ++ .../nestedcorrelation/in-subquery.sql | 22 - .../nestedcorrelation/lateral-subquery.sql | 3 +- .../nestedcorrelation/scalar-subquery.sql | 80 +- .../subquery-not-supported.sql | 85 ++ .../invalid-correlation.sql.out | 2 +- .../combined-subquery.sql.out | 48 +- .../nestedcorrelation/exists-subquery.sql.out | 406 ++++++++ .../nestedcorrelation/in-subquery.sql.out | 160 ---- .../nestedcorrelation/scalar-subquery.sql.out | 879 +++--------------- .../subquery-not-supported.sql.out | 568 +++++++++++ .../sql-tests/results/to_from_avro.sql.out | 80 +- 17 files changed, 1374 insertions(+), 1079 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 4dc8f246744ed..71aeb7d177488 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -112,9 +112,13 @@ object ValidateSubqueryExpression case f: Filter => if (hasOuterReferences(expr.plan)) { expr.plan.expressions.foreach(_.foreachUp { - case o: OuterReference => + case o@OuterReference(a) => p.children.foreach(e => - if (!e.output.exists(_.exprId == o.exprId)) { + if (!e.output.exists(_.exprId == o.exprId) && + !expr.getNestedOuterAttrs.contains(a)) { + // If the outer reference is not found in the children plan, + // it should be a nested outer reference. Otherwise, it is + // invalid. o.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "CORRELATED_COLUMN_NOT_FOUND", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 17aab1db08f6e..c89bf528dabee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -84,7 +84,8 @@ abstract class SubqueryExpression( hint: Option[HintInfo]) extends PlanExpression[LogicalPlan] { override lazy val resolved: Boolean = childrenResolved && plan.resolved override lazy val references: AttributeSet = - AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) + AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) -- + AttributeSet.fromAttributeSets(nestedOuterAttrs.map(_.references)) override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression def withNewOuterAttrs(outerAttrs: Seq[Expression]): SubqueryExpression diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out index 95b38e1c7e0f5..1f5db69fdb06b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out @@ -81,7 +81,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "messageParameters" : { "input" : "\"min(t2a)\", \"t2c\"", "missingAttributes" : "\"t2b\"", - "operator" : "!Filter t2c#x IN (list#x [t2b#x])" + "operator" : "Filter t2c#x IN (list#x [t2b#x])" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/to_from_avro.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/to_from_avro.sql.out index 8275e4f1c0ff0..c9bad43cabd02 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/to_from_avro.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/to_from_avro.sql.out @@ -1,4 +1,11 @@ -- Automatically generated by SQLQueryTestSuite +-- !query +drop table if exists t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + -- !query create table t as select named_struct('u', named_struct('member0', member0, 'member1', member1)) as s diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql index 01341d366c861..a01e4c835afb3 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql @@ -3,10 +3,8 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; DROP TABLE IF EXISTS tbl; -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP); +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP); INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00'); -SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; - -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')); \ No newline at end of file +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql new file mode 100644 index 0000000000000..df221a2c41f61 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql @@ -0,0 +1,94 @@ +DROP TABLE IF EXISTS table_integers; +CREATE TABLE table_integers(i INTEGER); +INSERT INTO table_integers VALUES (1), (2), (3), (NULL); + +SELECT + i, + ( + SELECT SUM(ss1.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + ) AS j +FROM table_integers i1 +ORDER BY i; + +SELECT + i, + ( + SELECT SUM(ss2.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE s1.i = i1.i + AND EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE t2.i = s1.i + ) + ) ss2 + ) AS j +FROM table_integers i1 +ORDER BY i; + +SELECT + i, + ( + SELECT SUM(ss1.i) + SUM(ss2.i) + FROM ( + -- First derived table: values greater than at least one other + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + LEFT OUTER JOIN ( + -- Second derived table: values equal to at least one other + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i = t2.i + ) + ) ss2 + ON ss1.i = ss2.i + ) AS j +FROM table_integers i1 +ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE (SELECT i FROM table_integers WHERE i=s1.i) = 1) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; + +DROP TABLE IF EXISTS tbl_ProductSales; +DROP TABLE IF EXISTS another_T; +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int); +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT); +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100); +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888); + +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON EXISTS (SELECT MAX(t1.col1 + t3.col4) AS mymax FROM another_T t3 HAVING t1.col7 <> mymax)) FROM another_T t1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql deleted file mode 100644 index d7c4af77ebf38..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/in-subquery.sql +++ /dev/null @@ -1,22 +0,0 @@ ---ONLY_IF spark - -set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; - -DROP TABLE IF EXISTS tbl_ProductSales; -DROP TABLE IF EXISTS another_T; -CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int); -CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT); -INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100); -INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888); - -SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; - -SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1; - -SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; - -SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1; - -SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> ANY(SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1; - -SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql index b6b4d2f0a2668..a9f11539932b2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql @@ -5,6 +5,8 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; DROP TABLE IF EXISTS t; CREATE TABLE t(ps_supplycost INT, n_name INT); +-- These two queries will fail analysis because +-- currently we don't support nested correlations in lateral subqueries. SELECT NULL FROM t AS ref_2, @@ -12,7 +14,6 @@ FROM FROM (SELECT * FROM t AS ref_5, LATERAL (SELECT ref_5.ps_supplycost AS c0, ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4; - SELECT * FROM t AS ref_2, diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql index 1c6dd5bddb1c3..6dd485ddbd280 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -1,29 +1,7 @@ --ONLY_IF spark set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)); - -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)); - -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)); - -DROP TABLE IF EXISTS t; -CREATE TABLE t(ps_supplycost INT, n_name INT); - -SELECT NULL -FROM - t AS ref_2, - (SELECT (SELECT NULL - FROM (FROM t AS ref_5, - (SELECT ref_2.n_name AS c1)))); - -SELECT NULL -FROM - t AS ref_2, - (SELECT (SELECT NULL - FROM (FROM t AS ref_5, - (SELECT ref_5.ps_supplycost AS c0, - ref_2.n_name AS c1)))); +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1; DROP TABLE IF EXISTS table_integers; CREATE TABLE table_integers(i INTEGER); @@ -33,74 +11,20 @@ SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM table_integers i2) AS j FROM table_integers i1 ORDER BY i; - SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM table_integers) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5)) FROM table_integers i4)) FROM table_integers i3)) FROM table_integers i2) AS j FROM table_integers i1 GROUP BY i ORDER BY i; - SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i; -SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3) FROM table_integers i2) FROM table_integers i1 ORDER BY 1; - -SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers) FROM table_integers i2) FROM table_integers i1 GROUP BY i ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 LEFT OUTER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; - SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT i=ANY(SELECT i FROM table_integers WHERE i=s1.i) FROM table_integers s1 WHERE i=i1.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM table_integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; - SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i; SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i; SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i; -SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i; - -SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM table_integers i3) FROM table_integers i2 LIMIT 1) FROM table_integers i1 ORDER BY 1; - -SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM table_integers i3) FROM table_integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM table_integers i1 ORDER BY 1; \ No newline at end of file +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql new file mode 100644 index 0000000000000..2523b3e307301 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql @@ -0,0 +1,85 @@ +--ONLY_IF spark +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; + +-- Spark SQL does not support correlations in ORDER BY clause. +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) = 1; + +-- Spark SQL does not support correlations in LIMIT/OFFSET clause. +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)) = 1; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(ps_supplycost INT, n_name INT); + +-- Spark SQL does not support correlated subqueries in FROM clause without +-- explicit Lateral keyword. +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_2.n_name AS c1)))); + +-- Spark SQL does not support correlated subqueries in FROM clause without +-- explicit Lateral keyword. +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1)))); + +DROP TABLE IF EXISTS table_integers; +CREATE TABLE table_integers(i INTEGER); +INSERT INTO table_integers VALUES (1), (2), (3), (NULL); + +-- Spark SQL only allow Project/Join/Filter to contain outer references. +-- Any subqueries containing outer references with aggregate expressions must +-- be on the having clause. +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM table_integers i2) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5)) FROM table_integers i4)) FROM table_integers i3)) FROM table_integers i2) AS j FROM table_integers i1 GROUP BY i ORDER BY i; +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3) FROM table_integers i2) FROM table_integers i1 ORDER BY 1; +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers) FROM table_integers i2) FROM table_integers i1 GROUP BY i ORDER BY i; + +-- ScalarSubquery cannot be in the groupBy/aggregate expressions. +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM table_integers) AS j FROM table_integers i1 ORDER BY i; + +-- No correlated subqueries in the join condition. +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 LEFT OUTER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i + +-- Spark sql does not allow mixing outer references and local references in one aggregates. +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM table_integers i3) FROM table_integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM table_integers i1 ORDER BY 1; +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM table_integers i3) FROM table_integers i2 LIMIT 1) FROM table_integers i1 ORDER BY 1; + +-- Spark sql does not allow correlations in the right child of left outer join. +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i<>i1.i OR EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; + +DROP TABLE IF EXISTS tbl_ProductSales; +DROP TABLE IF EXISTS another_T; +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int); +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT); +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100); +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888); + +-- Spark sql does not allow mixing outer references and local references in one aggregates. +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1; + +-- Spark SQL only allow Project/Join/Filter to contain outer references. +-- Any subqueries containing outer references with aggregate expressions must +-- be on the having clause. +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1; +SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; + +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +-- non deterministic +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')); + + diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 93c60fd49c582..77bcf7dd13d8e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -76,7 +76,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "messageParameters" : { "input" : "\"min(t2a)\", \"t2c\"", "missingAttributes" : "\"t2b\"", - "operator" : "!Filter t2c#x IN (list#x [t2b#x])" + "operator" : "Filter t2c#x IN (list#x [t2b#x])" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out index ffc61aba4775b..3fd05c63bae94 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -13,38 +13,30 @@ DropTable true, false -- !query -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR, f DATE, g TIMESTAMP) +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false -INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') -SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1 +-- !query +INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'INSERT'", - "hint" : "" - } -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl], Append, `spark_catalog`.`default`.`tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl), [a, b, c, d, e, f, g] ++- Project [cast(col1#x as tinyint) AS a#x, cast(col2#x as smallint) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as bigint) AS d#xL, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col5#x as string), 1)) AS e#x, cast(col6#x as date) AS f#x, cast(col7#x as timestamp) AS g#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x] -- !query -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')) +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`tbl`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 17, - "fragment" : "tbl" - } ] -} +Project [(c#x + cast(scalar-subquery#x [b#x] as int)) AS (c + scalarsubquery(b))#x] +: +- Project [outer(b#x)] +: +- Filter exists#x [b#x && a#x] +: : +- Project [(outer(b#x) + cast(outer(a#x) as smallint)) AS (outer(t1.b) + outer(t2.a))#x] +: : +- OneRowRelation +: +- SubqueryAlias t2 +: +- SubqueryAlias spark_catalog.default.tbl +: +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet ++- SubqueryAlias t1 + +- SubqueryAlias spark_catalog.default.tbl + +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out new file mode 100644 index 0000000000000..9cc385dc6c9a2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out @@ -0,0 +1,406 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT + i, + ( + SELECT SUM(ss1.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss1 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [1 AS 1#x] + : : +- Filter (outer(i#x) > i#x) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT + i, + ( + SELECT SUM(ss2.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE s1.i = i1.i + AND EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE t2.i = s1.i + ) + ) ss2 + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [1 AS 1#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT + i, + ( + SELECT SUM(ss1.i) + SUM(ss2.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + LEFT OUTER JOIN ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i = t2.i + ) + ) ss2 + ON ss1.i = ss2.i + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [(sum(i#x) + sum(i#x)) AS (sum(i) + sum(i))#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias ss1 + : : +- Project [i#x] + : : +- Filter exists#x [i#x] + : : : +- Project [1 AS 1#x] + : : : +- Filter (outer(i#x) > i#x) + : : : +- SubqueryAlias t2 + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- SubqueryAlias s1 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [1 AS 1#x] + : : +- Filter (outer(i#x) = i#x) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter CASE WHEN ((i#x = outer(i#x)) AND exists#x [i#x]) THEN true ELSE false END + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE (SELECT i FROM table_integers WHERE i=s1.i) = 1) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter (scalar-subquery#x [i#x] = 1) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter CASE WHEN ((i#x = outer(i#x)) AND exists#x [i#x]) THEN true ELSE false END + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss1 + : +- Project [i#x] + : +- Filter exists#x [i#x && i#x] + : : +- Project [i#x] + : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [(sum(i#x) + sum(i#x)) AS (sum(i) + sum(i))#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias ss1 + : : +- Project [i#x] + : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x && i#x]) + : : : +- Project [i#x] + : : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- SubqueryAlias s1 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +DROP TABLE IF EXISTS tbl_ProductSales +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales + + +-- !query +DROP TABLE IF EXISTS another_T +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T + + +-- !query +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false + + +-- !query +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false + + +-- !query +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] ++- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] ++- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON EXISTS (SELECT MAX(t1.col1 + t3.col4) AS mymax FROM another_T t3 HAVING t1.col7 <> mymax)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "max((outer(t1.col1) + t3.col4))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 91, + "stopIndex" : 112, + "fragment" : "MAX(t1.col1 + t3.col4)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out deleted file mode 100644 index 2d54d297e9ac3..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/in-subquery.sql.out +++ /dev/null @@ -1,160 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true --- !query analysis -SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) - - --- !query -DROP TABLE IF EXISTS tbl_ProductSales --- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales - - --- !query -DROP TABLE IF EXISTS another_T --- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T - - --- !query -CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false - - --- !query -CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false - - --- !query -INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) --- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] -+- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] - +- LocalRelation [col1#x, col2#x, col3#x, col4#x] - - --- !query -INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) --- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] -+- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] - - --- !query -SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 32, - "stopIndex" : 47, - "fragment" : "SELECT MAX(col7)" - } ] -} - - --- !query -SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1 --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 32, - "stopIndex" : 47, - "fragment" : "SELECT MAX(col7)" - } ] -} - - --- !query -SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 129, - "stopIndex" : 144, - "fragment" : "SELECT MAX(col7)" - } ] -} - - --- !query -SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", - "sqlState" : "0A000", - "messageParameters" : { - "function" : "max((outer(t1.col1) + t3.col4))" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 95, - "stopIndex" : 116, - "fragment" : "MAX(t1.col1 + t3.col4)" - } ] -} - - --- !query -SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> ANY(SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : ": missing ')'" - } -} - - --- !query -SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1 --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 43, - "stopIndex" : 58, - "fragment" : "SELECT MAX(col7)" - } ] -} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index 5cfd1b73ab9f0..ec597394136c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -6,438 +6,114 @@ SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(t -- !query -DROP TABLE IF EXISTS t0 +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1 -- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t0 - - --- !query -CREATE TABLE t0(c0 INT) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t0`, false - - --- !query -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"c0\"", - "operator" : "!Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 40, - "stopIndex" : 57, - "fragment" : "SELECT (SELECT c0)" - } ] -} - - --- !query -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"c0 ASC NULLS FIRST\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 57, - "stopIndex" : 67, - "fragment" : "ORDER BY c0" - } ] -} - - --- !query -SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)) --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", - "sqlState" : "42K0E", - "messageParameters" : { - "expr" : "\"outer(t0.c0)\"", - "name" : "limit" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 63, - "stopIndex" : 64, - "fragment" : "c0" - } ] -} - - --- !query -DROP TABLE IF EXISTS t --- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t - - --- !query -CREATE TABLE t(ps_supplycost INT, n_name INT) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false - - --- !query -SELECT NULL -FROM - t AS ref_2, - (SELECT (SELECT NULL - FROM (FROM t AS ref_5, - (SELECT ref_2.n_name AS c1)))) --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`ref_2`.`n_name`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 113, - "stopIndex" : 124, - "fragment" : "ref_2.n_name" - } ] -} - - --- !query -SELECT NULL -FROM - t AS ref_2, - (SELECT (SELECT NULL - FROM (FROM t AS ref_5, - (SELECT ref_5.ps_supplycost AS c0, - ref_2.n_name AS c1)))) --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`ref_5`.`ps_supplycost`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 113, - "stopIndex" : 131, - "fragment" : "ref_5.ps_supplycost" - } ] -} +Project [1 AS 1#x] ++- Filter (scalar-subquery#x [c0#x] = 1) + : +- Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x] + : : +- Project [outer(c0#x)] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias t0 + +- Project [1#x AS c0#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query -DROP TABLE IF EXISTS integers +DROP TABLE IF EXISTS table_integers -- !query analysis DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.integers ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers -- !query -CREATE TABLE integers(i INTEGER) +CREATE TABLE table_integers(i INTEGER) -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`integers`, false +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false -- !query -INSERT INTO integers VALUES (1), (2), (3), (NULL) +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/integers], Append, `spark_catalog`.`default`.`integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/integers), [i] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] +- Project [cast(col1#x as int) AS i#x] +- LocalRelation [col1#x] -- !query -SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 42, - "fragment" : "SELECT (SELECT 42+i1.i)+42+i1.i" - } ] -} - - --- !query -SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 56, - "fragment" : "SELECT (SELECT 42+i1.i)++i1.i" - } ] -} - - --- !query -SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM integers i2) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"(i + sum(i)) AS `(outer(i1.i) + sum(outer(i2.i)))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 40, - "fragment" : "SELECT i1.i+SUM(i2.i)" - } ] -} - - --- !query -SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 67, - "fragment" : "SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)" - } ] -} - - --- !query -SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM integers) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(i)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 41, - "fragment" : "(SELECT 42+i1.i)" - } ] -} - - --- !query -SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5)) FROM integers i4)) FROM integers i3)) FROM integers i2) AS j FROM integers i1 GROUP BY i ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"((((sum(i) + sum(i)) + sum(i)) + sum(i)) + sum(i)) AS `((((sum(i) + sum(outer(i4.i))) + sum(outer(i3.i))) + sum(outer(i2.i))) + sum(outer(i1.i)))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 39, - "stopIndex" : 108, - "fragment" : "SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM integers i5" - } ] -} - - --- !query -SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i))) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 106, - "fragment" : "SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM integers i2 WHERE i2.i=i1.i)" - } ] -} - - --- !query -SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3) FROM integers i2) FROM integers i1 ORDER BY 1 --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(outer(i1.i)) + sum(outer(i2.i))) + sum(i))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 17, - "stopIndex" : 69, - "fragment" : "SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM integers i3" - } ] -} - - --- !query -SELECT i, (SELECT SUM(s1.i) FROM integers s1 INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", - "sqlState" : "0A000", - "messageParameters" : { - "treeNode" : "Join Inner, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.integers\n: +- Relation spark_catalog.default.integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 46, - "stopIndex" : 108, - "fragment" : "INNER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" - } ] -} - - --- !query -SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers) FROM integers i2) FROM integers i1 GROUP BY i ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(i) + sum(outer(i1.i))) + sum(outer(i2.i)))`\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 74, - "fragment" : "SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM integers" - } ] -} - - --- !query -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] + : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] + : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] + : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(s1.i) FROM integers s1 LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", - "sqlState" : "0A000", - "messageParameters" : { - "treeNode" : "Join LeftOuter, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.integers\n: +- Relation spark_catalog.default.integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 46, - "stopIndex" : 113, - "fragment" : "LEFT OUTER JOIN integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" - } ] -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] + : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] + : : : : +- OneRowRelation + : : : +- Filter (i#x = outer(i#x)) + : : : +- SubqueryAlias i2 + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i=i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#xL] @@ -446,18 +122,18 @@ Sort [i#x ASC NULLS FIRST], true : :- SubqueryAlias s1 : : +- Project [i#x] : : +- Filter (i#x = outer(i#x)) - : : +- SubqueryAlias spark_catalog.default.integers - : : +- Relation spark_catalog.default.integers[i#x] parquet + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet : +- SubqueryAlias s2 - : +- SubqueryAlias spark_catalog.default.integers - : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.integers - +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM integers WHERE i<>i1.i) s1 LEFT OUTER JOIN integers s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#xL] @@ -466,86 +142,18 @@ Sort [i#x ASC NULLS FIRST], true : :- SubqueryAlias s1 : : +- Project [i#x] : : +- Filter NOT (i#x = outer(i#x)) - : : +- SubqueryAlias spark_catalog.default.integers - : : +- Relation spark_catalog.default.integers[i#x] parquet + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet : +- SubqueryAlias s2 - : +- SubqueryAlias spark_catalog.default.integers - : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.integers - +- Relation spark_catalog.default.integers[i#x] parquet - - --- !query -SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", - "sqlState" : "0A000", - "messageParameters" : { - "treeNode" : "Filter (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 63, - "stopIndex" : 97, - "fragment" : "SELECT i FROM integers WHERE i=i1.i" - } ] -} - - --- !query -SELECT i, (SELECT SUM(s2.i) FROM integers s1 LEFT OUTER JOIN (SELECT i FROM integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", - "sqlState" : "0A000", - "messageParameters" : { - "treeNode" : "Filter NOT (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.integers\n +- Relation spark_catalog.default.integers[i#x] parquet\n" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 63, - "stopIndex" : 98, - "fragment" : "SELECT i FROM integers WHERE i<>i1.i" - } ] -} + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'WHEN'", - "hint" : ": missing ')'" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i) ss2) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#xL] @@ -554,287 +162,80 @@ Sort [i#x ASC NULLS FIRST], true : +- Project [i#x] : +- Filter (i#x = outer(i#x)) : +- SubqueryAlias s1 - : +- SubqueryAlias spark_catalog.default.integers - : +- Relation spark_catalog.default.integers[i#x] parquet + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.integers - +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT i=ANY(SELECT i FROM integers WHERE i=s1.i) FROM integers s1 WHERE i=i1.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'i'", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE CASE WHEN (i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'WHEN'", - "hint" : ": missing ')'" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM integers WHERE i=s1.i)) ss2) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#xL] - : +- Aggregate [sum(i#x) AS sum(i)#xL] - : +- SubqueryAlias ss2 - : +- Project [i#x] - : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) - : : +- Project [i#x] - : : +- Filter (i#x = outer(i#x)) - : : +- SubqueryAlias spark_catalog.default.integers - : : +- Relation spark_catalog.default.integers[i#x] parquet - : +- SubqueryAlias s1 - : +- SubqueryAlias spark_catalog.default.integers - : +- Relation spark_catalog.default.integers[i#x] parquet ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalarsubquery(i)#x] + : +- SubqueryAlias s1 + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.integers - +- Relation spark_catalog.default.integers[i#x] parquet + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=i1.i AND i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM integers s1 WHERE i=i1.i AND i>ANY(SELECT i FROM integers WHERE i<>s1.i)) ss1 LEFT OUTER JOIN (SELECT i FROM integers s1 WHERE i<>i1.i OR i=ANY(SELECT i FROM integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'('", - "hint" : "" - } -} - - --- !query -SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 27, - "stopIndex" : 49, - "fragment" : "SELECT (SELECT 42+i1.i)" - } ] -} - - --- !query -SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM integers i1 ORDER BY i --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [scalar-subquery#x [i#x] AS k#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 35, - "stopIndex" : 62, - "fragment" : "SELECT (SELECT 42+i1.i) AS k" - } ] -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join Inner + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "", - "missingAttributes" : "\"i\"", - "operator" : "!Project [scalar-subquery#x [i#x] AS k#x]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 35, - "stopIndex" : 62, - "fragment" : "SELECT (SELECT 42+i1.i) AS k" - } ] -} +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join LeftOuter, (k#x = k#x) + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet -- !query -SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM integers i1 ORDER BY i +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [outer(i#x) IN (1,2,3,4,5,6,7,8) AS (outer(i1.i) IN (1, 2, 3, 4, 5, 6, 7, 8))#x] : +- OneRowRelation +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.integers - +- Relation spark_catalog.default.integers[i#x] parquet - - --- !query -SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM integers i3) FROM integers i2 LIMIT 1) FROM integers i1 ORDER BY 1 --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", - "sqlState" : "0A000", - "messageParameters" : { - "function" : "covar_pop(CAST(outer(i1.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 44, - "fragment" : "COVAR_POP(i1.i, i3.i)" - } ] -} - - --- !query -SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM integers i3) FROM integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM integers i1 ORDER BY 1 --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", - "sqlState" : "0A000", - "messageParameters" : { - "function" : "covar_pop(CAST(outer(i2.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 44, - "fragment" : "COVAR_POP(i2.i, i3.i)" - } ] -} + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out new file mode 100644 index 0000000000000..2e7509a1feebc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out @@ -0,0 +1,568 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) = 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"c0 ASC NULLS FIRST\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 57, + "stopIndex" : 67, + "fragment" : "ORDER BY c0" + } ] +} + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)) = 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"outer(t0.c0)\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 64, + "fragment" : "c0" + } ] +} + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t(ps_supplycost INT, n_name INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 124, + "fragment" : "ref_2.n_name" + } ] +} + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_5`.`ps_supplycost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 131, + "fragment" : "ref_5.ps_supplycost" + } ] +} + + +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM table_integers i2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"(i + sum(i)) AS `(outer(i1.i) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 40, + "fragment" : "SELECT i1.i+SUM(i2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5)) FROM table_integers i4)) FROM table_integers i3)) FROM table_integers i2) AS j FROM table_integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((((sum(i) + sum(i)) + sum(i)) + sum(i)) + sum(i)) AS `((((sum(i) + sum(outer(i4.i))) + sum(outer(i3.i))) + sum(outer(i2.i))) + sum(outer(i1.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 114, + "fragment" : "SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5" + } ] +} + + +-- !query +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3) FROM table_integers i2) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(outer(i1.i)) + sum(outer(i2.i))) + sum(i))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 75, + "fragment" : "SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3" + } ] +} + + +-- !query +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers) FROM table_integers i2) FROM table_integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(i) + sum(outer(i1.i))) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 80, + "fragment" : "SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM table_integers) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(i)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 41, + "fragment" : "(SELECT 42+i1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Join Inner, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 120, + "fragment" : "INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 LEFT OUTER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i + +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM table_integers i3) FROM table_integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM table_integers i3) FROM table_integers i2 LIMIT 1) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "covar_pop(CAST(outer(i1.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 44, + "fragment" : "COVAR_POP(i1.i, i3.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 160, + "stopIndex" : 257, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 160, + "stopIndex" : 257, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 171, + "stopIndex" : 268, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i<>i1.i OR EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (NOT (i#x = outer(i#x)) OR exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 180, + "stopIndex" : 277, + "fragment" : "SELECT i FROM table_integers s1 WHERE i<>i1.i OR EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 109, + "fragment" : "SELECT i FROM table_integers WHERE i=i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter NOT (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 110, + "fragment" : "SELECT i FROM table_integers WHERE i<>i1.i" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tbl_ProductSales +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales + + +-- !query +DROP TABLE IF EXISTS another_T +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T + + +-- !query +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false + + +-- !query +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false + + +-- !query +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] ++- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] ++- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "max((outer(t1.col1) + t3.col4))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 116, + "fragment" : "MAX(t1.col1 + t3.col4)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 129, + "stopIndex" : 144, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 58, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "Schema may not be specified in a Create Table As Select (CTAS) statement" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 245, + "fragment" : "CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP)\nSELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01'))" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out b/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out index 6d2aaaa8e0725..0c23f17c2920c 100644 --- a/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/to_from_avro.sql.out @@ -1,4 +1,12 @@ -- Automatically generated by SQLQueryTestSuite +-- !query +drop table if exists t +-- !query schema +struct<> +-- !query output + + + -- !query create table t as select named_struct('u', named_struct('member0', member0, 'member1', member1)) as s @@ -6,14 +14,7 @@ create table t as -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -{ - "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", - "sqlState" : "42P07", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t`" - } -} + -- !query @@ -40,18 +41,19 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", "messageParameters" : { - "objectName" : "`s`", - "proposal" : "`n_name`, `ps_supplycost`" + "hint" : "", + "msg" : "The second argument of the FROM_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value from AVRO format", + "sqlExpr" : "\"from_avro(s, 42, map())\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 18, - "stopIndex" : 18, - "fragment" : "s" + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "from_avro(s, 42, map())" } ] } @@ -63,18 +65,19 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", "messageParameters" : { - "objectName" : "`s`", - "proposal" : "`n_name`, `ps_supplycost`" + "hint" : "", + "msg" : "The third argument of the FROM_AVRO SQL function must be a constant map of strings to strings containing the options to use for converting the value from AVRO format", + "sqlExpr" : "\"from_avro(s, variablereference(system.session.avro_schema='{ \"type\": \"record\", \"name\": \"struct\", \"fields\": [{ \"name\": \"u\", \"type\": [\"int\",\"string\"] }] }'), 42)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 18, - "stopIndex" : 18, - "fragment" : "s" + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "from_avro(s, avro_schema, 42)" } ] } @@ -86,18 +89,19 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", "messageParameters" : { - "objectName" : "`s`", - "proposal" : "`n_name`, `ps_supplycost`" + "hint" : "", + "msg" : "The second argument of the TO_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value to AVRO format", + "sqlExpr" : "\"to_avro(s, 42)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 16, - "stopIndex" : 16, - "fragment" : "s" + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "to_avro(s, 42)" } ] } @@ -107,21 +111,13 @@ select to_avro(s, avro_schema) as result from t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", - "sqlState" : "42703", + "errorClass" : "AVRO_NOT_LOADED_SQL_FUNCTIONS_UNUSABLE", + "sqlState" : "22KD3", "messageParameters" : { - "objectName" : "`s`", - "proposal" : "`n_name`, `ps_supplycost`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 16, - "fragment" : "s" - } ] + "functionName" : "TO_AVRO" + } } From bb97392c9549e21793f8e1b46663585d59f3728f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 11:09:08 -0700 Subject: [PATCH 34/48] deduplicate --- .../sql/catalyst/analysis/Analyzer.scala | 5 ++- .../analyzer-results/join-lateral.sql.out | 38 +++++++++--------- .../named-function-arguments.sql.out | 6 +-- .../exists-subquery/exists-basic.sql.out | 2 +- .../exists-joins-and-set-ops.sql.out | 16 ++++---- .../nested-scalar-subquery-count-bug.sql.out | 4 +- .../scalar-subquery-group-by.sql.out | 6 +-- .../scalar-subquery-predicate.sql.out | 6 +-- .../scalar-subquery-select.sql.out | 4 +- .../scalar-subquery-set-op.sql.out | 40 +++++++++---------- .../subquery/subquery-nested-data.sql.out | 10 ++--- .../subquery-not-supported.sql | 4 +- .../combined-subquery.sql.out | 6 +++ .../nestedcorrelation/exists-subquery.sql.out | 4 +- .../nestedcorrelation/scalar-subquery.sql.out | 22 +++++----- .../subquery-not-supported.sql.out | 20 ---------- .../scalar-subquery-group-by.sql.out | 6 +-- .../scalar-subquery-select.sql.out | 4 +- .../scalar-subquery-set-op.sql.out | 40 +++++++++---------- 19 files changed, 115 insertions(+), 128 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 27f949b5816cf..9bc7bd361ec5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2370,8 +2370,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // them as children of SubqueryExpression. if (newSubqueryPlan.resolved) { // Record the outer references as children of subquery expression. - f(newSubqueryPlan, SubExprUtils.getOuterReferences(newSubqueryPlan) ++ - getOuterAttrsNeedToBePropagated(newSubqueryPlan)) + val outer = SubExprUtils.getOuterReferences(newSubqueryPlan) ++ + getOuterAttrsNeedToBePropagated(newSubqueryPlan) + f(newSubqueryPlan, outer.distinct) } else { e.withNewPlan(newSubqueryPlan) } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index f8e88a57bf426..8246bb089fe66 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -635,7 +635,7 @@ Project [c1#x, c2#x, c1#x, m#x, m#x] +- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [c1#x, m#x, m#x] - : +- LateralJoin lateral-subquery#x [m#x && m#x && c1#x], Inner + : +- LateralJoin lateral-subquery#x [m#x && c1#x], Inner : : +- SubqueryAlias __auto_generated_subquery_name : : +- Project [outer(m#x) AS m#x] : : +- Filter (outer(m#x) > outer(c1#x)) @@ -1201,7 +1201,7 @@ Project [c1#x, c2#x, count(1)#xL] SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) -- !query analysis Project [c1#x, c2#x, cnt#xL] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [cnt#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1222,7 +1222,7 @@ Project [c1#x, c2#x, cnt#xL] SELECT * FROM t1, LATERAL (SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) -- !query analysis Project [c1#x, c2#x, count(1)#xL] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Aggregate [count(1) AS count(1)#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1246,7 +1246,7 @@ SELECT * FROM t1, LATERAL ( ) -- !query analysis Project [c1#x, c2#x, count(1)#xL] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Aggregate [cnt#xL], [count(1) AS count(1)#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1299,7 +1299,7 @@ SELECT * FROM t1, LATERAL ( ) -- !query analysis Project [c1#x, c2#x, (cnt + cnt)#xL] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(cnt#xL + cnt#xL) AS (cnt + cnt)#xL] : +- Join Inner @@ -1472,7 +1472,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c2#x] @@ -1537,7 +1537,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true : :- Project [c2#x] @@ -1601,7 +1601,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true : :- Project [c2#x] @@ -1698,7 +1698,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c1#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c1#x, c2#x] @@ -2078,7 +2078,7 @@ SELECT * FROM t2 JOIN LATERAL -- !query analysis Union false, false :- Project [c1#x, c2#x, c2#x] -: +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner +: +- LateralJoin lateral-subquery#x [c1#x], Inner : : +- SubqueryAlias __auto_generated_subquery_name : : +- Union false, false : : :- Project [c2#x] @@ -2098,7 +2098,7 @@ Union false, false : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Project [c1#x, c2#x, c2#x] - +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + +- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c2#x] @@ -2139,7 +2139,7 @@ SELECT * FROM t1 JOIN LATERAL ) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct : +- Union false, false @@ -2195,7 +2195,7 @@ SELECT * FROM t1 JOIN LATERAL ) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false : :- Union false, false @@ -2437,7 +2437,7 @@ Project [c1#x, c2#x, col#x] SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3)) -- !query analysis Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] : +- SubqueryAlias t @@ -2454,7 +2454,7 @@ Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1) -- !query analysis Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x && c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] : +- Filter (outer(c2#x) > 1) @@ -2472,7 +2472,7 @@ Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y) -- !query analysis Project [c1#x, c2#x, x#x, y#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [x#x, y#x] : +- Join Inner, (x#x = y#x) @@ -2947,7 +2947,7 @@ select * from t1 join lateral (select t4.c1 from t4 where t1.c1 = t4.c1 order by t4.c1 limit 3)) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- GlobalLimit 1 @@ -2983,7 +2983,7 @@ select * from t1 join lateral order by foo.t limit 5) -- !query analysis Project [c1#x, c2#x, t#x] -+- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- GlobalLimit 5 : +- LocalLimit 5 @@ -3031,7 +3031,7 @@ left join ) as lateral_table -- !query analysis Project [1 AS 1#x] -+- LateralJoin lateral-subquery#x [c2#x && c1#x && c1#x], LeftOuter ++- LateralJoin lateral-subquery#x [c2#x && c1#x], LeftOuter : +- SubqueryAlias lateral_table : +- GlobalLimit 1 : +- LocalLimit 1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out index 2315a5f0678a0..b656e40e3fc20 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out @@ -131,7 +131,7 @@ Project [col#x, col#x] SELECT * FROM explode(array(1, 2)) AS t, LATERAL explode(array(3 * t.col, 4 * t.col)) -- !query analysis Project [col#x, col#x] -+- LateralJoin lateral-subquery#x [col#x && col#x], Inner ++- LateralJoin lateral-subquery#x [col#x], Inner : +- Generate explode(array((3 * outer(col#x)), (4 * outer(col#x)))), false, [col#x] : +- OneRowRelation +- SubqueryAlias t @@ -180,7 +180,7 @@ Project [pos#x, col#x, pos#x, col#x] SELECT * FROM posexplode(array(1, 2)) AS t, LATERAL posexplode(array(3 * t.col, 4 * t.col)) -- !query analysis Project [pos#x, col#x, pos#x, col#x] -+- LateralJoin lateral-subquery#x [col#x && col#x], Inner ++- LateralJoin lateral-subquery#x [col#x], Inner : +- Generate posexplode(array((3 * outer(col#x)), (4 * outer(col#x)))), false, [pos#x, col#x] : +- OneRowRelation +- SubqueryAlias t @@ -229,7 +229,7 @@ Project [col1#x, col2#x, col1#x, col2#x] SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t, LATERAL inline(array(struct(3 * t.col1, 4 * t.col1))) -- !query analysis Project [col1#x, col2#x, col1#x, col2#x] -+- LateralJoin lateral-subquery#x [col1#x && col1#x], Inner ++- LateralJoin lateral-subquery#x [col1#x], Inner : +- Generate inline(array(struct(col1, (3 * outer(col1#x)), col2, (4 * outer(col1#x))))), false, [col1#x, col2#x] : +- OneRowRelation +- SubqueryAlias t diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out index 81c4a15dc9f4a..02de5f287d48b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out @@ -138,7 +138,7 @@ WHERE EXISTS (SELECT dept.dept_name OR emp.dept_id IS NULL) -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Project [dept_name#x] : +- Filter ((outer(dept_id#x) = dept_id#x) OR isnull(outer(dept_id#x))) : +- SubqueryAlias dept diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out index 55b10125f7680..2fbe9f3cb8a8b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -686,7 +686,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Distinct : +- Union false, false : :- Project [dept_id#x, dept_name#x, state#x] @@ -725,7 +725,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x && dept_id#x] ++- Filter NOT exists#x [dept_id#x] : +- Distinct : +- Union false, false : :- Project [dept_id#x, dept_name#x, state#x] @@ -764,7 +764,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Intersect All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -802,7 +802,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Intersect false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -840,7 +840,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Except All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -878,7 +878,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x && dept_id#x] ++- Filter exists#x [dept_id#x] : +- Except false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -916,7 +916,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x && dept_id#x] ++- Filter NOT exists#x [dept_id#x] : +- Intersect All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -954,7 +954,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x && dept_id#x] ++- Filter NOT exists#x [dept_id#x] : +- Except false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out index 2a1abc0d48871..a689a6cd690fa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out @@ -79,7 +79,7 @@ select ( ) a from t1 order by a desc -- !query analysis Sort [a#xL DESC NULLS LAST], true -+- Project [scalar-subquery#x [a1#x && a1#x] AS a#xL] ++- Project [scalar-subquery#x [a1#x] AS a#xL] : +- Aggregate [sum((cnt#xL + cnt#xL)) AS sum((cnt + cnt))#xL] : +- Join Inner, (cnt#xL = cnt#xL) : :- SubqueryAlias l @@ -113,7 +113,7 @@ select ( ) a from t1 order by a desc -- !query analysis Sort [a#xL DESC NULLS LAST], true -+- Project [scalar-subquery#x [a1#x && a1#x] AS a#xL] ++- Project [scalar-subquery#x [a1#x] AS a#xL] : +- Aggregate [sum((cnt#xL + cnt#xL)) AS sum((cnt + cnt))#xL] : +- Join Inner, (cnt#xL = cnt#xL) : :- SubqueryAlias l diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out index 01de7beda551d..a2b9843c1bc7e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out @@ -96,7 +96,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] -- !query select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] : +- Aggregate [y2#x], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND (y2#x = (outer(x1#x) + 1))) : +- SubqueryAlias y @@ -113,7 +113,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)# select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 group by cast(y2 as double)) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] : +- Aggregate [cast(y2#x as double)], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND (cast(y2#x as double) = cast((outer(x1#x) + 1) as double))) : +- SubqueryAlias y @@ -250,7 +250,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] -- !query select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] : +- Aggregate [y2#x], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND ((y2#x + 10) = (outer(x1#x) + 1))) : +- SubqueryAlias y diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 0414782fb7e67..245bfbfe8b11f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -1320,7 +1320,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x]) : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1461,7 +1461,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1530,7 +1530,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x]) : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index f64b3736b5521..b3bbfbcb99147 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -529,7 +529,7 @@ Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#x] -- !query SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 -- !query analysis -Project [t1c#x, scalar-subquery#x [t1c#x && t1c#x] AS scalarsubquery(t1c, t1c)#x] +Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#x] : +- Project [outer(t1c#x)] : +- Filter (outer(t1c#x) = 8) : +- OneRowRelation @@ -1023,7 +1023,7 @@ WithCTE : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation -+- Project [scalar-subquery#x [col#x && col#x] AS scalarsubquery(col, col)#xL] ++- Project [scalar-subquery#x [col#x] AS scalarsubquery(col)#xL] : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index eaeff0ba9dedb..4df145e92d38f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -101,7 +101,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -203,7 +203,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -269,7 +269,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -399,7 +399,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -504,7 +504,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -572,7 +572,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -701,7 +701,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -803,7 +803,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -869,7 +869,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -997,7 +997,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1099,7 +1099,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1165,7 +1165,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1293,7 +1293,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1395,7 +1395,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1461,7 +1461,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1589,7 +1589,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1691,7 +1691,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1757,7 +1757,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1814,7 +1814,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(t1b#x) AS sum(t1b)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1854,7 +1854,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] : +- Aggregate [sum(t1b#x) AS sum(t1b)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out index a218238073dff..eef900f1a1b5f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out @@ -100,7 +100,7 @@ Project [xm#x, x2#x, ym#x, y2#x] select * from x join lateral (select * from y where xm[1] = ym[1] union all select * from y where xm[1] = ym[1] + 1) -- !query analysis Project [xm#x, x2#x, ym#x, y2#x] -+- LateralJoin lateral-subquery#x [xm#x && xm#x], Inner ++- LateralJoin lateral-subquery#x [xm#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [ym#x, y2#x] @@ -174,7 +174,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[1] >= 1) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x && xm#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[1] = ym#x[1]) AND (outer(xm#x)[1] >= 1)) : +- SubqueryAlias spark_catalog.default.y @@ -187,7 +187,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[2] >= ym[2]) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x && xm#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[1] = ym#x[1]) AND (outer(xm#x)[2] >= ym#x[2])) : +- SubqueryAlias spark_catalog.default.y @@ -230,7 +230,7 @@ Project [xm#x, x2#x, (outer(spark_catalog.default.x.xm)[1] - ym[1])#x] select * from x join lateral (select xm[1], xm[1] as s1, xm[1] - ym[1] as s2 from y) -- !query analysis Project [xm#x, x2#x, outer(spark_catalog.default.x.xm)[1]#x, s1#x, s2#x] -+- LateralJoin lateral-subquery#x [xm#x && xm#x && xm#x], Inner ++- LateralJoin lateral-subquery#x [xm#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [outer(xm#x)[1] AS outer(spark_catalog.default.x.xm)[1]#x, outer(xm#x)[1] AS s1#x, (outer(xm#x)[1] - ym#x[1]) AS s2#x] : +- SubqueryAlias spark_catalog.default.y @@ -289,7 +289,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[x2+1] = ym[1] and xm[1+x2] = ym[2]) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x && x2#x && xm#x && x2#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x && x2#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[(outer(x2#x) + 1)] = ym#x[1]) AND (outer(xm#x)[(1 + outer(x2#x))] = ym#x[2])) : +- SubqueryAlias spark_catalog.default.y diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql index 2523b3e307301..c7e1c7639106e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql @@ -78,8 +78,8 @@ SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FR SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP); -- non deterministic -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')); +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out index 3fd05c63bae94..e25e84d776941 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -40,3 +40,9 @@ Project [(c#x + cast(scalar-subquery#x [b#x] as int)) AS (c + scalarsubquery(b)) +- SubqueryAlias t1 +- SubqueryAlias spark_catalog.default.tbl +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet + + +-- !query +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out index 9cc385dc6c9a2..ccf7386de5337 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out @@ -300,7 +300,7 @@ Sort [i#x ASC NULLS FIRST], true : +- Aggregate [sum(i#x) AS sum(i)#xL] : +- SubqueryAlias ss1 : +- Project [i#x] - : +- Filter exists#x [i#x && i#x] + : +- Filter exists#x [i#x] : : +- Project [i#x] : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) : : +- SubqueryAlias spark_catalog.default.table_integers @@ -322,7 +322,7 @@ Sort [i#x ASC NULLS FIRST], true : +- Join LeftOuter, (i#x = i#x) : :- SubqueryAlias ss1 : : +- Project [i#x] - : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x && i#x]) + : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) : : : +- Project [i#x] : : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) : : : +- SubqueryAlias spark_catalog.default.table_integers diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index ec597394136c2..5f4501783a7fd 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -45,7 +45,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] : : +- OneRowRelation @@ -59,9 +59,9 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] - : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] - : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] + : : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] : : : : +- OneRowRelation @@ -77,10 +77,10 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] - : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] - : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] - : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] : : : : +- OneRowRelation : : : +- OneRowRelation @@ -98,7 +98,7 @@ Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x] AS scalarsubquery(i, i)#x] : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] : : : : +- OneRowRelation : : : +- Filter (i#x = outer(i#x)) @@ -189,7 +189,7 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [(k#x + k#x) AS (k + k)#x] : +- Join Inner : :- SubqueryAlias s1 @@ -211,7 +211,7 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [(k#x + k#x) AS (k + k)#x] : +- Join LeftOuter, (k#x = k#x) : :- SubqueryAlias s1 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out index 2e7509a1feebc..7006566746d3a 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out @@ -546,23 +546,3 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "fragment" : "SELECT MAX(col7)" } ] } - - --- !query -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01')) --- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "_LEGACY_ERROR_TEMP_0035", - "messageParameters" : { - "message" : "Schema may not be specified in a Create Table As Select (CTAS) statement" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 245, - "fragment" : "CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP)\nSELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT t2.f < DATE '2000-01-01'))" - } ] -} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out index 56932edd4e545..25dcab8798902 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out @@ -87,7 +87,7 @@ struct -- !query select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL @@ -97,7 +97,7 @@ struct select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 group by cast(y2 as double)) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL @@ -221,7 +221,7 @@ org.apache.spark.SparkRuntimeException -- !query select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 85bd9137602a3..61186f8c7f009 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -243,7 +243,7 @@ NULL NULL -- !query SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 -- !query schema -struct +struct -- !query output 12 NULL 12 NULL @@ -574,7 +574,7 @@ WITH T AS (SELECT 1 AS a) SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query schema -struct +struct -- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index 33a57a73be08e..7b19017ff0407 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -70,7 +70,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 8 2 7 @@ -124,7 +124,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 32 2 NULL @@ -158,7 +158,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 -2 @@ -240,7 +240,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 8 2 7 @@ -294,7 +294,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 32 2 NULL @@ -328,7 +328,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 -1 @@ -410,7 +410,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -464,7 +464,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -498,7 +498,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 0 2 -1 @@ -580,7 +580,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -634,7 +634,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -668,7 +668,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 0 2 -1 @@ -750,7 +750,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 3 2 NULL @@ -804,7 +804,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 11 2 NULL @@ -838,7 +838,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -920,7 +920,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 3 2 NULL @@ -974,7 +974,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 11 2 NULL @@ -1008,7 +1008,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -1053,7 +1053,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 2 2 NULL @@ -1069,7 +1069,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 1 From 26fb9fb9c37895193cf8557390d2a3f301cb9d0a Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 12:06:42 -0700 Subject: [PATCH 35/48] summarize not supported --- .../subquery-not-supported.sql | 1 + .../combined-subquery.sql.out | 6 ------ .../subquery-not-supported.sql.out | 19 +++++++++++++++++++ 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql index c7e1c7639106e..494529ae479eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql @@ -78,6 +78,7 @@ SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FR SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; +DROP TABLE IF EXISTS tbl; CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP); -- non deterministic SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out index e25e84d776941..3fd05c63bae94 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -40,9 +40,3 @@ Project [(c#x + cast(scalar-subquery#x [b#x] as int)) AS (c + scalarsubquery(b)) +- SubqueryAlias t1 +- SubqueryAlias spark_catalog.default.tbl +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet - - --- !query -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) --- !query analysis -[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out index 7006566746d3a..0529e422e4464 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out @@ -546,3 +546,22 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "fragment" : "SELECT MAX(col7)" } ] } + + +-- !query +DROP TABLE IF EXISTS tbl +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl + + +-- !query +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] From 7f30dfa30d43b932ea764aa3ec0cbc457f5d18bf Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 12:57:13 -0700 Subject: [PATCH 36/48] add new configs to control subquery type level feature --- .../resources/error/error-conditions.json | 2 +- .../analysis/ValidateSubqueryExpression.scala | 42 +++++++++++++++---- .../apache/spark/sql/internal/SQLConf.scala | 39 ++++++++++++++++- .../nestedcorrelation/combined-subquery.sql | 4 +- .../nestedcorrelation/exists-subquery.sql | 6 +++ .../nestedcorrelation/lateral-subquery.sql | 4 +- .../nestedcorrelation/scalar-subquery.sql | 3 ++ .../subquery-not-supported.sql | 3 ++ .../combined-subquery.sql.out | 18 ++++++++ .../nestedcorrelation/exists-subquery.sql.out | 24 +++++++++++ .../lateral-subquery.sql.out | 18 ++++++++ .../nestedcorrelation/scalar-subquery.sql.out | 18 ++++++++ .../subquery-not-supported.sql.out | 18 ++++++++ 13 files changed, 188 insertions(+), 11 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 9678bcbc153d4..a8123d2163bde 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4066,7 +4066,7 @@ }, "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED" : { "message" : [ - "Detected nested outer references in the subquery. Please set 'spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled' to true to enable this feature." + "Detected nested outer references in the subquery.This is not supported in the current version." ], "sqlState" : "0A000" }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 71aeb7d177488..57e632fce8ac2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -131,13 +131,41 @@ object ValidateSubqueryExpression } def checkNestedOuterReferences(expr: SubqueryExpression): Unit = { - if ((!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) && - expr.getNestedOuterAttrs.nonEmpty) { - throw new AnalysisException( - errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", - messageParameters = Map( - "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) - ) + if (expr.getNestedOuterAttrs.nonEmpty) { + if (!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) { + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + messageParameters = Map( + "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + ) + } + expr match { + case _: ScalarSubquery if + !SQLConf.get.getConf( + SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_SCALARSUBQUERIES) => + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + messageParameters = Map( + "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + ) + case _: ListQuery if + !SQLConf.get.getConf( + SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_INSUBQUERIES) => + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + messageParameters = Map( + "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + ) + case _: Exists if + !SQLConf.get.getConf( + SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_EXISTSSUBQUERIES) => + throw new AnalysisException( + errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + messageParameters = Map( + "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + ) + case _ => // Do nothing + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5ef3243e51666..fd601f76d3ffc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4237,10 +4237,47 @@ object SQLConf { .booleanConf .createWithDefault(true) + // TODO: remove this flag when the nested correlations handling is stable. val SUPPORT_NESTED_CORRELATED_SUBQUERIES = buildConf("spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled") .internal() - .doc("If enabled, support nested correlated subqueries") + .doc("If enabled, support nested correlated subqueries. This is a temporary flag " + + "to enable the new implementation of nested correlated subqueries." + + "This flag should be set to true when the new implementation is stable " + + "in both analyzer and optimizer. Otherwise, it should be set to false by default.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + + val SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_SCALARSUBQUERIES = + buildConf("spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled") + .internal() + .doc("If enabled, support nested correlated scalar subqueries. This is a temporary flag " + + "to enable the new implementation of nested correlated subqueries." + + "This flag should be set to true when the new implementation is stable " + + "in both analyzer and optimizer. Otherwise, it should be set to false by default.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + + val SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_INSUBQUERIES = + buildConf("spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled") + .internal() + .doc("If enabled, support nested correlated IN subqueries. This is a temporary flag " + + "to enable the new implementation of nested correlated subqueries." + + "This flag should be set to true when the new implementation is stable " + + "in both analyzer and optimizer. Otherwise, it should be set to false by default.") + .version("4.1.0") + .booleanConf + .createWithDefault(false) + + val SUPPORT_NESTED_CORRELATED_SUBQUERIES_FOR_EXISTSSUBQUERIES = + buildConf("spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled") + .internal() + .doc("If enabled, support nested correlated EXISTS subqueries. This is a temporary flag " + + "to enable the new implementation of nested correlated subqueries." + + "This flag should be set to true when the new implementation is stable " + + "in both analyzer and optimizer. Otherwise, it should be set to false by default.") .version("4.1.0") .booleanConf .createWithDefault(false) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql index a01e4c835afb3..f1325a5e92a56 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql @@ -1,6 +1,8 @@ --ONLY_IF spark - set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; DROP TABLE IF EXISTS tbl; CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql index df221a2c41f61..163f3823bf831 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql @@ -1,3 +1,9 @@ +--ONLY_IF spark +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; + DROP TABLE IF EXISTS table_integers; CREATE TABLE table_integers(i INTEGER); INSERT INTO table_integers VALUES (1), (2), (3), (NULL); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql index a9f11539932b2..53a0b13e23af7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql @@ -1,6 +1,8 @@ --ONLY_IF spark - set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; DROP TABLE IF EXISTS t; CREATE TABLE t(ps_supplycost INT, n_name INT); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql index 6dd485ddbd280..0000a5e35d2cd 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -1,5 +1,8 @@ --ONLY_IF spark set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql index 494529ae479eb..b1ef43d1397ad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql @@ -1,5 +1,8 @@ --ONLY_IF spark set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; -- Spark SQL does not support correlations in ORDER BY clause. SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) = 1; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out index 3fd05c63bae94..424747f0cb5d9 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -5,6 +5,24 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + -- !query DROP TABLE IF EXISTS tbl -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out index ccf7386de5337..601a3589e8f9a 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out @@ -1,4 +1,28 @@ -- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + -- !query DROP TABLE IF EXISTS table_integers -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out index af38633f19c9a..9cedbe2e28847 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/lateral-subquery.sql.out @@ -5,6 +5,24 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + -- !query DROP TABLE IF EXISTS t -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index 5f4501783a7fd..c8512a1a56e33 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -5,6 +5,24 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + -- !query SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out index 0529e422e4464..bcfd69b2e3906 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out @@ -5,6 +5,24 @@ set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + -- !query SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) = 1 -- !query analysis From 1465741b31907762646cf8a8c990d13ef5d4ddc0 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 14:52:37 -0700 Subject: [PATCH 37/48] queries returning nonderterministic results are also supported. --- .../nestedcorrelation/combined-subquery.sql | 5 +++- .../subquery-not-supported.sql | 9 +----- .../combined-subquery.sql.out | 6 ++++ .../subquery-not-supported.sql.out | 19 ------------ .../org/apache/spark/sql/SubquerySuite.scala | 29 +++++++++++++++++++ 5 files changed, 40 insertions(+), 28 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql index f1325a5e92a56..450b3421eb037 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql @@ -9,4 +9,7 @@ CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DAT INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00'); -SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; \ No newline at end of file +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1; + +-- non deterministic due to the timestamp type, the query itself is supported. +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql index b1ef43d1397ad..790405a088427 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql @@ -79,11 +79,4 @@ SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.co SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1; SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1; -SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; - -DROP TABLE IF EXISTS tbl; -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP); --- non deterministic -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); - - +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out index 424747f0cb5d9..eb712556160d6 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/combined-subquery.sql.out @@ -58,3 +58,9 @@ Project [(c#x + cast(scalar-subquery#x [b#x] as int)) AS (c + scalarsubquery(b)) +- SubqueryAlias t1 +- SubqueryAlias spark_catalog.default.tbl +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet + + +-- !query +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out index bcfd69b2e3906..c56057defa31c 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/subquery-not-supported.sql.out @@ -564,22 +564,3 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "fragment" : "SELECT MAX(col7)" } ] } - - --- !query -DROP TABLE IF EXISTS tbl --- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl - - --- !query -CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false - - --- !query -SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) --- !query analysis -[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 576f93e94ec1e..14b8154259ccb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2846,4 +2846,33 @@ class SubquerySuite extends QueryTest :: Row(true) :: Row(true) :: Row(true) :: Nil ) } + + test("test non deterministic query") { + sql("CREATE TABLE tbl(a TINYINT, b SMALLINT," + + " c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP);") + +// set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; + val query = + """ + |SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE + | EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND + | EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); + |""".stripMargin + withSQLConf( + "spark.sql.planChangeLog.level" -> "info", + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled" -> "true" + ) { + val df = sql(query) + df.collect() + val analyzedPlan = df.queryExecution.analyzed + // scalastyle:off println + println(analyzedPlan.toString) + // scalastyle:on println + } + } } From 9794f0f07c39f216c8db5c2ebb8658633a74cdbd Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 15:02:50 -0700 Subject: [PATCH 38/48] ignore tests under nestedcorrelation in ThriftServerQueryTestSuite --- .../hive/thriftserver/ThriftServerQueryTestSuite.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 32e11f06ae8b1..b74013f7ab107 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -110,7 +110,15 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // VARIANT type "variant/named-function-arguments.sql", // SPARK-51516: Support the TIME data type by Thrift Server - "time.sql" + "time.sql", + // SPARK-50983: Currently nested correlations tests are using + // analyzer results as results. The expected segment sizes are different + // from other testcases. + "subquery/nestedcorrelation/combined-subquery.sql", + "subquery/nestedcorrelation/exists-subquery.sql", + "subquery/nestedcorrelation/lateral-subquery.sql", + "subquery/nestedcorrelation/scalar-subquery.sql", + "subquery/nestedcorrelation/subquery-not-supported.sql" ) override def runQueries( From 8c3ce16a06bfc363e3d187c26b91614dabfe66d5 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 21 Apr 2025 15:33:17 -0700 Subject: [PATCH 39/48] rename nestedOuterAttrs to outerScopeAttrs --- .../resources/error/error-conditions.json | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 38 ++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 34 ++++---- .../analysis/ValidateSubqueryExpression.scala | 14 ++-- .../catalyst/expressions/DynamicPruning.scala | 10 +-- ...ctionTableSubqueryArgumentExpression.scala | 18 ++--- .../sql/catalyst/expressions/subquery.scala | 78 +++++++++---------- .../sql/catalyst/optimizer/subquery.scala | 16 ++-- 8 files changed, 105 insertions(+), 105 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index a8123d2163bde..e622b2fd1904b 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4066,7 +4066,7 @@ }, "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED" : { "message" : [ - "Detected nested outer references in the subquery.This is not supported in the current version." + "Detected outer scope references in the subquery.This is not supported in the current version." ], "sqlState" : "0A000" }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9bc7bd361ec5e..6a2df41b5ee62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2318,16 +2318,16 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor object ResolveSubquery extends Rule[LogicalPlan] { /** - * Returns the nested outer attributes referenced in the subquery expressions + * Returns the outer scope attributes referenced in the subquery expressions * in current plan and the children of the current plan. */ private def getOuterAttrsNeedToBePropagated(plan: LogicalPlan): Seq[Expression] = { plan.expressions.flatMap { - case subExpr: SubqueryExpression => subExpr.getNestedOuterAttrs - case in: InSubquery => in.query.getNestedOuterAttrs + case subExpr: SubqueryExpression => subExpr.getOuterScopeAttrs + case in: InSubquery => in.query.getOuterScopeAttrs case expr if expr.containsPattern(PLAN_EXPRESSION) => expr.collect { - case subExpr: SubqueryExpression => subExpr.getNestedOuterAttrs + case subExpr: SubqueryExpression => subExpr.getOuterScopeAttrs }.flatten case _ => Seq.empty } ++ plan.children.flatMap{ @@ -2353,7 +2353,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // We don't allow lateral subquery having nested correlation !e.isInstanceOf[LateralSubquery] ) { - // The previous outerPlanContext contains resolved nested outer plans + // The previous outerPlanContext contains resolved outer scope plans // and unresolved direct outer plan. Append the current outer plan into // new outerPlanContext as current outer is guaranteed to be resolved. val updatedOuterPlan = Seq(outer) ++ outerPlanContext.get @@ -2380,8 +2380,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor /** * Returns the outer references that are not resolved in the current plan {{p}}. - * These outer references are nested outer references which can be resolved - * in nested outer plans. + * These outer references are outer scope references which can be resolved + * in outer scope plans. * If these references cannot be resolved in the whole query plan, an analysis * exception will be thrown in checkAnalysis or ColumnResolutionHelper$resolve. */ @@ -2419,7 +2419,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // which cannot be resolved in current `plan` // It is extracted by `SubExprUtils.getOuterReferences(res.plan)` with // `getNestedOuterReferences(res, plan)` filter and stored in - // res.nestedOuterAttrs + // res.outerScopeAttrs // 3. Outer references which are introduced by nested subquery within `res.plan` // which can be resolved in current `plan` // It is extracted by `getOuterAttrsNeedToBePropagated(res.plan)`, filtered @@ -2428,22 +2428,22 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // which cannot be resolved in current `plan` // It is extracted by `getOuterAttrsNeedToBePropagated(res.plan)`, filtered // by `!plan.inputSet.contains(_)`, need to be stored in - // res.outerAttrs and res.nestedOuterAttrs + // res.outerAttrs and res.outerScopeAttrs case s @ ScalarSubquery(sub, _, _, exprId, _, _, _, _) if !sub.resolved => val res = resolveSubQuery(s, outer)(ScalarSubquery(_, _, Seq.empty, exprId)) val nestedOuterReferences = getNestedOuterReferences(res, plan) - res.withNewNestedOuterAttrs(nestedOuterReferences) + res.withNewOuterScopeAttrs(nestedOuterReferences) case e @ Exists(sub, _, _, exprId, _, _) if !sub.resolved => val res = resolveSubQuery(e, outer)(Exists(_, _, Seq.empty, exprId)) val nestedOuterReferences = getNestedOuterReferences(res, plan) - res.withNewNestedOuterAttrs(nestedOuterReferences) + res.withNewOuterScopeAttrs(nestedOuterReferences) case InSubquery(values, l) if values.forall(_.resolved) && !l.resolved => val expr = resolveSubQuery(l, outer)((plan, exprs) => { ListQuery(plan, exprs, Seq.empty, l.exprId, plan.output.length) }).asInstanceOf[ListQuery] val nestedOuterReferences = getNestedOuterReferences(expr, plan) - val newExpr = expr.withNewNestedOuterAttrs(nestedOuterReferences) + val newExpr = expr.withNewOuterScopeAttrs(nestedOuterReferences) InSubquery(values, newExpr) case s @ LateralSubquery(sub, _, _, exprId, _, _) if !sub.resolved => val res = resolveSubQuery(s, outer)(LateralSubquery(_, _, Seq.empty, exprId)) @@ -2901,13 +2901,13 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor object ResolveAggregateFunctions extends Rule[LogicalPlan] { def updateSubqueryOuterReferences(expression: Expression, aggregate: Aggregate): Expression = { expression.transformUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case sub: SubqueryExpression if sub.getNestedOuterAttrs.nonEmpty => - val newNestedOuterAttrs = - sub.getNestedOuterAttrs.filter( outerExpr => outerExpr match { + case sub: SubqueryExpression if sub.getOuterScopeAttrs.nonEmpty => + val newOuterScopeAttrs = + sub.getOuterScopeAttrs.filter( outerExpr => outerExpr match { case a: AttributeReference => !aggregate.outputSet.contains(a) case _ => true }) - sub.withNewNestedOuterAttrs(newNestedOuterAttrs) + sub.withNewOuterScopeAttrs(newOuterScopeAttrs) } } @@ -2926,7 +2926,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case UnresolvedHaving(cond, agg: Aggregate) if agg.resolved && cond.resolved => resolveOperatorWithAggregate(Seq(cond), agg, (newExprs, newChild) => { // Update the subquery in having clause as the aggregate output may be changed - // after the resolution. Some outer references being marked as nested outer + // after the resolution. Some outer references being marked as outer scope // references might be removed. val headCond = newExprs.head val newCond = updateSubqueryOuterReferences(headCond, newChild) @@ -4267,11 +4267,11 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { s: SubqueryExpression, outerAliases: Seq[Alias]): SubqueryExpression = { val planWithNestedSubqueriesRewritten = s.plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { - // Only update the nested subqueries if they have nested outer references + // Only update the nested subqueries if they have outer scope references // And we don't collect new outerAliases along s.plan because this rule // will be fired multiple times for each subquery plan in the Analyzer, // we only collect outerAliases in the outer plan each time. - case s: SubqueryExpression if s.getNestedOuterAttrs.nonEmpty => + case s: SubqueryExpression if s.getOuterScopeAttrs.nonEmpty => updateOuterReferenceInAllSubqueries(s, outerAliases) } val newPlan = updateOuterReferenceInSubquery(planWithNestedSubqueriesRewritten, outerAliases) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 2e0c278db1402..c32e8abb91218 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -229,19 +229,19 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } def checkNoNestedOuterReferencesInMainQuery(plan: LogicalPlan): Unit = { - def hasNestedOuterAttrsInSubqueryExpression(expr: Expression): Boolean = { + def hasOuterScopeAttrsInSubqueryExpression(expr: Expression): Boolean = { expr.exists { - case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => true + case subExpr: SubqueryExpression if subExpr.getOuterScopeAttrs.nonEmpty => true case _ => false } } - def getNestedOuterAttrsFromSubqueryExpression( + def getOuterScopeAttrsFromSubqueryExpression( plan: LogicalPlan): Seq[(SubqueryExpression, AttributeSet)] = { val res = plan.expressions.flatMap { expr => expr.collect { - case subExpr: SubqueryExpression if subExpr.getNestedOuterAttrs.nonEmpty => - (subExpr, subExpr.getNestedOuterAttrs) + case subExpr: SubqueryExpression if subExpr.getOuterScopeAttrs.nonEmpty => + (subExpr, subExpr.getOuterScopeAttrs) } } res.map { @@ -254,25 +254,25 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString } def findFirstOccurence( plan: LogicalPlan, - nestedOuterAttrs: AttributeSet, + outerScopeAttrs: AttributeSet, operator: LogicalPlan): (LogicalPlan, AttributeSet) = { val firstOccuredOperator = operator plan.foreach { - case p if p.expressions.exists(hasNestedOuterAttrsInSubqueryExpression) => - val res = getNestedOuterAttrsFromSubqueryExpression(p) - res.find(_._2.intersect(nestedOuterAttrs).nonEmpty) match { - case Some((subExpr, nestedOuterAttrsInP)) => + case p if p.expressions.exists(hasOuterScopeAttrsInSubqueryExpression) => + val res = getOuterScopeAttrsFromSubqueryExpression(p) + res.find(_._2.intersect(outerScopeAttrs).nonEmpty) match { + case Some((subExpr, outerScopeAttrsInP)) => return findFirstOccurence(subExpr.plan, - nestedOuterAttrsInP.intersect(nestedOuterAttrs), p) + outerScopeAttrsInP.intersect(outerScopeAttrs), p) case None => // Do nothing } case _ => // Do nothing } - (firstOccuredOperator, nestedOuterAttrs) + (firstOccuredOperator, outerScopeAttrs) } - def throwUnresolvedColumnErrorForNestedOuterAttrs(plan: LogicalPlan): Unit = { - val (subExpr, nestedOuterAttrs) = getNestedOuterAttrsFromSubqueryExpression(plan).head - val (operator, missingInput) = findFirstOccurence(subExpr.plan, nestedOuterAttrs, plan) + def throwUnresolvedColumnErrorForOuterScopeAttrs(plan: LogicalPlan): Unit = { + val (subExpr, outerScopeAttrs) = getOuterScopeAttrsFromSubqueryExpression(plan).head + val (operator, missingInput) = findFirstOccurence(subExpr.plan, outerScopeAttrs, plan) operator.failAnalysis( errorClass = "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", messageParameters = Map( @@ -283,8 +283,8 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString ) } plan.foreach { - case p: LogicalPlan if p.expressions.exists(hasNestedOuterAttrsInSubqueryExpression) => - throwUnresolvedColumnErrorForNestedOuterAttrs(p) + case p: LogicalPlan if p.expressions.exists(hasOuterScopeAttrsInSubqueryExpression) => + throwUnresolvedColumnErrorForOuterScopeAttrs(p) case _ => } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 57e632fce8ac2..36efde6e7efce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -115,9 +115,9 @@ object ValidateSubqueryExpression case o@OuterReference(a) => p.children.foreach(e => if (!e.output.exists(_.exprId == o.exprId) && - !expr.getNestedOuterAttrs.contains(a)) { + !expr.getOuterScopeAttrs.contains(a)) { // If the outer reference is not found in the children plan, - // it should be a nested outer reference. Otherwise, it is + // it should be a outer scope reference. Otherwise, it is // invalid. o.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + @@ -131,12 +131,12 @@ object ValidateSubqueryExpression } def checkNestedOuterReferences(expr: SubqueryExpression): Unit = { - if (expr.getNestedOuterAttrs.nonEmpty) { + if (expr.getOuterScopeAttrs.nonEmpty) { if (!SQLConf.get.getConf(SQLConf.SUPPORT_NESTED_CORRELATED_SUBQUERIES)) { throw new AnalysisException( errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", messageParameters = Map( - "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + "expression" -> expr.getOuterScopeAttrs.map(_.sql).mkString(",")) ) } expr match { @@ -146,7 +146,7 @@ object ValidateSubqueryExpression throw new AnalysisException( errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", messageParameters = Map( - "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + "expression" -> expr.getOuterScopeAttrs.map(_.sql).mkString(",")) ) case _: ListQuery if !SQLConf.get.getConf( @@ -154,7 +154,7 @@ object ValidateSubqueryExpression throw new AnalysisException( errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", messageParameters = Map( - "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + "expression" -> expr.getOuterScopeAttrs.map(_.sql).mkString(",")) ) case _: Exists if !SQLConf.get.getConf( @@ -162,7 +162,7 @@ object ValidateSubqueryExpression throw new AnalysisException( errorClass = "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", messageParameters = Map( - "expression" -> expr.getNestedOuterAttrs.map(_.sql).mkString(",")) + "expression" -> expr.getOuterScopeAttrs.map(_.sql).mkString(",")) ) case _ => // Do nothing } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index ebb4e4591a006..dd78457fc915e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -68,13 +68,13 @@ case class DynamicPruningSubquery( copy() } - override def withNewNestedOuterAttrs( - nestedOuterAttrs: Seq[Expression] + override def withNewOuterScopeAttrs( + outerScopeAttrs: Seq[Expression] ): DynamicPruningSubquery = { - // DynamicPruningSubquery should not have nested outer attrs - if (nestedOuterAttrs.nonEmpty) { + // DynamicPruningSubquery should not have outer scope attrs + if (outerScopeAttrs.nonEmpty) { throw SparkException.internalError( - "DynamicPruningSubquery should not have nested outer attributes.") + "DynamicPruningSubquery should not have outer scope attributes.") } copy() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index bfe8d1ab8f27d..06ee41423ccfe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -67,14 +67,14 @@ import org.apache.spark.sql.types.DataType case class FunctionTableSubqueryArgumentExpression( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - nestedOuterAttrs: Seq[Expression] = Seq.empty, + outerScopeAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, partitionByExpressions: Seq[Expression] = Seq.empty, withSinglePartition: Boolean = false, orderByExpressions: Seq[SortOrder] = Seq.empty, selectedInputExpressions: Seq[PythonUDTFSelectedExpression] = Seq.empty) extends SubqueryExpression( - plan, outerAttrs, nestedOuterAttrs, exprId, Seq.empty, None) with Unevaluable { + plan, outerAttrs, outerScopeAttrs, exprId, Seq.empty, None) with Unevaluable { assert(!(withSinglePartition && partitionByExpressions.nonEmpty), "WITH SINGLE PARTITION is mutually exclusive with PARTITION BY") @@ -88,20 +88,20 @@ case class FunctionTableSubqueryArgumentExpression( override def hint: Option[HintInfo] = None override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() - override def withNewNestedOuterAttrs( - nestedOuterAttrs: Seq[Expression] + override def withNewOuterScopeAttrs( + outerScopeAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + copy(outerScopeAttrs = outerScopeAttrs) } override def toString: String = s"table-argument#${exprId.id} $conditionString" override lazy val canonicalized: Expression = { FunctionTableSubqueryArgumentExpression( plan.canonicalized, outerAttrs.map(_.canonicalized), - nestedOuterAttrs.map(_.canonicalized), + outerScopeAttrs.map(_.canonicalized), ExprId(0), partitionByExpressions, withSinglePartition, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index c89bf528dabee..38df172d68721 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -67,7 +67,7 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { * * @param plan: the subquery plan * @param outerAttrs: the outer references in the subquery plan - * @param nestedOuterAttrs: the outer references in the subquery plan that cannot be resolved + * @param outerScopeAttrs: the outer references in the subquery plan that cannot be resolved * in its immediate parent plan * @param exprId: ID of the expression * @param joinCond: the join conditions with the outer query. It contains both inner and outer @@ -78,19 +78,19 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { abstract class SubqueryExpression( plan: LogicalPlan, outerAttrs: Seq[Expression], - nestedOuterAttrs: Seq[Expression], + outerScopeAttrs: Seq[Expression], exprId: ExprId, joinCond: Seq[Expression], hint: Option[HintInfo]) extends PlanExpression[LogicalPlan] { override lazy val resolved: Boolean = childrenResolved && plan.resolved override lazy val references: AttributeSet = AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) -- - AttributeSet.fromAttributeSets(nestedOuterAttrs.map(_.references)) + AttributeSet.fromAttributeSets(outerScopeAttrs.map(_.references)) override def children: Seq[Expression] = outerAttrs ++ joinCond override def withNewPlan(plan: LogicalPlan): SubqueryExpression def withNewOuterAttrs(outerAttrs: Seq[Expression]): SubqueryExpression - def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): SubqueryExpression - def getNestedOuterAttrs: Seq[Expression] = nestedOuterAttrs + def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): SubqueryExpression + def getOuterScopeAttrs: Seq[Expression] = outerScopeAttrs def getOuterAttrs: Seq[Expression] = outerAttrs def getJoinCond: Seq[Expression] = joinCond def isCorrelated: Boolean = outerAttrs.nonEmpty @@ -403,14 +403,14 @@ object SubExprUtils extends PredicateHelper { case class ScalarSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - nestedOuterAttrs: Seq[Expression] = Seq.empty, + outerScopeAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None, mayHaveCountBug: Option[Boolean] = None, needSingleJoin: Option[Boolean] = None) extends SubqueryExpression( - plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, @@ -422,13 +422,13 @@ case class ScalarSubquery( override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ScalarSubquery = copy( outerAttrs = outerAttrs) - override def withNewNestedOuterAttrs( - nestedOuterAttrs: Seq[Expression] + override def withNewOuterScopeAttrs( + outerScopeAttrs: Seq[Expression] ): ScalarSubquery = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + copy(outerScopeAttrs = outerScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" @@ -436,7 +436,7 @@ case class ScalarSubquery( ScalarSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), - nestedOuterAttrs.map(_.canonicalized), + outerScopeAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -493,25 +493,25 @@ case class UnresolvedTableArgPlanId( case class LateralSubquery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - nestedOuterAttrs: Seq[Expression] = Seq.empty, + outerScopeAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) extends SubqueryExpression( - plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { override def dataType: DataType = plan.output.toStructType override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) - override def withNewNestedOuterAttrs( - nestedOuterAttrs: Seq[Expression] + override def withNewOuterScopeAttrs( + outerScopeAttrs: Seq[Expression] ): LateralSubquery = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + copy(outerScopeAttrs = outerScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) @@ -520,7 +520,7 @@ case class LateralSubquery( LateralSubquery( plan.canonicalized, outerAttrs.map(_.canonicalized), - nestedOuterAttrs.map(_.canonicalized), + outerScopeAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } @@ -549,7 +549,7 @@ case class LateralSubquery( case class ListQuery( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - nestedOuterAttrs: Seq[Expression] = Seq.empty, + outerScopeAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, // The plan of list query may have more columns after de-correlation, and we need to track the // number of the columns of the original plan, to report the data type properly. @@ -557,7 +557,7 @@ case class ListQuery( joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) extends SubqueryExpression( - plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) with Unevaluable { + plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { def childOutputs: Seq[Attribute] = plan.output.take(numCols) override def dataType: DataType = if (numCols > 1) { childOutputs.toStructType @@ -577,11 +577,11 @@ case class ListQuery( override def withNewPlan(plan: LogicalPlan): ListQuery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) - override def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): ListQuery = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): ListQuery = { + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + copy(outerScopeAttrs = outerScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) override def toString: String = s"list#${exprId.id} $conditionString" @@ -589,7 +589,7 @@ case class ListQuery( ListQuery( plan.canonicalized, outerAttrs.map(_.canonicalized), - nestedOuterAttrs.map(_.canonicalized), + outerScopeAttrs.map(_.canonicalized), ExprId(0), numCols, joinCond.map(_.canonicalized)) @@ -632,22 +632,22 @@ case class ListQuery( case class Exists( plan: LogicalPlan, outerAttrs: Seq[Expression] = Seq.empty, - nestedOuterAttrs: Seq[Expression] = Seq.empty, + outerScopeAttrs: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId, joinCond: Seq[Expression] = Seq.empty, hint: Option[HintInfo] = None) - extends SubqueryExpression(plan, outerAttrs, nestedOuterAttrs, exprId, joinCond, hint) + extends SubqueryExpression(plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Predicate with Unevaluable { override def nullable: Boolean = false override def withNewPlan(plan: LogicalPlan): Exists = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) - override def withNewNestedOuterAttrs(nestedOuterAttrs: Seq[Expression]): Exists = { - assert(nestedOuterAttrs.toSet.subsetOf(outerAttrs.toSet), - s"nestedOuterAttrs must be a subset of outerAttrs, " + - s"but got ${nestedOuterAttrs.mkString(", ")}") - copy(nestedOuterAttrs = nestedOuterAttrs) + override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): Exists = { + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + copy(outerScopeAttrs = outerScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) override def toString: String = s"exists#${exprId.id} $conditionString" @@ -655,7 +655,7 @@ case class Exists( Exists( plan.canonicalized, outerAttrs.map(_.canonicalized), - nestedOuterAttrs.map(_.canonicalized), + outerScopeAttrs.map(_.canonicalized), ExprId(0), joinCond.map(_.canonicalized)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 99111db8b3122..0c94d3c714730 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -587,7 +587,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { - case ScalarSubquery(sub, children, nestedOuterAttrs, exprId, conditions, hint, + case ScalarSubquery(sub, children, outerScopeAttrs, exprId, conditions, hint, mayHaveCountBugOld, needSingleJoinOld) if children.nonEmpty => @@ -639,26 +639,26 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper } else { conf.getConf(SQLConf.SCALAR_SUBQUERY_USE_SINGLE_JOIN) && !guaranteedToReturnOneRow(sub) } - ScalarSubquery(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), + ScalarSubquery(newPlan, children, outerScopeAttrs, exprId, getJoinCondition(newCond, conditions), hint, Some(mayHaveCountBug), Some(needSingleJoin)) - case Exists(sub, children, nestedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => + case Exists(sub, children, outerScopeAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } - Exists(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) - case ListQuery(sub, children, nestedOuterAttrs, exprId, numCols, conditions, hint) if children.nonEmpty => + Exists(newPlan, children, outerScopeAttrs, exprId, getJoinCondition(newCond, conditions), hint) + case ListQuery(sub, children, outerScopeAttrs, exprId, numCols, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = if (SQLConf.get.decorrelateInnerQueryEnabledForExistsIn) { decorrelate(sub, plan, handleCountBug = true) } else { pullOutCorrelatedPredicates(sub, plan) } val joinCond = getJoinCondition(newCond, conditions) - ListQuery(newPlan, children, nestedOuterAttrs, exprId, numCols, joinCond, hint) - case LateralSubquery(sub, children, nestedOuterAttrs, exprId, conditions, hint) if children.nonEmpty => + ListQuery(newPlan, children, outerScopeAttrs, exprId, numCols, joinCond, hint) + case LateralSubquery(sub, children, outerScopeAttrs, exprId, conditions, hint) if children.nonEmpty => val (newPlan, newCond) = decorrelate(sub, plan, handleCountBug = true) - LateralSubquery(newPlan, children, nestedOuterAttrs, exprId, getJoinCondition(newCond, conditions), hint) + LateralSubquery(newPlan, children, outerScopeAttrs, exprId, getJoinCondition(newCond, conditions), hint) } } From a18e598f38544adc37ca84164c5783bab434c04a Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 22 Apr 2025 11:37:26 -0700 Subject: [PATCH 40/48] resolve comments --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 +++--- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 1 + .../spark/sql/catalyst/expressions/DynamicPruning.scala | 3 +-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6a2df41b5ee62..430dc0e37ecf3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -228,7 +228,7 @@ object AnalysisContext { try f finally { set(originContext) } } - def withOuterPlans[A](outerPlans: Seq[LogicalPlan])(f: => A): A = { + def withOuterPlan[A](outerPlans: Seq[LogicalPlan])(f: => A): A = { val originContext = value.get() val context = originContext.copy(outerPlans = Some(outerPlans)) set(context) @@ -2357,11 +2357,11 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // and unresolved direct outer plan. Append the current outer plan into // new outerPlanContext as current outer is guaranteed to be resolved. val updatedOuterPlan = Seq(outer) ++ outerPlanContext.get - AnalysisContext.withOuterPlans(updatedOuterPlan) { + AnalysisContext.withOuterPlan(updatedOuterPlan) { executeSameContext(e.plan) } } else { - AnalysisContext.withOuterPlans(Seq(outer)) { + AnalysisContext.withOuterPlan(Seq(outer)) { executeSameContext(e.plan) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c32e8abb91218..f66fbd33c8674 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -252,6 +252,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString (subExpr, AttributeSet(attrs)) } } + def findFirstOccurence( plan: LogicalPlan, outerScopeAttrs: AttributeSet, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index dd78457fc915e..e549f398b18bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -71,12 +71,11 @@ case class DynamicPruningSubquery( override def withNewOuterScopeAttrs( outerScopeAttrs: Seq[Expression] ): DynamicPruningSubquery = { - // DynamicPruningSubquery should not have outer scope attrs if (outerScopeAttrs.nonEmpty) { throw SparkException.internalError( "DynamicPruningSubquery should not have outer scope attributes.") } - copy() + this } override def withNewHint(hint: Option[HintInfo]): SubqueryExpression = copy(hint = hint) From 93d200358cbdf72ad33bec05dc401fa64efa02f4 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 23 Apr 2025 11:55:03 -0700 Subject: [PATCH 41/48] revert deduplication because we don't want to change current behavior --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../analyzer-results/join-lateral.sql.out | 38 +++++++++--------- .../named-function-arguments.sql.out | 6 +-- .../exists-subquery/exists-basic.sql.out | 2 +- .../exists-joins-and-set-ops.sql.out | 16 ++++---- .../nested-scalar-subquery-count-bug.sql.out | 4 +- .../scalar-subquery-group-by.sql.out | 6 +-- .../scalar-subquery-predicate.sql.out | 6 +-- .../scalar-subquery-select.sql.out | 4 +- .../scalar-subquery-set-op.sql.out | 40 +++++++++---------- .../subquery/subquery-nested-data.sql.out | 10 ++--- .../nestedcorrelation/exists-subquery.sql.out | 4 +- .../nestedcorrelation/scalar-subquery.sql.out | 22 +++++----- .../scalar-subquery-group-by.sql.out | 6 +-- .../scalar-subquery-select.sql.out | 4 +- .../scalar-subquery-set-op.sql.out | 40 +++++++++---------- 16 files changed, 105 insertions(+), 105 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 430dc0e37ecf3..4e7e7b5599986 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2372,7 +2372,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor // Record the outer references as children of subquery expression. val outer = SubExprUtils.getOuterReferences(newSubqueryPlan) ++ getOuterAttrsNeedToBePropagated(newSubqueryPlan) - f(newSubqueryPlan, outer.distinct) + f(newSubqueryPlan, outer) } else { e.withNewPlan(newSubqueryPlan) } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 8246bb089fe66..f8e88a57bf426 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -635,7 +635,7 @@ Project [c1#x, c2#x, c1#x, m#x, m#x] +- LateralJoin lateral-subquery#x [c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [c1#x, m#x, m#x] - : +- LateralJoin lateral-subquery#x [m#x && c1#x], Inner + : +- LateralJoin lateral-subquery#x [m#x && m#x && c1#x], Inner : : +- SubqueryAlias __auto_generated_subquery_name : : +- Project [outer(m#x) AS m#x] : : +- Filter (outer(m#x) > outer(c1#x)) @@ -1201,7 +1201,7 @@ Project [c1#x, c2#x, count(1)#xL] SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) -- !query analysis Project [c1#x, c2#x, cnt#xL] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [cnt#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1222,7 +1222,7 @@ Project [c1#x, c2#x, cnt#xL] SELECT * FROM t1, LATERAL (SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) -- !query analysis Project [c1#x, c2#x, count(1)#xL] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Aggregate [count(1) AS count(1)#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1246,7 +1246,7 @@ SELECT * FROM t1, LATERAL ( ) -- !query analysis Project [c1#x, c2#x, count(1)#xL] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Aggregate [cnt#xL], [count(1) AS count(1)#xL] : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) @@ -1299,7 +1299,7 @@ SELECT * FROM t1, LATERAL ( ) -- !query analysis Project [c1#x, c2#x, (cnt + cnt)#xL] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(cnt#xL + cnt#xL) AS (cnt + cnt)#xL] : +- Join Inner @@ -1472,7 +1472,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c2#x] @@ -1537,7 +1537,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true : :- Project [c2#x] @@ -1601,7 +1601,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true : :- Project [c2#x] @@ -1698,7 +1698,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 = t1.c1) -- !query analysis Project [c1#x, c2#x, c1#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c1#x, c2#x] @@ -2078,7 +2078,7 @@ SELECT * FROM t2 JOIN LATERAL -- !query analysis Union false, false :- Project [c1#x, c2#x, c2#x] -: +- LateralJoin lateral-subquery#x [c1#x], Inner +: +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : : +- SubqueryAlias __auto_generated_subquery_name : : +- Union false, false : : :- Project [c2#x] @@ -2098,7 +2098,7 @@ Union false, false : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] +- Project [c1#x, c2#x, c2#x] - +- LateralJoin lateral-subquery#x [c1#x], Inner + +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [c2#x] @@ -2139,7 +2139,7 @@ SELECT * FROM t1 JOIN LATERAL ) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct : +- Union false, false @@ -2195,7 +2195,7 @@ SELECT * FROM t1 JOIN LATERAL ) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false : :- Union false, false @@ -2437,7 +2437,7 @@ Project [c1#x, c2#x, col#x] SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3)) -- !query analysis Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] : +- SubqueryAlias t @@ -2454,7 +2454,7 @@ Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1) -- !query analysis Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c1#x && c2#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] : +- Filter (outer(c2#x) > 1) @@ -2472,7 +2472,7 @@ Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y) -- !query analysis Project [c1#x, c2#x, x#x, y#x] -+- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [x#x, y#x] : +- Join Inner, (x#x = y#x) @@ -2947,7 +2947,7 @@ select * from t1 join lateral (select t4.c1 from t4 where t1.c1 = t4.c1 order by t4.c1 limit 3)) -- !query analysis Project [c1#x, c2#x, c2#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- GlobalLimit 1 @@ -2983,7 +2983,7 @@ select * from t1 join lateral order by foo.t limit 5) -- !query analysis Project [c1#x, c2#x, t#x] -+- LateralJoin lateral-subquery#x [c1#x], Inner ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- GlobalLimit 5 : +- LocalLimit 5 @@ -3031,7 +3031,7 @@ left join ) as lateral_table -- !query analysis Project [1 AS 1#x] -+- LateralJoin lateral-subquery#x [c2#x && c1#x], LeftOuter ++- LateralJoin lateral-subquery#x [c2#x && c1#x && c1#x], LeftOuter : +- SubqueryAlias lateral_table : +- GlobalLimit 1 : +- LocalLimit 1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out index b656e40e3fc20..2315a5f0678a0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out @@ -131,7 +131,7 @@ Project [col#x, col#x] SELECT * FROM explode(array(1, 2)) AS t, LATERAL explode(array(3 * t.col, 4 * t.col)) -- !query analysis Project [col#x, col#x] -+- LateralJoin lateral-subquery#x [col#x], Inner ++- LateralJoin lateral-subquery#x [col#x && col#x], Inner : +- Generate explode(array((3 * outer(col#x)), (4 * outer(col#x)))), false, [col#x] : +- OneRowRelation +- SubqueryAlias t @@ -180,7 +180,7 @@ Project [pos#x, col#x, pos#x, col#x] SELECT * FROM posexplode(array(1, 2)) AS t, LATERAL posexplode(array(3 * t.col, 4 * t.col)) -- !query analysis Project [pos#x, col#x, pos#x, col#x] -+- LateralJoin lateral-subquery#x [col#x], Inner ++- LateralJoin lateral-subquery#x [col#x && col#x], Inner : +- Generate posexplode(array((3 * outer(col#x)), (4 * outer(col#x)))), false, [pos#x, col#x] : +- OneRowRelation +- SubqueryAlias t @@ -229,7 +229,7 @@ Project [col1#x, col2#x, col1#x, col2#x] SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t, LATERAL inline(array(struct(3 * t.col1, 4 * t.col1))) -- !query analysis Project [col1#x, col2#x, col1#x, col2#x] -+- LateralJoin lateral-subquery#x [col1#x], Inner ++- LateralJoin lateral-subquery#x [col1#x && col1#x], Inner : +- Generate inline(array(struct(col1, (3 * outer(col1#x)), col2, (4 * outer(col1#x))))), false, [col1#x, col2#x] : +- OneRowRelation +- SubqueryAlias t diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out index 02de5f287d48b..81c4a15dc9f4a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out @@ -138,7 +138,7 @@ WHERE EXISTS (SELECT dept.dept_name OR emp.dept_id IS NULL) -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Project [dept_name#x] : +- Filter ((outer(dept_id#x) = dept_id#x) OR isnull(outer(dept_id#x))) : +- SubqueryAlias dept diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out index 2fbe9f3cb8a8b..55b10125f7680 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -686,7 +686,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Distinct : +- Union false, false : :- Project [dept_id#x, dept_name#x, state#x] @@ -725,7 +725,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x] ++- Filter NOT exists#x [dept_id#x && dept_id#x] : +- Distinct : +- Union false, false : :- Project [dept_id#x, dept_name#x, state#x] @@ -764,7 +764,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Intersect All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -802,7 +802,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Intersect false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -840,7 +840,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Except All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -878,7 +878,7 @@ WHERE EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter exists#x [dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] : +- Except false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -916,7 +916,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x] ++- Filter NOT exists#x [dept_id#x && dept_id#x] : +- Intersect All true : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) @@ -954,7 +954,7 @@ WHERE NOT EXISTS (SELECT * WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -+- Filter NOT exists#x [dept_id#x] ++- Filter NOT exists#x [dept_id#x && dept_id#x] : +- Except false : :- Project [dept_id#x, dept_name#x, state#x] : : +- Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out index a689a6cd690fa..2a1abc0d48871 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out @@ -79,7 +79,7 @@ select ( ) a from t1 order by a desc -- !query analysis Sort [a#xL DESC NULLS LAST], true -+- Project [scalar-subquery#x [a1#x] AS a#xL] ++- Project [scalar-subquery#x [a1#x && a1#x] AS a#xL] : +- Aggregate [sum((cnt#xL + cnt#xL)) AS sum((cnt + cnt))#xL] : +- Join Inner, (cnt#xL = cnt#xL) : :- SubqueryAlias l @@ -113,7 +113,7 @@ select ( ) a from t1 order by a desc -- !query analysis Sort [a#xL DESC NULLS LAST], true -+- Project [scalar-subquery#x [a1#x] AS a#xL] ++- Project [scalar-subquery#x [a1#x && a1#x] AS a#xL] : +- Aggregate [sum((cnt#xL + cnt#xL)) AS sum((cnt + cnt))#xL] : +- Join Inner, (cnt#xL = cnt#xL) : :- SubqueryAlias l diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out index a2b9843c1bc7e..01de7beda551d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out @@ -96,7 +96,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] -- !query select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] : +- Aggregate [y2#x], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND (y2#x = (outer(x1#x) + 1))) : +- SubqueryAlias y @@ -113,7 +113,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 group by cast(y2 as double)) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] : +- Aggregate [cast(y2#x as double)], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND (cast(y2#x as double) = cast((outer(x1#x) + 1) as double))) : +- SubqueryAlias y @@ -250,7 +250,7 @@ Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] -- !query select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x -- !query analysis -Project [x1#x, x2#x, scalar-subquery#x [x1#x] AS scalarsubquery(x1)#xL] +Project [x1#x, x2#x, scalar-subquery#x [x1#x && x1#x] AS scalarsubquery(x1, x1)#xL] : +- Aggregate [y2#x], [count(1) AS count(1)#xL] : +- Filter ((outer(x1#x) = y1#x) AND ((y2#x + 10) = (outer(x1#x) + 1))) : +- SubqueryAlias y diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 245bfbfe8b11f..0414782fb7e67 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -1320,7 +1320,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1461,7 +1461,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x]) : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1530,7 +1530,7 @@ SELECT * FROM t0 WHERE t0a < ) -- !query analysis Project [t0a#x, t0b#x] -+- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x]) ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index b3bbfbcb99147..f64b3736b5521 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -529,7 +529,7 @@ Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#x] -- !query SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 -- !query analysis -Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#x] +Project [t1c#x, scalar-subquery#x [t1c#x && t1c#x] AS scalarsubquery(t1c, t1c)#x] : +- Project [outer(t1c#x)] : +- Filter (outer(t1c#x) = 8) : +- OneRowRelation @@ -1023,7 +1023,7 @@ WithCTE : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation -+- Project [scalar-subquery#x [col#x] AS scalarsubquery(col)#xL] ++- Project [scalar-subquery#x [col#x && col#x] AS scalarsubquery(col, col)#xL] : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index 4df145e92d38f..eaeff0ba9dedb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -101,7 +101,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -203,7 +203,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -269,7 +269,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -399,7 +399,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -504,7 +504,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -572,7 +572,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Distinct @@ -701,7 +701,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -803,7 +803,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -869,7 +869,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect All true @@ -997,7 +997,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1099,7 +1099,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1165,7 +1165,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Intersect false @@ -1293,7 +1293,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1395,7 +1395,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1461,7 +1461,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except All true @@ -1589,7 +1589,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(c#x) AS sum(c)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1691,7 +1691,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1757,7 +1757,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(d#x) AS sum(d)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Except false @@ -1814,7 +1814,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(t1b#x) AS sum(t1b)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false @@ -1854,7 +1854,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query analysis -Project [t0a#x, scalar-subquery#x [t0a#x] AS scalarsubquery(t0a)#xL] +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] : +- Aggregate [sum(t1b#x) AS sum(t1b)#xL] : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out index eef900f1a1b5f..a218238073dff 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-nested-data.sql.out @@ -100,7 +100,7 @@ Project [xm#x, x2#x, ym#x, y2#x] select * from x join lateral (select * from y where xm[1] = ym[1] union all select * from y where xm[1] = ym[1] + 1) -- !query analysis Project [xm#x, x2#x, ym#x, y2#x] -+- LateralJoin lateral-subquery#x [xm#x], Inner ++- LateralJoin lateral-subquery#x [xm#x && xm#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Union false, false : :- Project [ym#x, y2#x] @@ -174,7 +174,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[1] >= 1) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x && xm#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[1] = ym#x[1]) AND (outer(xm#x)[1] >= 1)) : +- SubqueryAlias spark_catalog.default.y @@ -187,7 +187,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[2] >= ym[2]) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x && xm#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[1] = ym#x[1]) AND (outer(xm#x)[2] >= ym#x[2])) : +- SubqueryAlias spark_catalog.default.y @@ -230,7 +230,7 @@ Project [xm#x, x2#x, (outer(spark_catalog.default.x.xm)[1] - ym[1])#x] select * from x join lateral (select xm[1], xm[1] as s1, xm[1] - ym[1] as s2 from y) -- !query analysis Project [xm#x, x2#x, outer(spark_catalog.default.x.xm)[1]#x, s1#x, s2#x] -+- LateralJoin lateral-subquery#x [xm#x], Inner ++- LateralJoin lateral-subquery#x [xm#x && xm#x && xm#x], Inner : +- SubqueryAlias __auto_generated_subquery_name : +- Project [outer(xm#x)[1] AS outer(spark_catalog.default.x.xm)[1]#x, outer(xm#x)[1] AS s1#x, (outer(xm#x)[1] - ym#x[1]) AS s2#x] : +- SubqueryAlias spark_catalog.default.y @@ -289,7 +289,7 @@ Project [xm#x, x2#x] select * from x where (select sum(y2) from y where xm[x2+1] = ym[1] and xm[1+x2] = ym[2]) > 2 -- !query analysis Project [xm#x, x2#x] -+- Filter (scalar-subquery#x [xm#x && x2#x] > cast(2 as bigint)) ++- Filter (scalar-subquery#x [xm#x && x2#x && xm#x && x2#x] > cast(2 as bigint)) : +- Aggregate [sum(y2#x) AS sum(y2)#xL] : +- Filter ((outer(xm#x)[(outer(x2#x) + 1)] = ym#x[1]) AND (outer(xm#x)[(1 + outer(x2#x))] = ym#x[2])) : +- SubqueryAlias spark_catalog.default.y diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out index 601a3589e8f9a..c47867443fde7 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/exists-subquery.sql.out @@ -324,7 +324,7 @@ Sort [i#x ASC NULLS FIRST], true : +- Aggregate [sum(i#x) AS sum(i)#xL] : +- SubqueryAlias ss1 : +- Project [i#x] - : +- Filter exists#x [i#x] + : +- Filter exists#x [i#x && i#x] : : +- Project [i#x] : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) : : +- SubqueryAlias spark_catalog.default.table_integers @@ -346,7 +346,7 @@ Sort [i#x ASC NULLS FIRST], true : +- Join LeftOuter, (i#x = i#x) : :- SubqueryAlias ss1 : : +- Project [i#x] - : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x && i#x]) : : : +- Project [i#x] : : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) : : : +- SubqueryAlias spark_catalog.default.table_integers diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index c8512a1a56e33..76805b07c6851 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -63,7 +63,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] : : +- OneRowRelation @@ -77,9 +77,9 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] - : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] - : : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] + : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] : : : : +- OneRowRelation @@ -95,10 +95,10 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] - : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] : : : : +- OneRowRelation : : : +- OneRowRelation @@ -116,7 +116,7 @@ Sort [i#x ASC NULLS FIRST], true +- Project [i#x, scalar-subquery#x [i#x] AS j#x] : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : : +- Project [scalar-subquery#x [i#x && i#x] AS scalarsubquery(i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] : : : : +- OneRowRelation : : : +- Filter (i#x = outer(i#x)) @@ -207,7 +207,7 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] : +- Project [(k#x + k#x) AS (k + k)#x] : +- Join Inner : :- SubqueryAlias s1 @@ -229,7 +229,7 @@ Sort [i#x ASC NULLS FIRST], true SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i -- !query analysis Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] : +- Project [(k#x + k#x) AS (k + k)#x] : +- Join LeftOuter, (k#x = k#x) : :- SubqueryAlias s1 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out index 25dcab8798902..56932edd4e545 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out @@ -87,7 +87,7 @@ struct -- !query select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL @@ -97,7 +97,7 @@ struct select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 group by cast(y2 as double)) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL @@ -221,7 +221,7 @@ org.apache.spark.SparkRuntimeException -- !query select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x -- !query schema -struct +struct -- !query output 1 1 NULL 2 2 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 61186f8c7f009..85bd9137602a3 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -243,7 +243,7 @@ NULL NULL -- !query SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 -- !query schema -struct +struct -- !query output 12 NULL 12 NULL @@ -574,7 +574,7 @@ WITH T AS (SELECT 1 AS a) SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query schema -struct +struct -- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index 7b19017ff0407..33a57a73be08e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -70,7 +70,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 8 2 7 @@ -124,7 +124,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 32 2 NULL @@ -158,7 +158,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 -2 @@ -240,7 +240,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 8 2 7 @@ -294,7 +294,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 32 2 NULL @@ -328,7 +328,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 -1 @@ -410,7 +410,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -464,7 +464,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -498,7 +498,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 0 2 -1 @@ -580,7 +580,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -634,7 +634,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -668,7 +668,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 0 2 -1 @@ -750,7 +750,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 3 2 NULL @@ -804,7 +804,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 11 2 NULL @@ -838,7 +838,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -920,7 +920,7 @@ SELECT t0a, (SELECT sum(c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 3 2 NULL @@ -974,7 +974,7 @@ SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 11 2 NULL @@ -1008,7 +1008,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 NULL 2 NULL @@ -1053,7 +1053,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 2 2 NULL @@ -1069,7 +1069,7 @@ SELECT t0a, (SELECT sum(t1b) FROM ) FROM t0 -- !query schema -struct +struct -- !query output 1 1 2 1 From d86f4b64cdfab71815d073321d3bf1bf713dff41 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Fri, 14 Mar 2025 17:22:23 -0700 Subject: [PATCH 42/48] add unresolved outer attrs fix wrong number of arguments error; fix assertions fix wrong number of arguments error fix wrong number of arguments error fix for mis-deleting ScalarSubquery.withNewOuterAttrs fmt fix wrong number of arguments error fix wrong number of arguments error rename unresolved outer attrs to nested outer attrs throw internalErrors and format compile and format resolve comments rename nestedOuterAttrs to outerScopeAttrs Update DynamicPruning.scala Update FunctionTableSubqueryArgumentExpression.scala add new lines for readability --- .../catalyst/expressions/DynamicPruning.scala | 2 + ...ctionTableSubqueryArgumentExpression.scala | 24 +++++-- .../sql/catalyst/expressions/subquery.scala | 67 +++++++++++++++---- 3 files changed, 76 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala index e549f398b18bc..edc84f73289ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/DynamicPruning.scala @@ -30,6 +30,8 @@ trait DynamicPruning extends Predicate * The DynamicPruningSubquery expression is only used in join operations to prune one side of the * join with a filter from the other side of the join. It is inserted in cases where partition * pruning can be applied. + * The DynamicPruningSubquery expression should only have a single outer + * attribute which is the pruning key and should not have any outer scope attributes. * * @param pruningKey the filtering key of the plan to be pruned. * @param buildQuery the build side of the join. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index 06ee41423ccfe..fe63169767693 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -46,6 +46,10 @@ import org.apache.spark.sql.types.DataType * relation or as a more complex logical plan in the event of a table subquery. * @param outerAttrs outer references of this subquery plan, generally empty since these table * arguments do not allow correlated references currently + * @param outerScopeAttrs outer references of the subquery plan that cannot be resolved by the + * direct containing query of the subquery. They have to be the subset of + * outerAttrs and are generally empty since these table arguments do not + * allow correlated references currently * @param exprId expression ID of this subquery expression, generally generated afresh each time * @param partitionByExpressions if non-empty, the TABLE argument included the PARTITION BY clause * to indicate that the input relation should be repartitioned by the @@ -74,29 +78,41 @@ case class FunctionTableSubqueryArgumentExpression( orderByExpressions: Seq[SortOrder] = Seq.empty, selectedInputExpressions: Seq[PythonUDTFSelectedExpression] = Seq.empty) extends SubqueryExpression( - plan, outerAttrs, outerScopeAttrs, exprId, Seq.empty, None) with Unevaluable { + plan, + outerAttrs, + outerScopeAttrs, + exprId, + Seq.empty, + None + ) with Unevaluable { assert(!(withSinglePartition && partitionByExpressions.nonEmpty), "WITH SINGLE PARTITION is mutually exclusive with PARTITION BY") override def dataType: DataType = plan.schema + override def nullable: Boolean = false + override def withNewPlan(plan: LogicalPlan): FunctionTableSubqueryArgumentExpression = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]) : FunctionTableSubqueryArgumentExpression = copy(outerAttrs = outerAttrs) + override def hint: Option[HintInfo] = None + override def withNewHint(hint: Option[HintInfo]): FunctionTableSubqueryArgumentExpression = copy() + override def withNewOuterScopeAttrs( outerScopeAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), - s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + validateOuterScopeAttrs() copy(outerScopeAttrs = outerScopeAttrs) } + override def toString: String = s"table-argument#${exprId.id} $conditionString" + override lazy val canonicalized: Expression = { FunctionTableSubqueryArgumentExpression( plan.canonicalized, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 38df172d68721..52c5018678a6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -82,19 +82,37 @@ abstract class SubqueryExpression( exprId: ExprId, joinCond: Seq[Expression], hint: Option[HintInfo]) extends PlanExpression[LogicalPlan] { + override lazy val resolved: Boolean = childrenResolved && plan.resolved + override lazy val references: AttributeSet = AttributeSet.fromAttributeSets(outerAttrs.map(_.references)) -- - AttributeSet.fromAttributeSets(outerScopeAttrs.map(_.references)) + AttributeSet.fromAttributeSets(outerScopeAttrs.map(_.references)) + override def children: Seq[Expression] = outerAttrs ++ joinCond + override def withNewPlan(plan: LogicalPlan): SubqueryExpression + def withNewOuterAttrs(outerAttrs: Seq[Expression]): SubqueryExpression + def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): SubqueryExpression + + def validateOuterScopeAttrs(): Unit = { + assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + s"outerScopeAttrs must be a subset of outerAttrs, " + + s"but got ${outerScopeAttrs.mkString(", ")}") + } + def getOuterScopeAttrs: Seq[Expression] = outerScopeAttrs + def getOuterAttrs: Seq[Expression] = outerAttrs + def getJoinCond: Seq[Expression] = joinCond + def isCorrelated: Boolean = outerAttrs.nonEmpty + def hint: Option[HintInfo] + def withNewHint(hint: Option[HintInfo]): SubqueryExpression } @@ -411,6 +429,7 @@ case class ScalarSubquery( needSingleJoin: Option[Boolean] = None) extends SubqueryExpression( plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { + override def dataType: DataType = { if (!plan.schema.fields.nonEmpty) { throw QueryCompilationErrors.subqueryReturnMoreThanOneColumn(plan.schema.fields.length, @@ -418,20 +437,25 @@ case class ScalarSubquery( } plan.schema.fields.head.dataType } + override def nullable: Boolean = true + override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ScalarSubquery = copy( outerAttrs = outerAttrs) + override def withNewOuterScopeAttrs( outerScopeAttrs: Seq[Expression] ): ScalarSubquery = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), - s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + validateOuterScopeAttrs() copy(outerScopeAttrs = outerScopeAttrs) } + override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) + override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" + override lazy val canonicalized: Expression = { ScalarSubquery( plan.canonicalized, @@ -499,23 +523,27 @@ case class LateralSubquery( hint: Option[HintInfo] = None) extends SubqueryExpression( plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { + override def dataType: DataType = plan.output.toStructType + override def nullable: Boolean = true + override def withNewPlan(plan: LogicalPlan): LateralSubquery = copy(plan = plan) override def withNewOuterAttrs(outerAttrs: Seq[Expression]): LateralSubquery = copy( outerAttrs = outerAttrs) + override def withNewOuterScopeAttrs( outerScopeAttrs: Seq[Expression] ): LateralSubquery = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), - s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + validateOuterScopeAttrs() copy(outerScopeAttrs = outerScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) + override def toString: String = s"lateral-subquery#${exprId.id} $conditionString" + override lazy val canonicalized: Expression = { LateralSubquery( plan.canonicalized, @@ -558,13 +586,17 @@ case class ListQuery( hint: Option[HintInfo] = None) extends SubqueryExpression( plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Unevaluable { + def childOutputs: Seq[Attribute] = plan.output.take(numCols) + override def dataType: DataType = if (numCols > 1) { childOutputs.toStructType } else { plan.output.head.dataType } + override lazy val resolved: Boolean = childrenResolved && plan.resolved && numCols != -1 + override def nullable: Boolean = { // ListQuery can't be executed alone so its nullability is not defined. // Consider using ListQuery.childOutputs.exists(_.nullable) @@ -574,17 +606,21 @@ case class ListQuery( } false } + override def withNewPlan(plan: LogicalPlan): ListQuery = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) + override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): ListQuery = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), - s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + validateOuterScopeAttrs() copy(outerScopeAttrs = outerScopeAttrs) } + override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) + override def toString: String = s"list#${exprId.id} $conditionString" + override lazy val canonicalized: Expression = { ListQuery( plan.canonicalized, @@ -639,18 +675,23 @@ case class Exists( extends SubqueryExpression(plan, outerAttrs, outerScopeAttrs, exprId, joinCond, hint) with Predicate with Unevaluable { + override def nullable: Boolean = false + override def withNewPlan(plan: LogicalPlan): Exists = copy(plan = plan) + override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) + override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): Exists = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), - s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + validateOuterScopeAttrs() copy(outerScopeAttrs = outerScopeAttrs) } + override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) + override def toString: String = s"exists#${exprId.id} $conditionString" + override lazy val canonicalized: Expression = { Exists( plan.canonicalized, From c3aa2b9cdee427133a6cfd47ff2a44a19d271ba4 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 22 Apr 2025 15:56:24 -0700 Subject: [PATCH 43/48] init compile --- .../expressions/namedExpressions.scala | 43 ++ .../sql/catalyst/expressions/subquery.scala | 169 +++++- .../optimizer/DecorrelateInnerQuery.scala | 544 ++++++++++++++++-- .../sql/catalyst/optimizer/Optimizer.scala | 4 + .../sql/catalyst/optimizer/subquery.scala | 371 +++++++----- .../sql/catalyst/trees/TreePatterns.scala | 1 + 6 files changed, 961 insertions(+), 171 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 2af6a1ba84ec8..6a43e26e61ce9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -444,6 +444,49 @@ case class OuterReference(e: NamedExpression) final override val nodePatterns: Seq[TreePattern] = Seq(OUTER_REFERENCE) } +/** + * A place holder used to hold attributes cannot be solved by the subquery plan and + * its immediate parent plan. When rewriting subqueries into joins, joinConds containing + * OuterScopeReferences are transformed into filter predicates between the joins transformed + * from the subqueries and the outer plan. + * It can only be used in the intermediate results in the optimization stage, should not appear + * in the physical plan. + */ +case class OuterScopeReference(e: NamedExpression) + extends LeafExpression with NamedExpression with Unevaluable { + override def dataType: DataType = e.dataType + override def nullable: Boolean = e.nullable + override def prettyName: String = "outerScope" + + override def sql: String = s"$prettyName(${e.sql})" + override def name: String = e.name + override def qualifier: Seq[String] = e.qualifier + override def exprId: ExprId = e.exprId + override def toAttribute: Attribute = e.toAttribute + override def newInstance(): NamedExpression = OuterScopeReference(e.newInstance()) +} + +/** + * A place holder used to hold attributes need to be propagated up through subqueries. + * This should be only used in PullUpCorrelatedPredicates, RewritePredicateSubquery, + * RewriteLateralSubquery, RewriteCorrelatedScalarSubquery rules. + * It can only be used in the intermediate results in the optimization stage, should not appear + * in the physical plan. + */ +case class InnerReference(e: NamedExpression) + extends LeafExpression with NamedExpression with Unevaluable { + override def dataType: DataType = e.dataType + override def nullable: Boolean = e.nullable + override def prettyName: String = "inner" + + override def sql: String = s"$prettyName(${e.sql})" + override def name: String = e.name + override def qualifier: Seq[String] = e.qualifier + override def exprId: ExprId = e.exprId + override def toAttribute: Attribute = e.toAttribute + override def newInstance(): NamedExpression = InnerReference(e.newInstance()) +} + /** * A placeholder used to hold a [[NamedExpression]] that has been temporarily resolved as the * reference to a lateral column alias. It will be restored back to [[UnresolvedAttribute]] if diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 52c5018678a6b..c88ed966d111c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet @@ -114,6 +114,14 @@ abstract class SubqueryExpression( def hint: Option[HintInfo] def withNewHint(hint: Option[HintInfo]): SubqueryExpression + + override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.nonEmpty) { + Seq(NESTED_CORRELATED_SUBQUERY) + } else { + Seq() + } + } } object SubqueryExpression { @@ -198,6 +206,165 @@ object SubExprUtils extends PredicateHelper { plan.exists(_.expressions.exists(containsOuter)) } + /** + * Given a logical plan, returns TRUE if it has a SubqueryExpression + * with non empty outer references + */ + def containsCorrelatedSubquery(e: Expression): Boolean = { + e.exists{ + case in: InSubquery => in.query.getOuterAttrs.nonEmpty && in.query.getJoinCond.isEmpty + case s: SubqueryExpression => s.getOuterAttrs.nonEmpty && s.getJoinCond.isEmpty + case _ => false + } + } + + /** + * Given a logical plan, returns TRUE if it has an outer reference or + * correlated subqueries. + */ + def hasOuterReferencesConsideringNestedCorrelation(plan: LogicalPlan): Boolean = { + plan.exists(_.expressions.exists { + expr => containsOuter(expr) || containsCorrelatedSubquery(expr) + }) + } + + /** + * Returns TRUE if the scalar subquery has multiple Aggregates and the lower Aggregate + * is vulnerable to count bug. + * If it returns TRUE, we need to handle the count bug in [[DecorrelateInnerQuery]]. + * If it returns FALSE, the scalar subquery either does not have a count bug or it + * has a count bug but we handle it in [[RewriteCorrelatedScalarSubquery#constructLeftJoins]]. + */ + def scalarSubqueryHasCountBug(sub: LogicalPlan): Boolean = { + def mayHaveCountBugAgg(a: Aggregate): Boolean = { + a.groupingExpressions.isEmpty && a.aggregateExpressions.exists(_.exists { + case a: AggregateExpression => a.aggregateFunction.defaultResult.isDefined + case _ => false + }) + } + + // The below logic controls handling count bug for scalar subqueries in + // [[DecorrelateInnerQuery]], and if we don't handle it here, we handle it in + // [[RewriteCorrelatedScalarSubquery#constructLeftJoins]]. Note that handling it in + // [[DecorrelateInnerQuery]] is always correct, and turning it off to handle it in + // constructLeftJoins is an optimization, so that additional, redundant left outer joins are + // not introduced. + val conf = SQLConf.get + conf.decorrelateInnerQueryEnabled && + !conf.getConf(SQLConf.LEGACY_SCALAR_SUBQUERY_COUNT_BUG_HANDLING) && + !(sub match { + // Handle count bug only if there exists lower level Aggs with count bugs. It does not + // matter if the top level agg is count bug vulnerable or not, because: + // 1. If the top level agg is count bug vulnerable, it can be handled in + // constructLeftJoins, unless there are lower aggs that are count bug vulnerable. + // E.g. COUNT(COUNT + COUNT) + // 2. If the top level agg is not count bug vulnerable, it can be count bug vulnerable if + // there are lower aggs that are count bug vulnerable. E.g. SUM(COUNT) + case agg: Aggregate => !agg.child.exists { + case lowerAgg: Aggregate => mayHaveCountBugAgg(lowerAgg) + case _ => false + } + case _ => false + }) + } + + /** Returns true if 'query' is guaranteed to return at most 1 row. */ + private def guaranteedToReturnOneRow(query: LogicalPlan): Boolean = { + if (query.maxRows.exists(_ <= 1)) { + return true + } + val aggNode = query match { + case havingPart@Filter(_, aggPart: Aggregate) => Some(aggPart) + case aggPart: Aggregate => Some(aggPart) + // LIMIT 1 is handled above, this is for all other types of LIMITs + case Limit(_, aggPart: Aggregate) => Some(aggPart) + case Project(_, aggPart: Aggregate) => Some(aggPart) + case _: LogicalPlan => None + } + if (!aggNode.isDefined) { + return false + } + val aggregates = aggNode.get.expressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + if (aggregates.isEmpty) { + return false + } + nonEquivalentGroupbyCols(query, aggNode.get).isEmpty + } + + /** Returns TRUE if the scalarSubquery needs a single join. */ + def scalarSubqueryNeedsSingleJoinAfterDecorrelate( + sub: LogicalPlan, needSingleJoinOld: Option[Boolean]): Boolean = { + if (needSingleJoinOld.isDefined) { + needSingleJoinOld.get + } else { + SQLConf.get.getConf(SQLConf.SCALAR_SUBQUERY_USE_SINGLE_JOIN) && !guaranteedToReturnOneRow(sub) + } + } + + /** + * Split the plan for a scalar subquery into the parts above the innermost query block + * (first part of returned value), the HAVING clause of the innermost query block + * (optional second part) and the Aggregate below the HAVING CLAUSE (optional third part). + * When the third part is empty, it means the subquery is a non-aggregated single-row subquery. + */ + def splitSubquery( + plan: LogicalPlan): (Seq[LogicalPlan], Option[Filter], Option[Aggregate]) = { + val topPart = ArrayBuffer.empty[LogicalPlan] + var bottomPart: LogicalPlan = plan + while (true) { + bottomPart match { + case havingPart @ Filter(_, aggPart: Aggregate) => + return (topPart.toSeq, Option(havingPart), Some(aggPart)) + + case aggPart: Aggregate => + // No HAVING clause + return (topPart.toSeq, None, Some(aggPart)) + + case p @ Project(_, child) => + topPart += p + bottomPart = child + + case s @ SubqueryAlias(_, child) => + topPart += s + bottomPart = child + + case p: LogicalPlan if p.maxRows.exists(_ <= 1) => + // Non-aggregated one row subquery. + return (topPart.toSeq, None, None) + + case Filter(_, op) => + throw QueryExecutionErrors.unexpectedOperatorInCorrelatedSubquery(op, " below filter") + + case op @ _ => throw QueryExecutionErrors.unexpectedOperatorInCorrelatedSubquery(op) + } + } + + throw QueryExecutionErrors.unreachableError() + + } + + def scalarSubqueryMayHaveCountBugAfterDecorrelate( + sub: LogicalPlan, + mayHaveCountBugOld: Option[Boolean], + handleCountBugInDecorrelate: Boolean): Boolean = { + if (mayHaveCountBugOld.isDefined) { + // For idempotency, we must save this variable the first time this rule is run, because + // decorrelation introduces a GROUP BY is if one wasn't already present. + mayHaveCountBugOld.get + } else if (handleCountBugInDecorrelate) { + // Count bug was already handled in the above decorrelate function call. + false + } else { + // Check whether the pre-rewrite subquery had empty groupingExpressions. If yes, it may + // be subject to the COUNT bug. If it has non-empty groupingExpressions, there is + // no COUNT bug. + val (topPart, havingNode, aggNode) = splitSubquery(sub) + (aggNode.isDefined && aggNode.get.groupingExpressions.isEmpty) + } + } + /** * Given an expression, returns the expressions which have outer references. Aggregate * expressions are treated in a special way. If the children of aggregate expression contains an diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala index 47cee2e789c7c..4f8d7aefb2567 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE +import org.apache.spark.sql.catalyst.trees.TreePattern.{NESTED_CORRELATED_SUBQUERY, OUTER_REFERENCE, PLAN_EXPRESSION} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.Utils @@ -214,7 +214,9 @@ object DecorrelateInnerQuery extends PredicateHelper { def deduplicate( innerPlan: LogicalPlan, conditions: Seq[Expression], - outerOutputSet: AttributeSet): (LogicalPlan, Seq[Expression]) = { + outerOutputSet: AttributeSet, + outerReferenceMap: AttributeMap[Attribute] = AttributeMap.empty[Attribute] + ): (LogicalPlan, Seq[Expression], AttributeMap[Attribute]) = { val duplicates = innerPlan.outputSet.intersect(outerOutputSet) if (duplicates.nonEmpty) { val aliasMap = AttributeMap(duplicates.map { dup => @@ -227,9 +229,12 @@ object DecorrelateInnerQuery extends PredicateHelper { val aliasedConditions = conditions.map(_.transform { case ref: Attribute => aliasMap.getOrElse(ref, ref).toAttribute }) - (aliasedProjection, aliasedConditions) + val aliasedOuterReferenceMap = AttributeMap(outerReferenceMap.map { + case (k, v) => k -> aliasMap.getOrElse(v, v).toAttribute + }) + (aliasedProjection, aliasedConditions, aliasedOuterReferenceMap) } else { - (innerPlan, conditions) + (innerPlan, conditions, outerReferenceMap) } } @@ -383,6 +388,123 @@ object DecorrelateInnerQuery extends PredicateHelper { } } + def groupDomainsByItsTargetOuterPlan( + domainAttrs: Seq[Attribute], + domainAttrMap: AttributeMap[Expression], + possibleOuterPlans: Seq[LogicalPlan] + ): Seq[(Seq[Attribute], AttributeMap[Expression], LogicalPlan)] = { + val reverseIndex = possibleOuterPlans.indices.reverse + val outerPlanIdToAttrMap = domainAttrs.foldLeft(Map.empty[Int, Seq[Attribute]]) { (acc, attr) => + val matchingPlanIndex = reverseIndex + .find { i => + val outerPlanOutputSet = possibleOuterPlans(i).outputSet + val outerReferences = collectOuterReferences(domainAttrMap(attr)) + outerReferences.subsetOf(outerPlanOutputSet) + } + .getOrElse(throw SparkException.internalError("Cannot find outer references")) + + acc.updated(matchingPlanIndex, acc.getOrElse(matchingPlanIndex, Seq.empty) :+ attr) + } + + outerPlanIdToAttrMap.map { case (i, attrs) => + val newDomainAttrMap = AttributeMap(attrs.map(attr => attr -> domainAttrMap(attr))) + (attrs, newDomainAttrMap, possibleOuterPlans(i)) + }.toSeq + } + + def rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans: Seq[LogicalPlan], + innerPlan: LogicalPlan + ): LogicalPlan = innerPlan match { + case d @ DomainJoin(domainAttrs, child, joinType, outerJoinCondition) => + assert(outerJoinCondition.isDefined, + "DomainJoin should always have the join condition defined") + val newChild = joinType match { + // Left outer domain joins are used to handle the COUNT bug. + case LeftOuter => + // Replace the attributes in the domain join condition with the actual outer expressions + // and use the new join conditions to rewrite domain joins in its child. For example: + // DomainJoin [c'] LeftOuter (a = c') with domainAttrMap: { c' -> _1 }. + // Then the new conditions to use will be [(a = _1)]. + assert(outerJoinCondition.isDefined, + s"LeftOuter domain join should always have the join condition defined:\n$d") + // Recursively rewrite domain joins using the new conditions. + rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans, child) + case Inner => + // The decorrelation framework adds domain inner joins by traversing down the plan tree + // recursively until it reaches a node that is not correlated with the outer query. + // So the child node of a domain inner join shouldn't contain another domain join. + assert(!child.exists(_.isInstanceOf[DomainJoin]), + s"Child of a domain inner join shouldn't contain another domain join.\n$child") + child + case o => + throw SparkException.internalError(s"Unexpected domain join type $o") + } + + // We only needs the domain join conditions that contain outer references, + // which stores the mapping between the domain attributes and the outer plan attributes. + val conditions = splitConjunctivePredicates(outerJoinCondition.get) + val (conditionsContainingOuter, conditionsNotContainingOuter) = + conditions.partition(_.containsPattern(OUTER_REFERENCE)) + val domainAttrMap = buildDomainAttrMap(conditionsContainingOuter, domainAttrs) + assert((joinType == Inner && conditionsNotContainingOuter.isEmpty) + || (joinType == LeftOuter && conditionsNotContainingOuter.nonEmpty), + "LeftOuter domain join should have conditions not containing outer references," + + "and Inner domain join should have all conditions containing outer references.") + val domainJoinCond = conditionsNotContainingOuter.reduceOption(And) + + // We should only rewrite a domain join when all corresponding outer plan attributes + // can be found from the join condition. + if (domainAttrMap.size == domainAttrs.size) { + val domainInfoSeq = + groupDomainsByItsTargetOuterPlan(domainAttrs, + AttributeMap(domainAttrMap), possibleOuterPlans) + val plan = domainInfoSeq.foldLeft(newChild) { + case (newChild, (domainAttrs, domainAttrMap, outerPlan)) => + val groupingExprs = stripOuterReferences(domainAttrs.map(domainAttrMap)) + val aggregateExprs = groupingExprs.zip(domainAttrs).map { + // Rebuild the aliases. + case (inputAttr, outputAttr) => Alias(inputAttr, outputAttr.name)(outputAttr.exprId) + } + // Construct a domain with the outer query plan. + // DomainJoin [a', b'] => Aggregate [a, b] [a AS a', b AS b'] + // +- Relation [a, b] + val domain = Aggregate(groupingExprs, aggregateExprs, outerPlan) + newChild match { + // A special optimization for OneRowRelation. + // TODO: add a more general rule to optimize join with OneRowRelation. + case _: OneRowRelation => domain + // Construct a domain join. + // Join joinType condition + // :- Domain + // +- Inner Query + case _ => Join(domain, newChild, joinType, outerJoinCondition, JoinHint.NONE) + } + } + assert(newChild.outputSet.subsetOf(plan.outputSet)) + // rearrange the output attrs to make sure original outputs are in the front + val projectList = newChild.output ++ plan.output.filterNot(newChild.outputSet.contains) + Project(projectList, plan) + } else { + throw SparkException.internalError( + s"Unable to rewrite domain join with conditions: $conditions\n$d.") + } + case s @ (_ : Union | _: SetOperation) => + // Remap the domain attributes for the children of the set op - see comments on the function. + s.mapChildren { child => + rewriteDomainJoinsConsideringNestedCorrelation(possibleOuterPlans, child) + } + case j: Join if j.joinType == LeftSemi || j.joinType == LeftAnti => + // For the INTERSECT/EXCEPT DISTINCT case, the set op is rewritten to a semi/anti join and we + // need to remap the domain attributes for the right child - see comments on the function. + j.mapChildren { child => + rewriteDomainJoinsConsideringNestedCorrelation(possibleOuterPlans, child) + } + case p: LogicalPlan => + p.mapChildren(rewriteDomainJoinsConsideringNestedCorrelation(possibleOuterPlans, _)) + } + /** * Rewrite all [[DomainJoin]]s in the inner query to actual joins with the outer query. */ @@ -461,11 +583,55 @@ object DecorrelateInnerQuery extends PredicateHelper { p.mapChildren(rewriteDomainJoins(outerPlan, _, conditions)) } + def pushDownSubqueriesToProject(agg: Aggregate): LogicalPlan = { + val subqueriesWithUnresolvedOuter = agg.expressions.flatMap { + expr => expr.collect { + case a@Alias(s: SubqueryExpression, _) if s.getOuterScopeAttrs.nonEmpty => + a -> a + case s: SubqueryExpression if s.getOuterScopeAttrs.nonEmpty => + s -> Alias(s, s"subquery${s.exprId}")() // TODO(avery): is there any other formal name? + } + }.toMap + + if (subqueriesWithUnresolvedOuter.isEmpty) { + // If there are no subqueries with unresolved outer attributes in + // the aggregate expressions, no transformation needed. + return agg + } + + val newAgg = + agg.transformExpressionsDownWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case a@Alias(s: SubqueryExpression, _) if subqueriesWithUnresolvedOuter.contains(a) => + a.toAttribute + case s: SubqueryExpression if subqueriesWithUnresolvedOuter.contains(s) => + subqueriesWithUnresolvedOuter(s).toAttribute + } + + val projectList = subqueriesWithUnresolvedOuter.values.toSeq + val newProject = Project(newAgg.output ++ projectList, newAgg.child) + Aggregate(newAgg.groupingExpressions, newAgg.aggregateExpressions, newProject, newAgg.hint) + } + + def transformPlanWithPotentialNewOutput(plan: LogicalPlan): LogicalPlan = { + plan transformUpWithNewOutput { + case agg: Aggregate => + // If there are subqueries with unresolved outer attrs, + // we need to push down the subqueries to make sure later + // decorrelation is correct. + val newPlan = pushDownSubqueriesToProject(agg) + val attrMapping = agg.output.zip(newPlan.output) + newPlan -> attrMapping + } + } + def apply( innerPlan: LogicalPlan, outerPlan: LogicalPlan, handleCountBug: Boolean = false): (LogicalPlan, Seq[Expression]) = { val outputPlanInputAttrs = outerPlan.inputSet + var innerHandleCountBug = handleCountBug + val containsNestedCorrelations = + innerPlan.containsPattern(NESTED_CORRELATED_SUBQUERY) // The return type of the recursion. // The first parameter is a new logical plan with correlation eliminated. @@ -474,6 +640,244 @@ object DecorrelateInnerQuery extends PredicateHelper { // expressions from the inner query that is used to replace outer references. type ReturnType = (LogicalPlan, Seq[Expression], AttributeMap[Attribute]) + /** + * This function is used as a aid to enforce idempotency of pullUpCorrelatedPredicate rule. + * In the first call to rewriteSubqueries, all the outer references from the subplan are + * pulled up and join predicates are recorded as children of the enclosing subquery expression. + * The subsequent call to rewriteSubqueries would simply re-records the `children` which would + * contains the pulled up correlated predicates (from the previous call) in the enclosing + * subquery expression. + */ + def getJoinCondition(newCond: Seq[Expression], oldCond: Seq[Expression]): Seq[Expression] = { + if (newCond.isEmpty) oldCond else newCond + } + + /** + * In short, transformSubqueryExpressions calls decorrelate on any subquery expressions + * in current operator. + * For each subquery expression, it does 4 steps: + * 1. If subquery expression has unresolvedOuterAttrs, + * set underNestedSubqueries for decorrelate to true as + * we'll insert domain joins to eliminate the correlation pessimistically. + * If it hasn't, set underNestedSubqueries to false as we can solve all + * pulled up predicates in current plan. + * 2. restore current innerHandleCountBug flag, recalculate it according to subquery types + * 3. Call decorrelate to pull up predicates or insert domain joins to eliminate correlation. + * 4. Collect join conditions and outer reference maps from decorrelating subqueries, + * these are conditions can be resolved in current plan. + * 5. Collect join conditions need to be pulled up, these include references cannot be resolved + * in current plan and need to be pulled up. + * 6. Reset joinCond, hints and other associative information. + */ + def transformSubqueryExpressions( + plan: LogicalPlan + ): LogicalPlan = { + + // processJoinCondsAndOuterReferenceMap is a helper function to + // 1. Strip OuterReference wrapper from joinConds + // 2. Separate joinConds into two parts: + // a. joinConds containing unresolvedOuterAttrs + // b. joinConds not containing unresolvedOuterAttrs + // + // 3. For the three parts, there are different processing logic: + // a. For a, unresolved outer attrs in the join conds are wrapped + // with OuterReference again and should be added to the pulled up + // joinConds from the outer plan. Other attrs are from the subquery plan + // and should be wrapped with InnerReference in the remained join conds as + // a hint to add these in the output of subqueries for rules rewriting + // subqueries to joins. + // b. For b, they do not have nested correlations and later rules can handle it normally. + // + // eg: + // Query: SELECT * FROM t0 WHERE t0.b = + // (SELECT SUM(t1.b) FROM t1 WHERE t1.a = + // (SELECT t2.a FROM t2 WHERE t2.a = t0.a AND t2.b > t0.b AND t2.a = t1.a)); + // + // Unprocessed joinConds from the inner most subquery is: + // [outer(t0.a) = t2.a, outer(t1.a) = t2.a, t2.b > outer(t0.b)] + // Among the three joinConds, + // the one can be pulled up(type a) is: [outer(t0.a) = t2.a, t2.b > outer(t0.b)] + // the one can be handled normally(type c) is: [outer(t1.a) = t2.a] + // + // After processing, the pulled up joinConds are: [outer(t0.a) = t2.a, t2.b > outer(t0.b)] + // The remained joinConds are: + // [t1.a = t2.a, InnerReference(t2.a), InnerReference(t2.b)] + // After decorrelate it should be: + // Project [t0.a, t0.b] + // +- Filter [t0.b = scalar-subquery#1[t0.a = t2.a AND t0.b = new_attr_b]] + // +- Aggregate [t2.a, new_attr_b][SUM(t1.b), t2.a, new_attr_b] + // +- Filter [t2.b > new_attr_b] + // +- DomainJoin [new_attr_b][outer(t0.b) = new_attr_b] + // +- Filter [t1.a = + // scalar-subquery#2 [t1.a = t2.a, + // InnerReference(t2.a), InnerReference(t2.b)]] + // +- Project [t2.a, t2.b] + // +- Scan [t2] + // +- Scan[t1] + // +- Scan[t0] + // + // The pulled up joinConds are transformed into a filter predicate. + // For joinConds can be pulled up over Agg and Union, after decorrelation, + // we won't have the filter left in the plan. (It is being further pulled up + // as joinConds in the subqueryExpression). For other joinConds, the proper way + // to insert filter is between the outer plan operator and the subquery expressions. + // That's why we do pushDownSubqueriesToProject before decorrelation, that makes sure + // the aggregate operator and subquery expressions are splitted and we can insert a + // filter between them. For union, there can't be correlated subquery expressions in + // union.expressions so we don't do any preprocessing for it. + def processJoinConds( + joinConds: Seq[Expression], + unresolvedOuterAttrs: Seq[Expression] + ): (Seq[Expression], Seq[Expression]) = { + val joinCondsWithoutOuterWrappers = stripOuterReferences(joinConds) + val (containUnresolvedOuterAttrs, notContainUnresolvedOuterAttrs) = + joinCondsWithoutOuterWrappers.partition( + _.exists(expr => unresolvedOuterAttrs.contains(expr))) + + val pulledUpJoinConds = containUnresolvedOuterAttrs.map { + _.transform { + case a: Attribute if unresolvedOuterAttrs.contains(a) => OuterReference(a) + } + } + + val remainedJoinConds = containUnresolvedOuterAttrs.flatMap { + expr => expr.collect { + case a: Attribute if !unresolvedOuterAttrs.contains(a) => InnerReference(a) + } + } ++ notContainUnresolvedOuterAttrs + assert(remainedJoinConds.nonEmpty, "empty join conds break the idempotency of the rule") + (remainedJoinConds, pulledUpJoinConds) + } + + // transformSubquery is a helper function to transform subquery expressions. + // It does the following: + // 1. calculate new handleCountBug + // 2. call decorrelate + // 3. restore previous handleCountBug + // 4. process joinConds + def transformSubquery( + sub: LogicalPlan, + outputPlanInputAttrs: AttributeSet, + unresolvedOuterAttrs: Seq[Expression], + handleCountBugInDecorrelate: Option[Boolean] = None + ): (LogicalPlan, Seq[Expression], Seq[Expression]) = { + // 1. calculate handleCountBug + val previousHandleCountBug = innerHandleCountBug + innerHandleCountBug = if (handleCountBugInDecorrelate.isDefined) { + handleCountBugInDecorrelate.get + } else { + true + } + // 2. call decorrelate + val (newSub, joinCondsFromSubqueries, outerReferenceMapFromSubqueries) = + decorrelate(BooleanSimplification(sub), AttributeSet.empty, + aggregated = false, underSetOp = false) + // 3. call deduplicate + val (newPlan, newJoinConds, newOuterReferenceMap) = + deduplicate(newSub, joinCondsFromSubqueries, + outputPlanInputAttrs, outerReferenceMapFromSubqueries) + // 4. restore previousHandleCountBug + innerHandleCountBug = previousHandleCountBug + // 5. process joinConds + val (remainedJoinConds, pulledUpJoinConds) = + processJoinConds(newJoinConds, unresolvedOuterAttrs) + (newPlan, remainedJoinConds, pulledUpJoinConds) + } + + val outputPlanInputAttrs = plan.inputSet + var totalPulledUpJoinConds = Seq.empty[Expression] + val newPlan = plan.transformExpressionsDownWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case in@InSubquery(_, + ListQuery(sub, outerAttrs, unresolvedOuterAttrs, exprId, numCols, joinCond, hint) + ) if outerAttrs.nonEmpty && joinCond.isEmpty => + val (newPlan, remainedJoinConds, pulledUpJoinConds) = + transformSubquery(sub, outputPlanInputAttrs, unresolvedOuterAttrs) + totalPulledUpJoinConds ++= pulledUpJoinConds + val newListQuery = ListQuery(newPlan, outerAttrs, unresolvedOuterAttrs, exprId, numCols, + getJoinCondition( + remainedJoinConds, joinCond), hint) + in.copy(query = newListQuery) + case ScalarSubquery( + sub, outerAttrs, unresolvedOuterAttrs, exprId, + joinCond, hint, mayHaveCountBugOld, needSingleJoinOld) + if outerAttrs.nonEmpty && joinCond.isEmpty => + val handleCountBugInDecorrelate = scalarSubqueryHasCountBug(sub) + val (newPlan, remainedJoinConds, pulledUpJoinConds) + = transformSubquery( + sub, outputPlanInputAttrs, unresolvedOuterAttrs, Some(handleCountBugInDecorrelate)) + totalPulledUpJoinConds ++= pulledUpJoinConds + val mayHaveCountBug = scalarSubqueryMayHaveCountBugAfterDecorrelate( + sub, mayHaveCountBugOld, handleCountBugInDecorrelate) + val needSingleJoin = scalarSubqueryNeedsSingleJoinAfterDecorrelate( + sub, needSingleJoinOld) + ScalarSubquery(newPlan, outerAttrs, unresolvedOuterAttrs, exprId, + getJoinCondition(remainedJoinConds, joinCond), + hint, + Some(mayHaveCountBug), + Some(needSingleJoin)) + case Exists(sub, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) + if outerAttrs.nonEmpty && joinCond.isEmpty => + val (newPlan, remainedJoinConds, pulledUpJoinConds) = + transformSubquery(sub, outputPlanInputAttrs, unresolvedOuterAttrs) + totalPulledUpJoinConds ++= pulledUpJoinConds + Exists(newPlan, outerAttrs, unresolvedOuterAttrs, exprId, + getJoinCondition( + remainedJoinConds, joinCond), hint) + case LateralSubquery(sub, outerAttrs, unresolvedOuterAttrs, exprId, joinCond, hint) + if outerAttrs.nonEmpty && joinCond.isEmpty => + val (newPlan, remainedJoinConds, pulledUpJoinConds) = + transformSubquery(sub, outputPlanInputAttrs, unresolvedOuterAttrs) + totalPulledUpJoinConds ++= pulledUpJoinConds + LateralSubquery(newPlan, outerAttrs, unresolvedOuterAttrs, exprId, + getJoinCondition( + remainedJoinConds, joinCond), hint) + } + if (totalPulledUpJoinConds.nonEmpty) { + Filter(totalPulledUpJoinConds.reduce(And), newPlan) + } else { + newPlan + } + } + + def insertDomainJoin( + plan: LogicalPlan, + attributes: Seq[Attribute] + ): (LogicalPlan, Seq[Expression], AttributeMap[Attribute]) = { + val domains = attributes.map(_.newInstance()) + // A placeholder to be rewritten into domain join. + val outerReferenceMap = Utils.toMap(attributes, domains) + // Build join conditions between domain attributes and outer references. + // EqualNullSafe is used to make sure null key can be joined together. Note + // outer referenced attributes can be changed during the outer query optimization. + // The equality conditions will also serve as an attribute mapping between new + // outer references and domain attributes when rewriting the domain joins. + // E.g. if the attribute a is changed to a1, the join condition a' <=> outer(a) + // will become a' <=> a1, and we can construct the aliases based on the condition: + // DomainJoin [a'] Join Inner + // +- InnerQuery => :- InnerQuery + // +- Aggregate [a1] [a1 AS a'] + // +- OuterQuery + val conditions = outerReferenceMap.map { + case (o, a) => + val cond = EqualNullSafe(a, OuterReference(o)) + // SPARK-40615: Certain data types (e.g. MapType) do not support ordering, so + // the EqualNullSafe join condition can become unresolved. + if (!cond.resolved) { + if (!RowOrdering.isOrderable(a.dataType)) { + throw QueryCompilationErrors.unsupportedCorrelatedReferenceDataTypeError( + o, a.dataType, plan.origin) + } else { + throw SparkException.internalError(s"Unable to decorrelate subquery: " + + s"join condition '${cond.sql}' cannot be resolved.") + } + } + cond + } + val joinConditions: Seq[Expression] = conditions.toSeq + val domainJoin = DomainJoin(domains, plan, Inner, Some(joinConditions.reduce(And))) + (domainJoin, joinConditions, AttributeMap(outerReferenceMap)) + } + // Decorrelate the input plan. // parentOuterReferences: a set of parent outer references. As we recurse down we collect the // set of outer references that are part of the Domain, and use it to construct the DomainJoins @@ -498,7 +902,16 @@ object DecorrelateInnerQuery extends PredicateHelper { aggregated: Boolean = false, underSetOp: Boolean = false ): ReturnType = { - val isCorrelated = hasOuterReferences(plan) + // We can't directly use pattern NESTED_CORRELATED_SUBQUERY to check if the plan + // is correlated as this rule might be run multiple times and if the outer references + // or the correlated subqueries are already rewritten in previous puns, the plan should + // be treated as non-correlated. + val isCorrelated = + if (containsNestedCorrelations) { + hasOuterReferencesConsideringNestedCorrelation(plan) + } else { + hasOuterReferences(plan) + } if (!isCorrelated) { // We have reached a plan without correlation to the outer plan. if (parentOuterReferences.isEmpty) { @@ -509,41 +922,57 @@ object DecorrelateInnerQuery extends PredicateHelper { } else { // Build the domain join with the parent outer references. val attributes = parentOuterReferences.toSeq - val domains = attributes.map(_.newInstance()) - // A placeholder to be rewritten into domain join. - val domainJoin = DomainJoin(domains, plan) - val outerReferenceMap = Utils.toMap(attributes, domains) - // Build join conditions between domain attributes and outer references. - // EqualNullSafe is used to make sure null key can be joined together. Note - // outer referenced attributes can be changed during the outer query optimization. - // The equality conditions will also serve as an attribute mapping between new - // outer references and domain attributes when rewriting the domain joins. - // E.g. if the attribute a is changed to a1, the join condition a' <=> outer(a) - // will become a' <=> a1, and we can construct the aliases based on the condition: - // DomainJoin [a'] Join Inner - // +- InnerQuery => :- InnerQuery - // +- Aggregate [a1] [a1 AS a'] - // +- OuterQuery - val conditions = outerReferenceMap.map { - case (o, a) => - val cond = EqualNullSafe(a, OuterReference(o)) - // SPARK-40615: Certain data types (e.g. MapType) do not support ordering, so - // the EqualNullSafe join condition can become unresolved. - if (!cond.resolved) { - if (!RowOrdering.isOrderable(a.dataType)) { - throw QueryCompilationErrors.unsupportedCorrelatedReferenceDataTypeError( - o, a.dataType, plan.origin) - } else { - throw SparkException.internalError(s"Unable to decorrelate subquery: " + - s"join condition '${cond.sql}' cannot be resolved.") + if (containsNestedCorrelations) { + val (domainJoin, joinConds, outerReferenceMap) = insertDomainJoin(plan, attributes) + (domainJoin, joinConds, outerReferenceMap) + } else { + val domains = attributes.map(_.newInstance()) + // A placeholder to be rewritten into domain join. + val domainJoin = DomainJoin(domains, plan) + val outerReferenceMap = Utils.toMap(attributes, domains) + // Build join conditions between domain attributes and outer references. + // EqualNullSafe is used to make sure null key can be joined together. Note + // outer referenced attributes can be changed during the outer query optimization. + // The equality conditions will also serve as an attribute mapping between new + // outer references and domain attributes when rewriting the domain joins. + // E.g. if the attribute a is changed to a1, the join condition a' <=> outer(a) + // will become a' <=> a1, and we can construct the aliases based on the condition: + // DomainJoin [a'] Join Inner + // +- InnerQuery => :- InnerQuery + // +- Aggregate [a1] [a1 AS a'] + // +- OuterQuery + val conditions = outerReferenceMap.map { + case (o, a) => + val cond = EqualNullSafe(a, OuterReference(o)) + // SPARK-40615: Certain data types (e.g. MapType) do not support ordering, so + // the EqualNullSafe join condition can become unresolved. + if (!cond.resolved) { + if (!RowOrdering.isOrderable(a.dataType)) { + throw QueryCompilationErrors.unsupportedCorrelatedReferenceDataTypeError( + o, + a.dataType, + plan.origin + ) + } else { + throw SparkException.internalError( + s"Unable to decorrelate subquery: " + + s"join condition '${cond.sql}' cannot be resolved." + ) + } } - } - cond + cond + } + (domainJoin, conditions.toSeq, AttributeMap(outerReferenceMap)) } - (domainJoin, conditions.toSeq, AttributeMap(outerReferenceMap)) } } else { - plan match { + val planWithSubqueriesProcessed = + if (containsNestedCorrelations) { + transformSubqueryExpressions(plan) + } else { + plan + } + planWithSubqueriesProcessed match { case Filter(condition, child) => val conditions = splitConjunctivePredicates(condition) val (correlated, uncorrelated) = conditions.partition(containsOuter) @@ -838,7 +1267,7 @@ object DecorrelateInnerQuery extends PredicateHelper { // | 0 | 2 | true | 2 | // | 0 | null | null | 0 | <--- correct result // +---+------+------------+--------------------------------+ - if (groupingExpressions.isEmpty && handleCountBug) { + if (groupingExpressions.isEmpty && innerHandleCountBug) { // Evaluate the aggregate expressions with zero tuples. val resultMap = RewriteCorrelatedScalarSubquery.evalAggregateOnZeroTups(newAggregate) val alwaysTrue = Alias(Literal.TrueLiteral, "alwaysTrue")() @@ -904,8 +1333,31 @@ object DecorrelateInnerQuery extends PredicateHelper { // Use the current join conditions returned from the recursive call as the join // conditions for the left outer join. All outer references in the join // conditions are replaced by the newly created domain attributes. - val condition = replaceOuterReferences(joinCond, mapping).reduceOption(And) - val domainJoin = DomainJoin(domainAttrs, agg, LeftOuter, condition) + val domainJoin = + if (containsNestedCorrelations) { + val conds: Seq[Expression] = mapping.map { + case (o, a) => + val cond = EqualNullSafe(a, OuterReference(o)) + // SPARK-40615: Certain data types (e.g. MapType) do not support ordering, so + // the EqualNullSafe join condition can become unresolved. + if (!cond.resolved) { + if (!RowOrdering.isOrderable(a.dataType)) { + throw QueryCompilationErrors.unsupportedCorrelatedReferenceDataTypeError( + o, a.dataType, plan.origin) + } else { + throw SparkException.internalError(s"Unable to decorrelate subquery: " + + s"join condition '${cond.sql}' cannot be resolved.") + } + } + cond + }.toSeq + val condition = replaceOuterReferences(joinCond, mapping).reduce(And) + val addedConditions = (Seq(condition) ++ conds).reduceOption(And) + DomainJoin(domainAttrs, agg, LeftOuter, addedConditions) + } else { + val condition = replaceOuterReferences(joinCond, mapping).reduceOption(And) + DomainJoin(domainAttrs, agg, LeftOuter, condition) + } // Original domain attributes preserved through Aggregate are no longer needed. val newProjectList = projectList.filter(!referencesToAdd.contains(_)) val project = Project(newProjectList ++ domainAttrs, domainJoin) @@ -991,10 +1443,14 @@ object DecorrelateInnerQuery extends PredicateHelper { parentOuterReferences ++ outerReferences -- equivalences.keySet var shouldPushToLeft = joinType match { case LeftOuter | LeftSemiOrAnti(_) | FullOuter => true + case _ if containsNestedCorrelations => + hasOuterReferencesConsideringNestedCorrelation(left) case _ => hasOuterReferences(left) } val shouldPushToRight = joinType match { case RightOuter | FullOuter => true + case _ if containsNestedCorrelations => + hasOuterReferencesConsideringNestedCorrelation(right) case _ => hasOuterReferences(right) } if (shouldDecorrelatePredicates && !shouldPushToLeft && !shouldPushToRight @@ -1107,8 +1563,14 @@ object DecorrelateInnerQuery extends PredicateHelper { } } } - val (newChild, joinCond, _) = decorrelate(BooleanSimplification(innerPlan), AttributeSet.empty) - val (plan, conditions) = deduplicate(newChild, joinCond, outputPlanInputAttrs) + val updatedInnerPlan = if (containsNestedCorrelations) { + transformPlanWithPotentialNewOutput(innerPlan) + } else { + innerPlan + } + val (newChild, joinCond, _) = + decorrelate(BooleanSimplification(updatedInnerPlan), AttributeSet.empty) + val (plan, conditions, _) = deduplicate(newChild, joinCond, outputPlanInputAttrs) (plan, stripOuterReferences(conditions)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 00acbc572b01e..1c2f87837e9c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -205,6 +205,9 @@ abstract class Optimizer(catalogManager: CatalogManager) OptimizeOneRowRelationSubquery, PullOutNestedDataOuterRefExpressions, PullupCorrelatedPredicates), + Batch("Rewrite Nested Correlated Subqueries", Once, + RewriteDomainJoinsInOnePass, + ), // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). Batch("Subquery", FixedPoint(1), @@ -295,6 +298,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RewriteIntersectAll.ruleName, ReplaceDistinctWithAggregate.ruleName, PullupCorrelatedPredicates.ruleName, + RewriteDomainJoinsInOnePass.ruleName, RewriteCorrelatedScalarSubquery.ruleName, RewritePredicateSubquery.ruleName, NormalizeFloatingNumbers.ruleName, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 0c94d3c714730..d755ed9455365 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -26,16 +26,14 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.ScalarSubquery._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery.splitSubquery import org.apache.spark.sql.catalyst.planning.PhysicalAggregation import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.trees.TreePattern.{EXISTS_SUBQUERY, IN_SUBQUERY, LATERAL_JOIN, LIST_SUBQUERY, PLAN_EXPRESSION, SCALAR_SUBQUERY} +import org.apache.spark.sql.catalyst.trees.TreePattern.{EXISTS_SUBQUERY, IN_SUBQUERY, LATERAL_JOIN, LIST_SUBQUERY, NESTED_CORRELATED_SUBQUERY, PLAN_EXPRESSION, SCALAR_SUBQUERY} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.{DECORRELATE_PREDICATE_SUBQUERIES_IN_JOIN_CONDITION, OPTIMIZE_UNCORRELATED_IN_SUBQUERIES_IN_JOIN_CONDITION, - WRAP_EXISTS_IN_AGGREGATE_FUNCTION} +import org.apache.spark.sql.internal.SQLConf.{DECORRELATE_PREDICATE_SUBQUERIES_IN_JOIN_CONDITION, OPTIMIZE_UNCORRELATED_IN_SUBQUERIES_IN_JOIN_CONDITION, WRAP_EXISTS_IN_AGGREGATE_FUNCTION} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -528,7 +526,7 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper val baseConditions = predicateMap.values.flatten.toSeq val outerPlanInputAttrs = outer.inputSet val (newPlan, newCond) = if (outerPlanInputAttrs.nonEmpty) { - val (plan, deDuplicatedConditions) = + val (plan, deDuplicatedConditions, _) = DecorrelateInnerQuery.deduplicate(transformed, baseConditions, outerPlanInputAttrs) (plan, stripOuterReferences(deDuplicatedConditions)) } else { @@ -537,31 +535,6 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper (newPlan, newCond) } - // Returns true if 'query' is guaranteed to return at most 1 row. - private def guaranteedToReturnOneRow(query: LogicalPlan): Boolean = { - if (query.maxRows.exists(_ <= 1)) { - return true - } - val aggNode = query match { - case havingPart@Filter(_, aggPart: Aggregate) => Some(aggPart) - case aggPart: Aggregate => Some(aggPart) - // LIMIT 1 is handled above, this is for all other types of LIMITs - case Limit(_, aggPart: Aggregate) => Some(aggPart) - case Project(_, aggPart: Aggregate) => Some(aggPart) - case _: LogicalPlan => None - } - if (!aggNode.isDefined) { - return false - } - val aggregates = aggNode.get.expressions.flatMap(_.collect { - case a: AggregateExpression => a - }) - if (aggregates.isEmpty) { - return false - } - nonEquivalentGroupbyCols(query, aggNode.get).isEmpty - } - private def rewriteSubQueries(plan: LogicalPlan): LogicalPlan = { /** * This function is used as a aid to enforce idempotency of pullUpCorrelatedPredicate rule. @@ -590,55 +563,14 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper case ScalarSubquery(sub, children, outerScopeAttrs, exprId, conditions, hint, mayHaveCountBugOld, needSingleJoinOld) if children.nonEmpty => + val handleCountBugInDecorrelate = scalarSubqueryHasCountBug(sub) + val (newPlan, newCond) = decorrelate(sub, plan, handleCountBugInDecorrelate) - def mayHaveCountBugAgg(a: Aggregate): Boolean = { - a.groupingExpressions.isEmpty && a.aggregateExpressions.exists(_.exists { - case a: AggregateExpression => a.aggregateFunction.defaultResult.isDefined - case _ => false - }) - } + val mayHaveCountBug = scalarSubqueryMayHaveCountBugAfterDecorrelate( + sub, mayHaveCountBugOld, handleCountBugInDecorrelate) + val needSingleJoin = scalarSubqueryNeedsSingleJoinAfterDecorrelate( + sub, needSingleJoinOld) - // The below logic controls handling count bug for scalar subqueries in - // [[DecorrelateInnerQuery]], and if we don't handle it here, we handle it in - // [[RewriteCorrelatedScalarSubquery#constructLeftJoins]]. Note that handling it in - // [[DecorrelateInnerQuery]] is always correct, and turning it off to handle it in - // constructLeftJoins is an optimization, so that additional, redundant left outer joins are - // not introduced. - val handleCountBugInDecorrelate = SQLConf.get.decorrelateInnerQueryEnabled && - !conf.getConf(SQLConf.LEGACY_SCALAR_SUBQUERY_COUNT_BUG_HANDLING) && !(sub match { - // Handle count bug only if there exists lower level Aggs with count bugs. It does not - // matter if the top level agg is count bug vulnerable or not, because: - // 1. If the top level agg is count bug vulnerable, it can be handled in - // constructLeftJoins, unless there are lower aggs that are count bug vulnerable. - // E.g. COUNT(COUNT + COUNT) - // 2. If the top level agg is not count bug vulnerable, it can be count bug vulnerable if - // there are lower aggs that are count bug vulnerable. E.g. SUM(COUNT) - case agg: Aggregate => !agg.child.exists { - case lowerAgg: Aggregate => mayHaveCountBugAgg(lowerAgg) - case _ => false - } - case _ => false - }) - val (newPlan, newCond) = decorrelate(sub, plan, handleCountBugInDecorrelate) - val mayHaveCountBug = if (mayHaveCountBugOld.isDefined) { - // For idempotency, we must save this variable the first time this rule is run, because - // decorrelation introduces a GROUP BY is if one wasn't already present. - mayHaveCountBugOld.get - } else if (handleCountBugInDecorrelate) { - // Count bug was already handled in the above decorrelate function call. - false - } else { - // Check whether the pre-rewrite subquery had empty groupingExpressions. If yes, it may - // be subject to the COUNT bug. If it has non-empty groupingExpressions, there is - // no COUNT bug. - val (topPart, havingNode, aggNode) = splitSubquery(sub) - (aggNode.isDefined && aggNode.get.groupingExpressions.isEmpty) - } - val needSingleJoin = if (needSingleJoinOld.isDefined) { - needSingleJoinOld.get - } else { - conf.getConf(SQLConf.SCALAR_SUBQUERY_USE_SINGLE_JOIN) && !guaranteedToReturnOneRow(sub) - } ScalarSubquery(newPlan, children, outerScopeAttrs, exprId, getJoinCondition(newCond, conditions), hint, Some(mayHaveCountBug), Some(needSingleJoin)) case Exists(sub, children, outerScopeAttrs, exprId, conditions, hint) if children.nonEmpty => @@ -842,48 +774,6 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe } } - /** - * Split the plan for a scalar subquery into the parts above the innermost query block - * (first part of returned value), the HAVING clause of the innermost query block - * (optional second part) and the Aggregate below the HAVING CLAUSE (optional third part). - * When the third part is empty, it means the subquery is a non-aggregated single-row subquery. - */ - def splitSubquery( - plan: LogicalPlan): (Seq[LogicalPlan], Option[Filter], Option[Aggregate]) = { - val topPart = ArrayBuffer.empty[LogicalPlan] - var bottomPart: LogicalPlan = plan - while (true) { - bottomPart match { - case havingPart @ Filter(_, aggPart: Aggregate) => - return (topPart.toSeq, Option(havingPart), Some(aggPart)) - - case aggPart: Aggregate => - // No HAVING clause - return (topPart.toSeq, None, Some(aggPart)) - - case p @ Project(_, child) => - topPart += p - bottomPart = child - - case s @ SubqueryAlias(_, child) => - topPart += s - bottomPart = child - - case p: LogicalPlan if p.maxRows.exists(_ <= 1) => - // Non-aggregated one row subquery. - return (topPart.toSeq, None, None) - - case Filter(_, op) => - throw QueryExecutionErrors.unexpectedOperatorInCorrelatedSubquery(op, " below filter") - - case op @ _ => throw QueryExecutionErrors.unexpectedOperatorInCorrelatedSubquery(op) - } - } - - throw QueryExecutionErrors.unreachableError() - - } - // Name of generated column used in rewrite below val ALWAYS_TRUE_COLNAME = "alwaysTrue" @@ -898,8 +788,16 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe subqueries: ArrayBuffer[ScalarSubquery]): (LogicalPlan, AttributeMap[Attribute]) = { val subqueryAttrMapping = ArrayBuffer[(Attribute, Attribute)]() val newChild = subqueries.foldLeft(child) { - case (currentChild, ScalarSubquery(sub, _, _, _, conditions, subHint, mayHaveCountBug, + case (currentChild, ScalarSubquery(sub, _, outerScopeAttrs, _, rawConditions, subHint, mayHaveCountBug, needSingleJoin)) => + val (conditionsContainInnerReferences, conditions) = + rawConditions.partition(_.exists(_.isInstanceOf[InnerReference])) + val neededInnerAttrs = conditionsContainInnerReferences.flatMap(_.collect { + case InnerReference(a) => a + }) + assert(neededInnerAttrs.isEmpty == outerScopeAttrs.isEmpty, + "Inner references are not allowed for subqueries without nested correlations") + val query = DecorrelateInnerQuery.rewriteDomainJoins(currentChild, sub, conditions) val origOutput = query.output.head // The subquery appears on the right side of the join, hence add its hint to the right @@ -912,7 +810,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe case _ => LeftOuter } lazy val planWithoutCountBug = Project( - currentChild.output :+ origOutput, + (currentChild.output :+ origOutput) ++ neededInnerAttrs, Join(currentChild, query, joinType, conditions.reduceOption(And), joinHint)) if (Utils.isTesting) { @@ -959,7 +857,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe aggValRef), origOutput.name)() subqueryAttrMapping += ((origOutput, subqueryResultExpr.toAttribute)) Project( - currentChild.output :+ subqueryResultExpr, + (currentChild.output :+ subqueryResultExpr) ++ neededInnerAttrs, Join(currentChild, Project(query.output :+ alwaysTrueExpr, query), joinType, conditions.reduceOption(And), joinHint)) @@ -991,7 +889,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe subqueryAttrMapping += ((origOutput, caseExpr.toAttribute)) Project( - currentChild.output :+ caseExpr, + (currentChild.output :+ caseExpr) ++ neededInnerAttrs, Join(currentChild, Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), joinType, conditions.reduceOption(And), joinHint)) @@ -1028,6 +926,19 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe } } + def extractInnerReferenceFromCorrelatedSubqueries( + subqueries: Seq[SubqueryExpression] + ): AttributeSet = { + val innerAttrs = subqueries.flatMap { + sub => sub.getJoinCond.flatMap { + expr => expr.collect { + case InnerReference(a) => a + } + } + } + AttributeSet(innerAttrs) + } + /** * Rewrite [[Filter]], [[Project]] and [[Aggregate]] plans containing correlated scalar * subqueries. @@ -1045,8 +956,10 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe } val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) val newExprs = updateAttrs(rewriteExprs, subqueryAttrMapping) - val newAgg = Aggregate(newGrouping, newExprs, newChild) - val attrMapping = a.output.zip(newAgg.output) + val innerAttrs = extractInnerReferenceFromCorrelatedSubqueries(subqueries.toSeq) + val newAgg = Aggregate( + newGrouping ++ innerAttrs.toSeq, newExprs ++ innerAttrs.toSeq, newChild) + val attrMapping = a.output.zip(newAgg.output.take(a.output.size)) checkScalarSubqueryInAgg(newAgg) newAgg -> attrMapping } else { @@ -1057,9 +970,10 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe val rewriteExprs = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) if (subqueries.nonEmpty) { val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) + val innerAttrs = extractInnerReferenceFromCorrelatedSubqueries(subqueries.toSeq) val newExprs = updateAttrs(rewriteExprs, subqueryAttrMapping) - val newProj = Project(newExprs, newChild) - val attrMapping = p.output.zip(newProj.output) + val newProj = Project(newExprs ++ innerAttrs.toSeq, newChild) + val attrMapping = p.output.zip(newProj.output.take(p.output.size)) newProj -> attrMapping } else { p -> Nil @@ -1069,9 +983,10 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe val rewriteCondition = extractCorrelatedScalarSubqueries(condition, subqueries) if (subqueries.nonEmpty) { val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) + val innerAttrs = extractInnerReferenceFromCorrelatedSubqueries(subqueries.toSeq) val newCondition = updateAttrs(Seq(rewriteCondition), subqueryAttrMapping).head - val newProj = Project(f.output, Filter(newCondition, newChild)) - val attrMapping = f.output.zip(newProj.output) + val newProj = Project(f.output ++ innerAttrs.toSeq, Filter(newCondition, newChild)) + val attrMapping = f.output.zip(newProj.output.take(f.output.size)) newProj -> attrMapping } else { f -> Nil @@ -1160,3 +1075,201 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { } } } + +/** + * This rule rewrites domain joins created by PullUpCorrelatedPredicates, + * It rewrites all the domain joins within the main query and nested subqueries + * in a top down manner. + */ +object RewriteDomainJoinsInOnePass extends Rule[LogicalPlan] { + private def containsCorrelatedSubquery(expr: Expression): Boolean = { + expr exists { + case s: SubqueryExpression => s.children.nonEmpty + case _ => false + } + } + + private def rewriteDomainJoinsUnderJoin( + j: Join, + possibleOuterPlans: Seq[LogicalPlan] + ): (LogicalPlan, Seq[(Attribute, Attribute)]) = { + val relevantSubqueries = j.condition.get.collect { + case i: InSubquery if i.query.isCorrelated => i + case e: Exists if e.isCorrelated => e + } + if (relevantSubqueries.isEmpty) { + j -> j.output.zip(j.output) + } else { + // `subqueriesWithJoinInputReferenceInfo`is of type Seq[(Expression, Boolean, Boolean)] + // (1): Expression, the join predicate containing some predicate subquery we are interested + // in re-writing + // (2): Boolean, whether (1) references the left join input + // (3): Boolean, whether (1) references the right join input + val subqueriesWithJoinInputReferenceInfo = relevantSubqueries.map { e => + val referenceLeft = e.references.intersect(j.left.outputSet).nonEmpty + val referenceRight = e.references.intersect(j.right.outputSet).nonEmpty + (e, referenceLeft, referenceRight) + } + val subqueriesReferencingBothJoinInputs = subqueriesWithJoinInputReferenceInfo + .filter(i => i._2 && i._3) + + // Currently do not support correlated subqueries in the join predicate that reference both + // join inputs + if (subqueriesReferencingBothJoinInputs.nonEmpty) { + throw QueryCompilationErrors.unsupportedCorrelatedSubqueryInJoinConditionError( + subqueriesReferencingBothJoinInputs.map(_._1) + ) + } + val subqueriesReferencingLeft = subqueriesWithJoinInputReferenceInfo.filter(_._2).map(_._1) + val subqueriesReferencingRight = subqueriesWithJoinInputReferenceInfo.filter(_._3).map(_._1) + if (subqueriesReferencingLeft.isEmpty && subqueriesReferencingRight.isEmpty) { + j -> j.output.zip(j.output) + } else { + var newCondition = j.condition.get + subqueriesReferencingLeft.foldLeft(j.left) { + case (p, e) => + val newSubExpr = e match { + case i: InSubquery => + val subExpr = i.query + val newPlan = + DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(p), subExpr.plan) + val newQuery = subExpr.withNewPlan(newPlan) + i.copy(query = newQuery) + case ex: Exists => + val newPlan = + DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(p), ex.plan) + ex.copy(plan = newPlan) + } + // Update the join condition to rewrite the subquery expression + newCondition = newCondition.transform { + case expr if expr.fastEquals(e) => newSubExpr + } + p + } + subqueriesReferencingRight.foldLeft(j.right) { + case (p, e) => + val newSubExpr = e match { + case i: InSubquery => + val subExpr = i.query + val newPlan = + DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(p), subExpr.plan) + val newQuery = subExpr.withNewPlan(newPlan) + i.copy(query = newQuery) + case ex: Exists => + val newPlan = + DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(p), ex.plan) + ex.copy(plan = newPlan) + } + // Update the join condition to rewrite the subquery expression + newCondition = newCondition.transform { + case expr if expr.fastEquals(e) => newSubExpr + } + p + } + val newJ = j.copy(condition = Some(newCondition)) + (newJ, j.output.zip(newJ.output)) + } + } + } + + private def rewriteDomainJoinsUnderUnaryNode( + plan: UnaryNode, + possibleOuterPlans: Seq[LogicalPlan] + ): (LogicalPlan, Seq[(Attribute, Attribute)]) = { + val newPlan = plan.transformExpressionsWithPruning(_.containsPattern(SCALAR_SUBQUERY)) { + case s: ScalarSubquery if s.children.nonEmpty => + // rewriteDomainJoins in pre order traversal to + // ensure outer domain joins are rewritten first + val rewrittenSub = DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(plan.child), s.plan) + s.copy(plan = rewrittenSub) + case e: Exists + if e.children.nonEmpty && SQLConf.get.decorrelateInnerQueryEnabledForExistsIn => + // TODO(avery): This is different from original rewritePredicateSubquery, check + // if there are any plan changes + val rewrittenSub = DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(plan.child), e.plan) + e.copy(plan = rewrittenSub) + case l: ListQuery + if l.children.nonEmpty && SQLConf.get.decorrelateInnerQueryEnabledForExistsIn => + val rewrittenSub = DecorrelateInnerQuery.rewriteDomainJoinsConsideringNestedCorrelation( + possibleOuterPlans ++ Seq(plan.child), l.plan) + l.copy(plan = rewrittenSub) + } + // TODO(avery): I assume there are no changed outputs, check if this is true + assert(plan.sameOutput(newPlan)) + assert(plan.output.size == newPlan.output.size) + val attrMapping = plan.output.zip(newPlan.output) + (newPlan, attrMapping) + } + + private def rewriteDomainJoinsInOneLayer( + plan: LogicalPlan, possibleOuterPlans: Seq[LogicalPlan] + ): LogicalPlan = { + plan transformUpWithNewOutput { + case u: UnaryNode if u.expressions.exists(containsCorrelatedSubquery) => + val (newU, attrMapping) = rewriteDomainJoinsUnderUnaryNode(u, possibleOuterPlans) + newU -> attrMapping + case j: Join if j.condition.exists(cond => + SubqueryExpression.hasInOrCorrelatedExistsSubquery(cond)) && + conf.getConf(DECORRELATE_PREDICATE_SUBQUERIES_IN_JOIN_CONDITION) => + val (newJ, attrMapping) = rewriteDomainJoinsUnderJoin(j, possibleOuterPlans) + newJ -> attrMapping + case l: LateralJoin => + val ls = l.right + val newRightPlan = DecorrelateInnerQuery.rewriteDomainJoins(l.left, ls.plan, ls.getJoinCond) + val newLateralSubquery = ls.copy(plan = newRightPlan) + val newLateralJoin = l.copy(right = newLateralSubquery) + newLateralJoin -> l.output.zip(newLateralJoin.output) + } + } + + private def apply0( + plan: LogicalPlan, + possibleOuterPlans: Seq[LogicalPlan] = Seq.empty[LogicalPlan] + ): LogicalPlan = { + val updatedPlan = rewriteDomainJoinsInOneLayer(plan, possibleOuterPlans) + val res = updatedPlan.transformDownWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case u: UnaryNode if u.expressions.exists(containsCorrelatedSubquery) => + u.transformAllExpressionsWithPruning( + _.containsPattern(PLAN_EXPRESSION)) { + case s: SubqueryExpression if s.children.nonEmpty => + val newPlan = apply0(s.plan, possibleOuterPlans ++ Seq(u.child)) + s.withNewPlan(newPlan) + } + case j: Join if j.condition.exists(cond => + SubqueryExpression.hasInOrCorrelatedExistsSubquery(cond)) && + conf.getConf(DECORRELATE_PREDICATE_SUBQUERIES_IN_JOIN_CONDITION) => + j.transformAllExpressionsWithPruning( + _.containsPattern(PLAN_EXPRESSION)) { + case s: SubqueryExpression if s.children.nonEmpty => + // TODO(avery): (not sure if all the children should be included) + val newPlan = apply0(s.plan, possibleOuterPlans ++ j.children) + s.withNewPlan(newPlan) + } + case l: LateralJoin => + l.transformAllExpressionsWithPruning( + _.containsPattern(PLAN_EXPRESSION)) { + case s: SubqueryExpression if s.children.nonEmpty => + val newPlan = apply0(s.plan, possibleOuterPlans ++ Seq(l.left)) + s.withNewPlan(newPlan) + } + } + res + } + + /** + * Rewrite domain joins in any correlated subquery plan. + */ + def apply(plan: LogicalPlan): LogicalPlan = { + if (plan.containsPattern(NESTED_CORRELATED_SUBQUERY)) { + apply0(plan) + } else { + plan + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 3ea32f3cc464f..ac85b0de2859c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -69,6 +69,7 @@ object TreePattern extends Enumeration { val LITERAL: Value = Value val MAP_OBJECTS: Value = Value val MULTI_ALIAS: Value = Value + val NESTED_CORRELATED_SUBQUERY: Value = Value val NEW_INSTANCE: Value = Value val NOT: Value = Value val NULL_CHECK: Value = Value From 0373be1422dace99acda8de98ad380cbdf307e44 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Wed, 23 Apr 2025 13:53:26 -0700 Subject: [PATCH 44/48] style basic framework done, tests WIP init test test wip --- ...ctionTableSubqueryArgumentExpression.scala | 11 +- .../expressions/namedExpressions.scala | 17 +- .../sql/catalyst/expressions/subquery.scala | 36 +- .../optimizer/DecorrelateInnerQuery.scala | 35 +- .../sql/catalyst/optimizer/Optimizer.scala | 4 + .../sql/catalyst/optimizer/subquery.scala | 54 +- .../sql/catalyst/trees/TreePatterns.scala | 1 + .../nestedcorrelation/scalar-subquery.sql.out | 800 +++++++++++++++++ .../combined-subquery.sql | 0 .../exists-subquery.sql | 0 .../lateral-subquery.sql | 0 .../scalar-subquery.sql | 33 + .../subquery-not-supported.sql | 0 .../nestedcorrelation/scalar-subquery.sql | 259 +++++- .../combined-subquery.sql.out | 66 ++ .../exists-subquery.sql.out | 430 +++++++++ .../lateral-subquery.sql.out | 88 ++ .../scalar-subquery.sql.out | 259 ++++++ .../subquery-not-supported.sql.out | 566 ++++++++++++ .../nestedcorrelation/scalar-subquery.sql.out | 844 ++++++++++++++---- .../apache/spark/sql/SQLQueryTestSuite.scala | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 115 ++- .../ThriftServerQueryTestSuite.scala | 11 +- 23 files changed, 3390 insertions(+), 241 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out rename sql/core/src/test/resources/sql-tests/inputs/subquery/{nestedcorrelation => nestedcorrelation-analyzer-only}/combined-subquery.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/subquery/{nestedcorrelation => nestedcorrelation-analyzer-only}/exists-subquery.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/subquery/{nestedcorrelation => nestedcorrelation-analyzer-only}/lateral-subquery.sql (100%) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql rename sql/core/src/test/resources/sql-tests/inputs/subquery/{nestedcorrelation => nestedcorrelation-analyzer-only}/subquery-not-supported.sql (100%) create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index fe63169767693..695110223bebc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, LogicalPlan, Project, Repartition, RepartitionByExpression, Sort} -import org.apache.spark.sql.catalyst.trees.TreePattern.{FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION, TreePattern} +import org.apache.spark.sql.catalyst.trees.TreePattern.{FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION, NESTED_CORRELATED_SUBQUERY, TreePattern} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.DataType @@ -128,8 +128,13 @@ case class FunctionTableSubqueryArgumentExpression( newChildren: IndexedSeq[Expression]): FunctionTableSubqueryArgumentExpression = copy(outerAttrs = newChildren) - final override def nodePatternsInternal(): Seq[TreePattern] = - Seq(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION) + final override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.isEmpty) { + Seq(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION) + } else { + Seq(NESTED_CORRELATED_SUBQUERY, FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION) + } + } def hasRepartitioning: Boolean = withSinglePartition || partitionByExpressions.nonEmpty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 6a43e26e61ce9..f7981c586264b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -445,25 +445,24 @@ case class OuterReference(e: NamedExpression) } /** - * A place holder used to hold attributes cannot be solved by the subquery plan and - * its immediate parent plan. When rewriting subqueries into joins, joinConds containing - * OuterScopeReferences are transformed into filter predicates between the joins transformed - * from the subqueries and the outer plan. - * It can only be used in the intermediate results in the optimization stage, should not appear - * in the physical plan. + * A place holder used to hold outer references in DomainJoins to instruct RewriteDomainJoins + * the mapping between domain attributes and outer references. + * We use it instead of OuterReference to avoid treating the already decorrelated subqueries + * as correlated when we rerun PullUpCorrelatedPredicates. */ -case class OuterScopeReference(e: NamedExpression) +case class OuterReferenceForDomainJoin(e: NamedExpression) extends LeafExpression with NamedExpression with Unevaluable { override def dataType: DataType = e.dataType override def nullable: Boolean = e.nullable - override def prettyName: String = "outerScope" + override def prettyName: String = "outer" override def sql: String = s"$prettyName(${e.sql})" override def name: String = e.name override def qualifier: Seq[String] = e.qualifier override def exprId: ExprId = e.exprId override def toAttribute: Attribute = e.toAttribute - override def newInstance(): NamedExpression = OuterScopeReference(e.newInstance()) + override def newInstance(): NamedExpression = OuterReferenceForDomainJoin(e.newInstance()) + final override val nodePatterns: Seq[TreePattern] = Seq(OUTER_REFERENCE_FOR_DOMAIN_JOIN) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index c88ed966d111c..ab885ef175a44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -638,7 +638,13 @@ case class ScalarSubquery( outerAttrs = newChildren.take(outerAttrs.size), joinCond = newChildren.drop(outerAttrs.size)) - final override def nodePatternsInternal(): Seq[TreePattern] = Seq(SCALAR_SUBQUERY) + final override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.isEmpty) { + Seq(SCALAR_SUBQUERY) + } else { + Seq(NESTED_CORRELATED_SUBQUERY, SCALAR_SUBQUERY) + } + } } object ScalarSubquery { @@ -726,7 +732,17 @@ case class LateralSubquery( outerAttrs = newChildren.take(outerAttrs.size), joinCond = newChildren.drop(outerAttrs.size)) - final override def nodePatternsInternal(): Seq[TreePattern] = Seq(LATERAL_SUBQUERY) + final override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.isEmpty) { + Seq(LATERAL_SUBQUERY) + } else { + // Currently we don't support lateral subqueries with + // nested outer references. + assert(false, "Nested outer references are not supported in lateral subqueries." + + " Please file a bug if you see this error.") + Seq(NESTED_CORRELATED_SUBQUERY, LATERAL_SUBQUERY) + } + } } /** @@ -803,7 +819,13 @@ case class ListQuery( outerAttrs = newChildren.take(outerAttrs.size), joinCond = newChildren.drop(outerAttrs.size)) - final override def nodePatternsInternal(): Seq[TreePattern] = Seq(LIST_SUBQUERY) + final override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.isEmpty) { + Seq(LIST_SUBQUERY) + } else { + Seq(NESTED_CORRELATED_SUBQUERY, LIST_SUBQUERY) + } + } } /** @@ -873,7 +895,13 @@ case class Exists( outerAttrs = newChildren.take(outerAttrs.size), joinCond = newChildren.drop(outerAttrs.size)) - final override def nodePatternsInternal(): Seq[TreePattern] = Seq(EXISTS_SUBQUERY) + final override def nodePatternsInternal(): Seq[TreePattern] = { + if (outerScopeAttrs.isEmpty) { + Seq(EXISTS_SUBQUERY) + } else { + Seq(NESTED_CORRELATED_SUBQUERY, EXISTS_SUBQUERY) + } + } } case class UnresolvedExistsPlanId(planId: Long) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala index 4f8d7aefb2567..311db629055b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.TreePattern.{NESTED_CORRELATED_SUBQUERY, OUTER_REFERENCE, PLAN_EXPRESSION} +import org.apache.spark.sql.catalyst.trees.TreePattern.{NESTED_CORRELATED_SUBQUERY, OUTER_REFERENCE, OUTER_REFERENCE_FOR_DOMAIN_JOIN, PLAN_EXPRESSION} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.Utils @@ -128,6 +128,14 @@ object DecorrelateInnerQuery extends PredicateHelper { AttributeSet(expression.collect { case o: OuterReference => o.toAttribute }) } + /** + * Collect outer references for domain joins + * in an expressions that are in the output attributes of the outer plan. + */ + private def collectOuterReferencesForDomainJoin(expression: Expression): AttributeSet = { + AttributeSet(expression.collect { case o: OuterReferenceForDomainJoin => o.toAttribute }) + } + /** * Collect outer references in a sequence of expressions that are in the output attributes * of the outer plan. @@ -445,8 +453,12 @@ object DecorrelateInnerQuery extends PredicateHelper { // We only needs the domain join conditions that contain outer references, // which stores the mapping between the domain attributes and the outer plan attributes. val conditions = splitConjunctivePredicates(outerJoinCondition.get) - val (conditionsContainingOuter, conditionsNotContainingOuter) = - conditions.partition(_.containsPattern(OUTER_REFERENCE)) + val (conditionsContainingOuterReferenceForDomainJoin, conditionsNotContainingOuter) = + conditions.partition(_.containsPattern(OUTER_REFERENCE_FOR_DOMAIN_JOIN)) + val conditionsContainingOuter = + conditionsContainingOuterReferenceForDomainJoin.map(_.transform { + case OuterReferenceForDomainJoin(a) => OuterReference(a) + }) val domainAttrMap = buildDomainAttrMap(conditionsContainingOuter, domainAttrs) assert((joinType == Inner && conditionsNotContainingOuter.isEmpty) || (joinType == LeftOuter && conditionsNotContainingOuter.nonEmpty), @@ -479,7 +491,7 @@ object DecorrelateInnerQuery extends PredicateHelper { // Join joinType condition // :- Domain // +- Inner Query - case _ => Join(domain, newChild, joinType, outerJoinCondition, JoinHint.NONE) + case _ => Join(domain, newChild, joinType, domainJoinCond, JoinHint.NONE) } } assert(newChild.outputSet.subsetOf(plan.outputSet)) @@ -840,9 +852,9 @@ object DecorrelateInnerQuery extends PredicateHelper { } def insertDomainJoin( - plan: LogicalPlan, - attributes: Seq[Attribute] - ): (LogicalPlan, Seq[Expression], AttributeMap[Attribute]) = { + plan: LogicalPlan, + attributes: Seq[Attribute] + ): (LogicalPlan, Seq[Expression], AttributeMap[Attribute]) = { val domains = attributes.map(_.newInstance()) // A placeholder to be rewritten into domain join. val outerReferenceMap = Utils.toMap(attributes, domains) @@ -874,7 +886,12 @@ object DecorrelateInnerQuery extends PredicateHelper { cond } val joinConditions: Seq[Expression] = conditions.toSeq - val domainJoin = DomainJoin(domains, plan, Inner, Some(joinConditions.reduce(And))) + val domainJoinCondition = joinConditions.map{ + _.transform { + case OuterReference(a) => OuterReferenceForDomainJoin(a) + } + } + val domainJoin = DomainJoin(domains, plan, Inner, Some(domainJoinCondition.reduce(And))) (domainJoin, joinConditions, AttributeMap(outerReferenceMap)) } @@ -1337,7 +1354,7 @@ object DecorrelateInnerQuery extends PredicateHelper { if (containsNestedCorrelations) { val conds: Seq[Expression] = mapping.map { case (o, a) => - val cond = EqualNullSafe(a, OuterReference(o)) + val cond = EqualNullSafe(a, OuterReferenceForDomainJoin(o)) // SPARK-40615: Certain data types (e.g. MapType) do not support ordering, so // the EqualNullSafe join condition can become unresolved. if (!cond.resolved) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1c2f87837e9c5..54352d1597ebb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -205,8 +205,12 @@ abstract class Optimizer(catalogManager: CatalogManager) OptimizeOneRowRelationSubquery, PullOutNestedDataOuterRefExpressions, PullupCorrelatedPredicates), + // This batch rewrites all correlated subqueries along with any domain joins inside. + // Each rule in the batch is only effective when there are nested correlated subqueries + // in the plan. Batch("Rewrite Nested Correlated Subqueries", Once, RewriteDomainJoinsInOnePass, + RewriteCorrelatedSubqueriesInOnePass ), // Subquery batch applies the optimizer rules recursively. Therefore, it makes no sense // to enforce idempotence on it and we change this batch from Once to FixedPoint(1). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index d755ed9455365..4bd8e0e2f086b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -627,7 +627,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe * Extract all correlated scalar subqueries from an expression. The subqueries are collected using * the given collector. The expression is rewritten and returned. */ - private def extractCorrelatedScalarSubqueries[E <: Expression]( + def extractCorrelatedScalarSubqueries[E <: Expression]( expression: E, subqueries: ArrayBuffer[ScalarSubquery]): E = { val newExpression = expression.transformWithPruning(_.containsPattern(SCALAR_SUBQUERY)) { @@ -795,7 +795,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] with AliasHelpe val neededInnerAttrs = conditionsContainInnerReferences.flatMap(_.collect { case InnerReference(a) => a }) - assert(neededInnerAttrs.isEmpty == outerScopeAttrs.isEmpty, + assert(neededInnerAttrs.isEmpty || outerScopeAttrs.nonEmpty, "Inner references are not allowed for subqueries without nested correlations") val query = DecorrelateInnerQuery.rewriteDomainJoins(currentChild, sub, conditions) @@ -1273,3 +1273,53 @@ object RewriteDomainJoinsInOnePass extends Rule[LogicalPlan] { } } } + +/** + * This rule rewrites all correlated subqueries into joins. + * It rewrites the subqueries in a bottom up manner. + * Now it only works for correlated scalar subqueries as currently + * only scalar subqueries are allowed to have nested outer references. + */ +object RewriteCorrelatedSubqueriesInOnePass extends Rule[LogicalPlan] { + private def containsCorrelatedScalarSubquery(e: Expression): Boolean = { + e.exists { + case s: ScalarSubquery if s.children.nonEmpty => true + case _ => false + } + } + + private def apply0(plan: LogicalPlan): LogicalPlan = { + val newPlan = plan.transformUpWithPruning(_.containsPattern(SCALAR_SUBQUERY)) { + case p: LogicalPlan if p.expressions.exists(containsCorrelatedScalarSubquery) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + p.expressions.map( + RewriteCorrelatedScalarSubquery.extractCorrelatedScalarSubqueries(_, subqueries)) + val newSubqueries = subqueries.map { + case sub => + val newPlan = apply0(sub.plan) + sub.withNewPlan(newPlan) + } + val replaceMap = subqueries.zip(newSubqueries).map { + case (oldSub, newSub) => oldSub -> newSub + }.toMap + val newP = p.transformExpressionsUpWithPruning(_.containsPattern(SCALAR_SUBQUERY)) { + case s: ScalarSubquery => replaceMap.applyOrElse(s, (s: ScalarSubquery) => s) + } + newP + } + RewriteCorrelatedScalarSubquery(newPlan) + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (plan.containsPattern(NESTED_CORRELATED_SUBQUERY)) { + val newPlan = apply0(plan) + // After rewrite, there should be no nested correlated subqueries left. + assert(!newPlan.containsPattern(NESTED_CORRELATED_SUBQUERY), + "There should be no nested correlated subqueries left after" + + "RewriteCorrelatedSubqueriesInOnePass.") + newPlan + } else { + plan + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index ac85b0de2859c..ba833e58a07e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -78,6 +78,7 @@ object TreePattern extends Enumeration { val SERIALIZE_FROM_OBJECT: Value = Value val OR: Value = Value val OUTER_REFERENCE: Value = Value + val OUTER_REFERENCE_FOR_DOMAIN_JOIN: Value = Value val PARAMETER: Value = Value val PARAMETERIZED_QUERY: Value = Value val PIPE_EXPRESSION: Value = Value diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out new file mode 100644 index 0000000000000..cd2a759c71451 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -0,0 +1,800 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS myt1 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.myt1 + + +-- !query +DROP TABLE IF EXISTS myt2 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.myt2 + + +-- !query +DROP TABLE IF EXISTS myt3 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.myt3 + + +-- !query +CREATE TABLE myt1(a INT, b INT, c INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`myt1`, false + + +-- !query +CREATE TABLE myt2(a INT, b INT, c INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`myt2`, false + + +-- !query +CREATE TABLE myt3(a INT, b INT, c INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`myt3`, false + + +-- !query +INSERT INTO myt1 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/myt1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/myt1], Append, `spark_catalog`.`default`.`myt1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/myt1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO myt2 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/myt2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/myt2], Append, `spark_catalog`.`default`.`myt2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/myt2), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO myt3 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/myt3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/myt3], Append, `spark_catalog`.`default`.`myt3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/myt3), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query analysis +Project [a#x, b#x, c#x] ++- Filter (a#x = scalar-subquery#x [b#x && c#x]) + : +- Aggregate [max(a#x) AS max(a)#x] + : +- Filter ((a#x = scalar-subquery#x [b#x && c#x]) AND (b#x > outer(b#x))) + : : +- Aggregate [max(a#x) AS max(a)#x] + : : +- Filter ((b#x > outer(b#x)) AND (c#x > outer(c#x))) + : : +- SubqueryAlias spark_catalog.default.myt3 + : : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.default.myt2 + : +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +) +-- !query analysis +Project [a#x, b#x, c#x] ++- Filter (a#x = scalar-subquery#x [b#x && c#x]) + : +- Aggregate [max(a#x) AS max(a)#x] + : +- Filter ((a#x = scalar-subquery#x [b#x && c#x]) AND (b#x = outer(b#x))) + : : +- Aggregate [max(a#x) AS max(a)#x] + : : +- Filter ((b#x = outer(b#x)) AND (c#x = outer(c#x))) + : : +- SubqueryAlias spark_catalog.default.myt3 + : : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.default.myt2 + : +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query analysis +Project [a#x, b#x, c#x] ++- Filter (cast(a#x as bigint) = scalar-subquery#x [b#x && c#x]) + : +- Aggregate [count(a#x) AS count(a)#xL] + : +- Filter ((cast(a#x as bigint) = scalar-subquery#x [b#x && c#x]) AND (b#x > outer(b#x))) + : : +- Aggregate [count(a#x) AS count(a)#xL] + : : +- Filter ((b#x > outer(b#x)) AND (c#x > outer(c#x))) + : : +- SubqueryAlias spark_catalog.default.myt3 + : : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.default.myt2 + : +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +) +-- !query analysis +Project [a#x, b#x, c#x] ++- Filter (cast(a#x as bigint) = scalar-subquery#x [b#x && c#x]) + : +- Aggregate [count(a#x) AS count(a)#xL] + : +- Filter ((cast(a#x as bigint) = scalar-subquery#x [b#x && c#x]) AND (b#x = outer(b#x))) + : : +- Aggregate [count(a#x) AS count(a)#xL] + : : +- Filter ((b#x = outer(b#x)) AND (c#x = outer(c#x))) + : : +- SubqueryAlias spark_catalog.default.myt3 + : : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.default.myt2 + : +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet + + +-- !query +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 159, + "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 159, + "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 163, + "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 163, + "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" + } ] +} + + +-- !query +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'MAX'", + "hint" : "" + } +} + + +-- !query +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'MAX'", + "hint" : "" + } +} + + +-- !query +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'COUNT'", + "hint" : "" + } +} + + +-- !query +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'COUNT'", + "hint" : "" + } +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 203, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 203, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 207, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 207, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" + } ] +} + + +-- !query +SELECT myt1.a +FROM myt1 +WHERE EXISTS ( + SELECT 1 + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query analysis +Project [a#x] ++- Filter exists#x [b#x && c#x] + : +- Project [1 AS 1#x] + : +- Filter ((a#x = scalar-subquery#x [b#x && c#x]) AND (b#x > outer(b#x))) + : : +- Aggregate [max(a#x) AS max(a)#x] + : : +- Filter ((b#x > outer(b#x)) AND (c#x > outer(c#x))) + : : +- SubqueryAlias spark_catalog.default.myt3 + : : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.default.myt2 + : +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet + + +-- !query +SELECT myt1.a +FROM myt1 +WHERE myt1.b = ( + SELECT myt2.b + FROM myt2 + WHERE EXISTS ( + SELECT 1 + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "spark_catalog.default.myt1.c" + } +} + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1 +-- !query analysis +Project [1 AS 1#x] ++- Filter (scalar-subquery#x [c0#x] = 1) + : +- Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x] + : : +- Project [outer(c0#x)] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias t0 + +- Project [1#x AS c0#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] + : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] + : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] + : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] + : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] + : : : : +- OneRowRelation + : : : +- Filter (i#x = outer(i#x)) + : : : +- SubqueryAlias i2 + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter NOT (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter (i#x = outer(i#x)) + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalarsubquery(i)#x] + : +- SubqueryAlias s1 + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join Inner + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join LeftOuter, (k#x = k#x) + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [outer(i#x) IN (1,2,3,4,5,6,7,8) AS (outer(i1.i) IN (1, 2, 3, 4, 5, 6, 7, 8))#x] + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/combined-subquery.sql rename to sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/exists-subquery.sql rename to sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/lateral-subquery.sql rename to sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql new file mode 100644 index 0000000000000..0000a5e35d2cd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql @@ -0,0 +1,33 @@ +--ONLY_IF spark +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; + +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1; + +DROP TABLE IF EXISTS table_integers; +CREATE TABLE table_integers(i INTEGER); +INSERT INTO table_integers VALUES (1), (2), (3), (NULL); + +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i; + +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/subquery-not-supported.sql rename to sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql index 0000a5e35d2cd..4f7ac915a091f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -1,9 +1,262 @@ ---ONLY_IF spark set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; -set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; -set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; +-- One test case for each following type: +-- --------------------------------------------------------------- +-- | SubqueryPosition | VulnerableToCountBug | TriggerDomainJoin| +-- --------------------------------------------------------------- +-- 1 | Filter | False | True | +-- --------------------------------------------------------------- +-- 2 | Filter | False | False | +-- --------------------------------------------------------------- +-- 3 | Filter | True | True | +-- --------------------------------------------------------------- +-- 4 | Filter | True | False | +-- --------------------------------------------------------------- +-- 5 | Project | False | True | +-- --------------------------------------------------------------- +-- 6 | Project | False | False | +-- --------------------------------------------------------------- +-- 7 | Project | True | True | +-- --------------------------------------------------------------- +-- 8 | Project | True | False | +-- --------------------------------------------------------------- +-- 9 | Aggregate | False | True | +-- --------------------------------------------------------------- +-- 10| Aggregate | False | False | +-- --------------------------------------------------------------- +-- 11| Aggregate | True | True | +-- --------------------------------------------------------------- +-- 12| Aggregate | True | False | +-- --------------------------------------------------------------- +-- 13| Filter(havingClause) | False | True | +-- --------------------------------------------------------------- +-- 14| Filter(havingClause) | False | False | +-- --------------------------------------------------------------- +-- 15| Filter(havingClause) | True | True | +-- --------------------------------------------------------------- +-- 16| Filter(havingClause) | True | False | +-- --------------------------------------------------------------- + +DROP TABLE IF EXISTS myt1; +DROP TABLE IF EXISTS myt2; +DROP TABLE IF EXISTS myt3; +CREATE TABLE myt1(a INT, b INT, c INT); +CREATE TABLE myt2(a INT, b INT, c INT); +CREATE TABLE myt3(a INT, b INT, c INT); +INSERT INTO myt1 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL); +INSERT INTO myt2 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL); +INSERT INTO myt3 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL); + +-- query 1 +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +); +-- query 2 +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +); +-- query 3 +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +); +-- query 4 +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +); +-- query 5 +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1; +-- query 6 +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1; +-- query 7 +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1; +-- query 8 +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1; +-- query 9 +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1; +-- query 10 +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1; +-- query 11 +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1; +-- query 12 +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1; +-- query 13 +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +); +-- query 14 +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +); +-- query 15 +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +); +-- query 16 +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +); + +-- test that queries containing both nested correlated scalar subqueries +-- and other types of subqueries will be blocked by the analyzer when +-- we only support nested correlated scalar subqueries. +SELECT myt1.a +FROM myt1 +WHERE EXISTS ( + SELECT 1 + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +); + +SELECT myt1.a +FROM myt1 +WHERE myt1.b = ( + SELECT myt2.b + FROM myt2 + WHERE EXISTS ( + SELECT 1 + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +); + +-- testcases extracted from DUCKDB SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1; DROP TABLE IF EXISTS table_integers; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql.out new file mode 100644 index 0000000000000..eb712556160d6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/combined-subquery.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS tbl +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl + + +-- !query +CREATE TABLE tbl(a TINYINT, b SMALLINT, c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +INSERT INTO tbl VALUES (1, 2, 3, 4, '5', DATE '1992-01-01', TIMESTAMP '1992-01-01 00:00:00') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl], Append, `spark_catalog`.`default`.`tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl), [a, b, c, d, e, f, g] ++- Project [cast(col1#x as tinyint) AS a#x, cast(col2#x as smallint) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as bigint) AS d#xL, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col5#x as string), 1)) AS e#x, cast(col6#x as date) AS f#x, cast(col7#x as timestamp) AS g#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x] + + +-- !query +SELECT t1.c+(SELECT t1.b FROM tbl t2 WHERE EXISTS(SELECT t1.b+t2.a)) FROM tbl t1 +-- !query analysis +Project [(c#x + cast(scalar-subquery#x [b#x] as int)) AS (c + scalarsubquery(b))#x] +: +- Project [outer(b#x)] +: +- Filter exists#x [b#x && a#x] +: : +- Project [(outer(b#x) + cast(outer(a#x) as smallint)) AS (outer(t1.b) + outer(t2.a))#x] +: : +- OneRowRelation +: +- SubqueryAlias t2 +: +- SubqueryAlias spark_catalog.default.tbl +: +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet ++- SubqueryAlias t1 + +- SubqueryAlias spark_catalog.default.tbl + +- Relation spark_catalog.default.tbl[a#x,b#x,c#x,d#xL,e#x,f#x,g#x] parquet + + +-- !query +SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql.out new file mode 100644 index 0000000000000..c47867443fde7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/exists-subquery.sql.out @@ -0,0 +1,430 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT + i, + ( + SELECT SUM(ss1.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss1 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [1 AS 1#x] + : : +- Filter (outer(i#x) > i#x) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT + i, + ( + SELECT SUM(ss2.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE s1.i = i1.i + AND EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE t2.i = s1.i + ) + ) ss2 + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [1 AS 1#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT + i, + ( + SELECT SUM(ss1.i) + SUM(ss2.i) + FROM ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i > t2.i + ) + ) ss1 + LEFT OUTER JOIN ( + SELECT s1.i + FROM table_integers s1 + WHERE EXISTS ( + SELECT 1 + FROM table_integers t2 + WHERE s1.i = t2.i + ) + ) ss2 + ON ss1.i = ss2.i + ) AS j +FROM table_integers i1 +ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [(sum(i#x) + sum(i#x)) AS (sum(i) + sum(i))#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias ss1 + : : +- Project [i#x] + : : +- Filter exists#x [i#x] + : : : +- Project [1 AS 1#x] + : : : +- Filter (outer(i#x) > i#x) + : : : +- SubqueryAlias t2 + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- SubqueryAlias s1 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [1 AS 1#x] + : : +- Filter (outer(i#x) = i#x) + : : +- SubqueryAlias t2 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter CASE WHEN ((i#x = outer(i#x)) AND exists#x [i#x]) THEN true ELSE false END + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE (SELECT i FROM table_integers WHERE i=s1.i) = 1) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter (scalar-subquery#x [i#x] = 1) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS (SELECT i FROM table_integers WHERE i=s1.i)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i OR i=ANY(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE CASE WHEN (i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) THEN true ELSE false END) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter CASE WHEN ((i#x = outer(i#x)) AND exists#x [i#x]) THEN true ELSE false END + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x]) + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss1 + : +- Project [i#x] + : +- Filter exists#x [i#x && i#x] + : : +- Project [i#x] + : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [(sum(i#x) + sum(i#x)) AS (sum(i) + sum(i))#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias ss1 + : : +- Project [i#x] + : : +- Filter ((i#x = outer(i#x)) AND exists#x [i#x && i#x]) + : : : +- Project [i#x] + : : : +- Filter (NOT (i#x = outer(i#x)) AND (outer(i#x) > i#x)) + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- SubqueryAlias s1 + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter exists#x [i#x] + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +DROP TABLE IF EXISTS tbl_ProductSales +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales + + +-- !query +DROP TABLE IF EXISTS another_T +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T + + +-- !query +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false + + +-- !query +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false + + +-- !query +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] ++- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] ++- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON EXISTS (SELECT MAX(t1.col1 + t3.col4) AS mymax FROM another_T t3 HAVING t1.col7 <> mymax)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "max((outer(t1.col1) + t3.col4))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 91, + "stopIndex" : 112, + "fragment" : "MAX(t1.col1 + t3.col4)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql.out new file mode 100644 index 0000000000000..9cedbe2e28847 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql.out @@ -0,0 +1,88 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t(ps_supplycost INT, n_name INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT NULL +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 187, + "stopIndex" : 198, + "fragment" : "ref_2.n_name" + } ] +} + + +-- !query +SELECT * +FROM + t AS ref_2, + LATERAL (SELECT (SELECT NULL + FROM (SELECT * FROM t AS ref_5, + LATERAL (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1) AS alias1) AS alias2) AS alias3) AS alias4 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 184, + "stopIndex" : 195, + "fragment" : "ref_2.n_name" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql.out new file mode 100644 index 0000000000000..76805b07c6851 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql.out @@ -0,0 +1,259 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1 +-- !query analysis +Project [1 AS 1#x] ++- Filter (scalar-subquery#x [c0#x] = 1) + : +- Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x] + : : +- Project [outer(c0#x)] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias t0 + +- Project [1#x AS c0#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] + : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] + : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] + : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] + : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] + : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] + : : : : +- OneRowRelation + : : : +- OneRowRelation + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] + : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] + : : : : +- OneRowRelation + : : : +- Filter (i#x = outer(i#x)) + : : : +- SubqueryAlias i2 + : : : +- SubqueryAlias spark_catalog.default.table_integers + : : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- Join LeftOuter, (i#x = i#x) + : :- SubqueryAlias s1 + : : +- Project [i#x] + : : +- Filter NOT (i#x = outer(i#x)) + : : +- SubqueryAlias spark_catalog.default.table_integers + : : +- Relation spark_catalog.default.table_integers[i#x] parquet + : +- SubqueryAlias s2 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#xL] + : +- Aggregate [sum(i#x) AS sum(i)#xL] + : +- SubqueryAlias ss2 + : +- Project [i#x] + : +- Filter (i#x = outer(i#x)) + : +- SubqueryAlias s1 + : +- SubqueryAlias spark_catalog.default.table_integers + : +- Relation spark_catalog.default.table_integers[i#x] parquet + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [scalarsubquery(i)#x] + : +- SubqueryAlias s1 + : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join Inner + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] + : +- Project [(k#x + k#x) AS (k + k)#x] + : +- Join LeftOuter, (k#x = k#x) + : :- SubqueryAlias s1 + : : +- Project [scalar-subquery#x [i#x] AS k#x] + : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : : +- OneRowRelation + : : +- OneRowRelation + : +- SubqueryAlias s2 + : +- Project [scalar-subquery#x [i#x] AS k#x] + : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] + : : +- OneRowRelation + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet + + +-- !query +SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +Sort [i#x ASC NULLS FIRST], true ++- Project [i#x, scalar-subquery#x [i#x] AS j#x] + : +- Project [outer(i#x) IN (1,2,3,4,5,6,7,8) AS (outer(i1.i) IN (1, 2, 3, 4, 5, 6, 7, 8))#x] + : +- OneRowRelation + +- SubqueryAlias i1 + +- SubqueryAlias spark_catalog.default.table_integers + +- Relation spark_catalog.default.table_integers[i#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql.out new file mode 100644 index 0000000000000..c56057defa31c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql.out @@ -0,0 +1,566 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) + + +-- !query +set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true +-- !query analysis +SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 ORDER BY c0)) = 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"c0 ASC NULLS FIRST\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 57, + "stopIndex" : 67, + "fragment" : "ORDER BY c0" + } ] +} + + +-- !query +SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT 1 LIMIT c0)) = 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"outer(t0.c0)\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 64, + "fragment" : "c0" + } ] +} + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t(ps_supplycost INT, n_name INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_2`.`n_name`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 124, + "fragment" : "ref_2.n_name" + } ] +} + + +-- !query +SELECT NULL +FROM + t AS ref_2, + (SELECT (SELECT NULL + FROM (FROM t AS ref_5, + (SELECT ref_5.ps_supplycost AS c0, + ref_2.n_name AS c1)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ref_5`.`ps_supplycost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 131, + "fragment" : "ref_5.ps_supplycost" + } ] +} + + +-- !query +DROP TABLE IF EXISTS table_integers +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers + + +-- !query +CREATE TABLE table_integers(i INTEGER) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false + + +-- !query +INSERT INTO table_integers VALUES (1), (2), (3), (NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] ++- Project [cast(col1#x as int) AS i#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT i, (SELECT (SELECT i1.i+SUM(i2.i)) FROM table_integers i2) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"(i + sum(i)) AS `(outer(i1.i) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 40, + "fragment" : "SELECT i1.i+SUM(i2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT ((SELECT ((SELECT ((SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5)) FROM table_integers i4)) FROM table_integers i3)) FROM table_integers i2) AS j FROM table_integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((((sum(i) + sum(i)) + sum(i)) + sum(i)) + sum(i)) AS `((((sum(i) + sum(outer(i4.i))) + sum(outer(i3.i))) + sum(outer(i2.i))) + sum(outer(i1.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 114, + "fragment" : "SELECT SUM(i)+SUM(i4.i)+SUM(i3.i)+SUM(i2.i)+SUM(i1.i) FROM table_integers i5" + } ] +} + + +-- !query +SELECT (SELECT (SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3) FROM table_integers i2) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(outer(i1.i)) + sum(outer(i2.i))) + sum(i))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 75, + "fragment" : "SELECT SUM(i1.i)+SUM(i2.i)+SUM(i3.i) FROM table_integers i3" + } ] +} + + +-- !query +SELECT i, SUM(i), (SELECT (SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers) FROM table_integers i2) FROM table_integers i1 GROUP BY i ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"((sum(i) + sum(i)) + sum(i)) AS `((sum(i) + sum(outer(i1.i))) + sum(outer(i2.i)))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 80, + "fragment" : "SELECT SUM(i)+SUM(i1.i)+SUM(i2.i) FROM table_integers" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(i)+(SELECT 42+i1.i) FROM table_integers) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(i)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 41, + "fragment" : "(SELECT 42+i1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Join Inner, (scalar-subquery#x [i#x && i#x] = scalar-subquery#x [i#x && i#x])\n: :- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s1.i))#x]\n: : +- OneRowRelation\n: +- Project [(outer(i#x) + outer(i#x)) AS (outer(i1.i) + outer(s2.i))#x]\n: +- OneRowRelation\n:- SubqueryAlias s1\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s2\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 120, + "fragment" : "INNER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s1.i) FROM table_integers s1 LEFT OUTER JOIN table_integers s2 ON (SELECT i1.i+s1.i)=(SELECT i1.i+s2.i)) AS j FROM table_integers i1 ORDER BY i + +SELECT (SELECT (SELECT COVAR_POP(i2.i, i3.i) FROM table_integers i3) FROM table_integers i2 ORDER BY i NULLS LAST LIMIT 1) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT (SELECT (SELECT COVAR_POP(i1.i, i3.i) FROM table_integers i3) FROM table_integers i2 LIMIT 1) FROM table_integers i1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "covar_pop(CAST(outer(i1.i) AS DOUBLE), CAST(i3.i AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 44, + "fragment" : "COVAR_POP(i1.i, i3.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 160, + "stopIndex" : 257, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 160, + "stopIndex" : 257, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i > i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((i#x = outer(i#x)) AND exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 171, + "stopIndex" : 268, + "fragment" : "SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(ss1.i)+SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i AND EXISTS(SELECT i FROM table_integers WHERE i<>s1.i AND s1.i>i)) ss1 LEFT OUTER JOIN (SELECT i FROM table_integers s1 WHERE i<>i1.i OR EXISTS(SELECT i FROM table_integers WHERE i=s1.i)) ss2 ON ss1.i=ss2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (NOT (i#x = outer(i#x)) OR exists#x [i#x])\n: +- Project [i#x]\n: +- Filter (i#x = outer(i#x))\n: +- SubqueryAlias spark_catalog.default.table_integers\n: +- Relation spark_catalog.default.table_integers[i#x] parquet\n+- SubqueryAlias s1\n +- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 180, + "stopIndex" : 277, + "fragment" : "SELECT i FROM table_integers s1 WHERE i<>i1.i OR EXISTS(SELECT i FROM table_integers WHERE i=s1.i)" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i=i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 109, + "fragment" : "SELECT i FROM table_integers WHERE i=i1.i" + } ] +} + + +-- !query +SELECT i, (SELECT SUM(s2.i) FROM table_integers s1 LEFT OUTER JOIN (SELECT i FROM table_integers WHERE i<>i1.i) s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter NOT (i#x = outer(i#x))\n+- SubqueryAlias spark_catalog.default.table_integers\n +- Relation spark_catalog.default.table_integers[i#x] parquet\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 110, + "fragment" : "SELECT i FROM table_integers WHERE i<>i1.i" + } ] +} + + +-- !query +DROP TABLE IF EXISTS tbl_ProductSales +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl_ProductSales + + +-- !query +DROP TABLE IF EXISTS another_T +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.another_T + + +-- !query +CREATE TABLE tbl_ProductSales (ColID int, Product_Category varchar(64), Product_Name varchar(64), TotalSales int) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl_ProductSales`, false + + +-- !query +CREATE TABLE another_T (col1 INT, col2 INT, col3 INT, col4 INT, col5 INT, col6 INT, col7 INT, col8 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`another_T`, false + + +-- !query +INSERT INTO tbl_ProductSales VALUES (1,'Game','Mobo Game',200),(2,'Game','PKO Game',400),(3,'Fashion','Shirt',500),(4,'Fashion','Shorts',100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tbl_productsales, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tbl_productsales], Append, `spark_catalog`.`default`.`tbl_productsales`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tbl_productsales), [ColID, Product_Category, Product_Name, TotalSales] ++- Project [cast(col1#x as int) AS ColID#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col2#x as string), 64)) AS Product_Category#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(col3#x as string), 64)) AS Product_Name#x, cast(col4#x as int) AS TotalSales#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO another_T VALUES (1,2,3,4,5,6,7,8), (11,22,33,44,55,66,77,88), (111,222,333,444,555,666,777,888), (1111,2222,3333,4444,5555,6666,7777,8888) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/another_t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/another_t], Append, `spark_catalog`.`default`.`another_t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/another_t), [col1, col2, col3, col4, col5, col6, col7, col8] ++- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t1.col7 <> (SELECT MAX(t1.col1 + t3.col4) FROM another_T t3)) FROM another_T t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "max((outer(t1.col1) + t3.col4))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 116, + "fragment" : "MAX(t1.col1 + t3.col4)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MAX(col7)) UNION ALL (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1)) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 47, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN 1 IN (SELECT (SELECT MIN(ColID) FROM tbl_ProductSales INNER JOIN another_T t2 ON t2.col5 = t2.col1) UNION ALL (SELECT MAX(col7))) THEN 2 ELSE NULL END FROM another_T t1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 129, + "stopIndex" : 144, + "fragment" : "SELECT MAX(col7)" + } ] +} + + +-- !query +SELECT CASE WHEN NOT col1 NOT IN (SELECT (SELECT MAX(col7)) UNION (SELECT MIN(ColID) FROM tbl_ProductSales LEFT JOIN another_T t2 ON t2.col5 = t1.col1)) THEN 1 ELSE 2 END FROM another_T t1 GROUP BY col1 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"max(col7) AS `max(outer(t1.col7))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 58, + "fragment" : "SELECT MAX(col7)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index 76805b07c6851..0c999f3b2bce7 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -1,259 +1,731 @@ -- Automatically generated by SQLQueryTestSuite -- !query set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true --- !query analysis -SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled,Some(true)) +-- !query schema +struct +-- !query output +spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled true -- !query set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true --- !query analysis -SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled,Some(true)) +-- !query schema +struct +-- !query output +spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled true -- !query -set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true --- !query analysis -SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled,Some(true)) +DROP TABLE IF EXISTS myt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS myt2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS myt3 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE myt1(a INT, b INT, c INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE myt2(a INT, b INT, c INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE myt3(a INT, b INT, c INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO myt1 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO myt2 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO myt3 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +) +-- !query schema +struct +-- !query output +0 0 0 +1 1 1 +2 2 2 +3 3 3 + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query schema +struct +-- !query output +0 0 0 + + +-- !query +SELECT * +FROM myt1 +WHERE myt1.a = ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) AND myt2.b = myt1.b +) +-- !query schema +struct +-- !query output +0 0 0 +1 1 1 + + +-- !query +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 159, + "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT MAX(myt2.a), ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 159, + "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 163, + "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" + } ] +} + + +-- !query +SELECT myt1.a, ( + SELECT COUNT(myt2.a), ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExpr" : "\"scalarsubquery(b, c)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 163, + "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" + } ] +} + + +-- !query +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'MAX'", + "hint" : "" + } +} + + +-- !query +SELECT MIN( + SELECT MAX( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'MAX'", + "hint" : "" + } +} + + +-- !query +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'COUNT'", + "hint" : "" + } +} + + +-- !query +SELECT COUNT( + SELECT COUNT( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.b = myt2.b AND myt3.c = myt1.c + ) + FROM myt2 + ) +) +FROM myt1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'COUNT'", + "hint" : "" + } +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 203, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT MAX(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 203, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a > MAX(myt1.a) + ) AND myt2.b > myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 207, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" + } ] +} + + +-- !query +SELECT MAX(myt1.a) +FROM myt1 +HAVING ( + SELECT COUNT(myt2.a) + FROM myt2 + WHERE myt2.a = ( + SELECT COUNT(myt3.a) + FROM myt3 + WHERE myt3.a = MAX(myt1.a) + ) AND myt2.b = myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"max(a)\"", + "missingAttributes" : "\"b\"", + "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 207, + "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" + } ] +} -- !query -set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true --- !query analysis -SetCommand (spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled,Some(true)) +SELECT myt1.a +FROM myt1 +WHERE EXISTS ( + SELECT 1 + FROM myt2 + WHERE myt2.a = ( + SELECT MAX(myt3.a) + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", + "messageParameters" : { + "message" : "The Spark SQL phase optimization failed with an internal error. You hit a bug in Spark or the Spark plugins you use. Please, report this bug to the corresponding communities or vendors, and provide the full stack trace." + } +} + + +-- !query +SELECT myt1.a +FROM myt1 +WHERE myt1.b = ( + SELECT myt2.b + FROM myt2 + WHERE EXISTS ( + SELECT 1 + FROM myt3 + WHERE myt3.b > myt2.b AND myt3.c > myt1.c + ) AND myt2.b > myt1.b +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_REFERENCES_IN_SUBQUERY_NOT_SUPPORTED", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "spark_catalog.default.myt1.c" + } +} -- !query SELECT 1 FROM (SELECT 1) t0(c0) WHERE (SELECT (SELECT c0)) = 1 --- !query analysis -Project [1 AS 1#x] -+- Filter (scalar-subquery#x [c0#x] = 1) - : +- Project [scalar-subquery#x [c0#x] AS scalarsubquery(c0)#x] - : : +- Project [outer(c0#x)] - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias t0 - +- Project [1#x AS c0#x] - +- Project [1 AS 1#x] - +- OneRowRelation +-- !query schema +struct<1:int> +-- !query output +1 -- !query DROP TABLE IF EXISTS table_integers --- !query analysis -DropTable true, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_integers +-- !query schema +struct<> +-- !query output + -- !query CREATE TABLE table_integers(i INTEGER) --- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`table_integers`, false +-- !query schema +struct<> +-- !query output + -- !query INSERT INTO table_integers VALUES (1), (2), (3), (NULL) --- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/table_integers, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/table_integers], Append, `spark_catalog`.`default`.`table_integers`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/table_integers), [i] -+- Project [cast(col1#x as int) AS i#x] - +- LocalRelation [col1#x] +-- !query schema +struct<> +-- !query output + -- !query SELECT i, (SELECT (SELECT 42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] - : +- Project [((scalar-subquery#x [i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i) + 42) + outer(i1.i))#x] - : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 86 +2 88 +3 90 -- !query SELECT i, (SELECT (SELECT (SELECT (SELECT 42+i1.i)++i1.i)+42+i1.i)+42+i1.i) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x] AS j#x] - : +- Project [((scalar-subquery#x [i#x && i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i, i) + 42) + outer(i1.i))#x] - : : +- Project [((scalar-subquery#x [i#x && i#x] + 42) + outer(i#x)) AS ((scalarsubquery(i, i) + 42) + outer(i1.i))#x] - : : : +- Project [(scalar-subquery#x [i#x] + positive(outer(i#x))) AS (scalarsubquery(i) + (+ outer(i1.i)))#x] - : : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : : : +- OneRowRelation - : : : +- OneRowRelation - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 130 +2 134 +3 138 -- !query SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i)))) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS j#x] - : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] - : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] - : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i)#x] - : : : : +- Project [((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS ((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i))#x] - : : : : +- OneRowRelation - : : : +- OneRowRelation - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 5 +2 10 +3 15 -- !query SELECT i, (SELECT (SELECT (SELECT (SELECT i1.i+i1.i+i1.i+i1.i+i1.i+i2.i) FROM table_integers i2 WHERE i2.i=i1.i))) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] - : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : : +- Project [scalar-subquery#x [i#x && i#x && i#x && i#x && i#x && i#x] AS scalarsubquery(i, i, i, i, i, i)#x] - : : : : +- Project [(((((outer(i#x) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) + outer(i#x)) AS (((((outer(i1.i) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i1.i)) + outer(i2.i))#x] - : : : : +- OneRowRelation - : : : +- Filter (i#x = outer(i#x)) - : : : +- SubqueryAlias i2 - : : : +- SubqueryAlias spark_catalog.default.table_integers - : : : +- Relation spark_catalog.default.table_integers[i#x] parquet - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} -- !query SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i=i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#xL] - : +- Aggregate [sum(i#x) AS sum(i)#xL] - : +- Join LeftOuter, (i#x = i#x) - : :- SubqueryAlias s1 - : : +- Project [i#x] - : : +- Filter (i#x = outer(i#x)) - : : +- SubqueryAlias spark_catalog.default.table_integers - : : +- Relation spark_catalog.default.table_integers[i#x] parquet - : +- SubqueryAlias s2 - : +- SubqueryAlias spark_catalog.default.table_integers - : +- Relation spark_catalog.default.table_integers[i#x] parquet - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 1 +2 2 +3 3 -- !query SELECT i, (SELECT SUM(s1.i) FROM (SELECT i FROM table_integers WHERE i<>i1.i) s1 LEFT OUTER JOIN table_integers s2 ON s1.i=s2.i) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#xL] - : +- Aggregate [sum(i#x) AS sum(i)#xL] - : +- Join LeftOuter, (i#x = i#x) - : :- SubqueryAlias s1 - : : +- Project [i#x] - : : +- Filter NOT (i#x = outer(i#x)) - : : +- SubqueryAlias spark_catalog.default.table_integers - : : +- Relation spark_catalog.default.table_integers[i#x] parquet - : +- SubqueryAlias s2 - : +- SubqueryAlias spark_catalog.default.table_integers - : +- Relation spark_catalog.default.table_integers[i#x] parquet - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 5 +2 4 +3 3 -- !query SELECT i, (SELECT SUM(ss2.i) FROM (SELECT i FROM table_integers s1 WHERE i=i1.i) ss2) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#xL] - : +- Aggregate [sum(i#x) AS sum(i)#xL] - : +- SubqueryAlias ss2 - : +- Project [i#x] - : +- Filter (i#x = outer(i#x)) - : +- SubqueryAlias s1 - : +- SubqueryAlias spark_catalog.default.table_integers - : +- Relation spark_catalog.default.table_integers[i#x] parquet - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 1 +2 2 +3 3 -- !query SELECT i, (SELECT * FROM (SELECT (SELECT 42+i1.i)) s1) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] - : +- Project [scalarsubquery(i)#x] - : +- SubqueryAlias s1 - : +- Project [scalar-subquery#x [i#x] AS scalarsubquery(i)#x] - : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 43 +2 44 +3 45 -- !query SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] - : +- Project [(k#x + k#x) AS (k + k)#x] - : +- Join Inner - : :- SubqueryAlias s1 - : : +- Project [scalar-subquery#x [i#x] AS k#x] - : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : : +- OneRowRelation - : : +- OneRowRelation - : +- SubqueryAlias s2 - : +- Project [scalar-subquery#x [i#x] AS k#x] - : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", + "messageParameters" : { + "message" : "Couldn't find i#x in []" + } +} -- !query SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x && i#x] AS j#x] - : +- Project [(k#x + k#x) AS (k + k)#x] - : +- Join LeftOuter, (k#x = k#x) - : :- SubqueryAlias s1 - : : +- Project [scalar-subquery#x [i#x] AS k#x] - : : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : : +- OneRowRelation - : : +- OneRowRelation - : +- SubqueryAlias s2 - : +- Project [scalar-subquery#x [i#x] AS k#x] - : : +- Project [(42 + outer(i#x)) AS (42 + outer(i1.i))#x] - : : +- OneRowRelation - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", + "messageParameters" : { + "message" : "Couldn't find i#x in []" + } +} -- !query SELECT i, (SELECT i1.i IN (1, 2, 3, 4, 5, 6, 7, 8)) AS j FROM table_integers i1 ORDER BY i --- !query analysis -Sort [i#x ASC NULLS FIRST], true -+- Project [i#x, scalar-subquery#x [i#x] AS j#x] - : +- Project [outer(i#x) IN (1,2,3,4,5,6,7,8) AS (outer(i1.i) IN (1, 2, 3, 4, 5, 6, 7, 8))#x] - : +- OneRowRelation - +- SubqueryAlias i1 - +- SubqueryAlias spark_catalog.default.table_integers - +- Relation spark_catalog.default.table_integers[i#x] parquet +-- !query schema +struct +-- !query output +NULL NULL +1 true +2 true +3 true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 49506f746eedb..3f8ece0f34962 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -472,7 +472,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}cte.sql")) { CTETestCase(testCaseName, absPath, resultFile) :: Nil } else if (file.getAbsolutePath.startsWith( - s"$inputFilePath${File.separator}subquery${File.separator}nestedcorrelation" + s"$inputFilePath${File.separator}subquery${File.separator}nestedcorrelation-analyzer-only" )) { AnalyzerTestCase(testCaseName, absPath, resultFile) :: Nil } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 14b8154259ccb..196460943fad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2847,32 +2847,109 @@ class SubquerySuite extends QueryTest ) } - test("test non deterministic query") { - sql("CREATE TABLE tbl(a TINYINT, b SMALLINT," + - " c INTEGER, d BIGINT, e VARCHAR(1), f DATE, g TIMESTAMP);") - -// set spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled=true; -// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled=true; -// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForINSubqueries.enabled=true; -// set spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled=true; + test("query without count bug without domain joins") { + sql("CREATE TEMP VIEW t0 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t1 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t2 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t3 AS SELECT 1 AS a, 2 AS b, 3 AS c") val query = """ - |SELECT 1 FROM tbl t1 JOIN tbl t2 ON (t1.d=t2.d) WHERE - | EXISTS(SELECT t1.c FROM tbl t3 WHERE t1.d+t3.c<100 AND - | EXISTS(SELECT 1 FROM tbl t4 WHERE t2.f < DATE '2000-01-01')); + |SELECT * + |FROM t1 + |WHERE t1.a = ( + | SELECT MAX(t2.a) + | FROM t2 + | WHERE t2.a = ( + | SELECT MAX(t3.a) + | FROM t3 + | WHERE t3.b = t2.b AND t3.c = t1.c + | ) AND t2.b = t1.b + |) |""".stripMargin withSQLConf( - "spark.sql.planChangeLog.level" -> "info", "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true", - "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForEXISTSSubqueries.enabled" -> "true" + "spark.sql.planChangeLog.level" -> "info" ) { - val df = sql(query) - df.collect() - val analyzedPlan = df.queryExecution.analyzed - // scalastyle:off println - println(analyzedPlan.toString) - // scalastyle:on println + val df = sql(query).collect() + } + val querySuperNested = + """ + |SELECT * + |FROM t0 + |WHERE t0.a = ( + |SELECT t1.a + |FROM t1 + |WHERE t1.a = ( + | SELECT MAX(t2.a) + | FROM t2 + | WHERE t2.a = ( + | SELECT MAX(t3.a) + | FROM t3 + | WHERE t3.b = t2.b AND t3.c = t0.c + | ) AND t2.b = t1.b + | ) + |) + |""".stripMargin + withSQLConf( + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true", + "spark.sql.planChangeLog.level" -> "info" + ) { + val df = sql(querySuperNested).collect() + } + } + + test("query without count bug with domain joins") { + sql("CREATE TEMP VIEW t0 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t1 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t2 AS SELECT 1 AS a, 2 AS b, 3 AS c") + sql("CREATE TEMP VIEW t3 AS SELECT 1 AS a, 2 AS b, 3 AS c") + val query = + """ + |SELECT * + |FROM t1 + |WHERE t1.a > ( + | SELECT MAX(t2.a) + | FROM t2 + | WHERE t2.a > ( + | SELECT MAX(t3.a) + | FROM t3 + | WHERE t3.b > t2.b AND t3.c > t1.c + | ) AND t2.b > t1.b + |) + |""".stripMargin + withSQLConf( + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true", + "spark.sql.planChangeLog.level" -> "info" + ) { + val df = sql(query).collect() + } + val querySuperNested = + """ + |SELECT * + |FROM t0 + |WHERE t0.a > ( + |SELECT t1.a + |FROM t1 + |WHERE t1.a > ( + | SELECT MAX(t2.a) + | FROM t2 + | WHERE t2.a > ( + | SELECT MAX(t3.a) + | FROM t3 + | WHERE t3.b > t2.b AND t3.c > t0.c + | ) AND t2.b > t1.b + | ) + |) + |""".stripMargin + withSQLConf( + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true", + "spark.sql.planChangeLog.level" -> "info" + ) { + val df = sql(querySuperNested).collect() } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index b74013f7ab107..acfbd5eba511a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -114,11 +114,12 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // SPARK-50983: Currently nested correlations tests are using // analyzer results as results. The expected segment sizes are different // from other testcases. - "subquery/nestedcorrelation/combined-subquery.sql", - "subquery/nestedcorrelation/exists-subquery.sql", - "subquery/nestedcorrelation/lateral-subquery.sql", - "subquery/nestedcorrelation/scalar-subquery.sql", - "subquery/nestedcorrelation/subquery-not-supported.sql" + "subquery/nestedcorrelation-analyzer-only/combined-subquery.sql", + "subquery/nestedcorrelation-analyzer-only/exists-subquery.sql", + "subquery/nestedcorrelation-analyzer-only/lateral-subquery.sql", + "subquery/nestedcorrelation-analyzer-only/scalar-subquery.sql", + "subquery/nestedcorrelation-analyzer-only/subquery-not-supported.sql", + "subquery/nestedcorrelation/scalar-subquery.sql" ) override def runQueries( From 3f60249c5d7c8929f989b86a707516e88d71781b Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 28 Apr 2025 17:25:45 -0700 Subject: [PATCH 45/48] validateOuterScopeAttrs are used to check new outerScopeAttrs --- ...ctionTableSubqueryArgumentExpression.scala | 6 ++-- .../sql/catalyst/expressions/subquery.scala | 30 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index 695110223bebc..87b6f91d0dbb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -105,10 +105,10 @@ case class FunctionTableSubqueryArgumentExpression( copy() override def withNewOuterScopeAttrs( - outerScopeAttrs: Seq[Expression] + newOuterScopeAttrs: Seq[Expression] ): FunctionTableSubqueryArgumentExpression = { - validateOuterScopeAttrs() - copy(outerScopeAttrs = outerScopeAttrs) + validateOuterScopeAttrs(newOuterScopeAttrs) + copy(outerScopeAttrs = newOuterScopeAttrs) } override def toString: String = s"table-argument#${exprId.id} $conditionString" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index ab885ef175a44..2bd1af7b6825b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -97,10 +97,10 @@ abstract class SubqueryExpression( def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): SubqueryExpression - def validateOuterScopeAttrs(): Unit = { - assert(outerScopeAttrs.toSet.subsetOf(outerAttrs.toSet), + def validateOuterScopeAttrs(newOuterScopeAttrs: Seq[Expression]): Unit = { + assert(newOuterScopeAttrs.toSet.subsetOf(outerAttrs.toSet), s"outerScopeAttrs must be a subset of outerAttrs, " + - s"but got ${outerScopeAttrs.mkString(", ")}") + s"but got ${newOuterScopeAttrs.mkString(", ")}") } def getOuterScopeAttrs: Seq[Expression] = outerScopeAttrs @@ -613,10 +613,10 @@ case class ScalarSubquery( outerAttrs = outerAttrs) override def withNewOuterScopeAttrs( - outerScopeAttrs: Seq[Expression] + newOuterScopeAttrs: Seq[Expression] ): ScalarSubquery = { - validateOuterScopeAttrs() - copy(outerScopeAttrs = outerScopeAttrs) + validateOuterScopeAttrs(newOuterScopeAttrs) + copy(outerScopeAttrs = newOuterScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): ScalarSubquery = copy(hint = hint) @@ -707,10 +707,10 @@ case class LateralSubquery( outerAttrs = outerAttrs) override def withNewOuterScopeAttrs( - outerScopeAttrs: Seq[Expression] + newOuterScopeAttrs: Seq[Expression] ): LateralSubquery = { - validateOuterScopeAttrs() - copy(outerScopeAttrs = outerScopeAttrs) + validateOuterScopeAttrs(newOuterScopeAttrs) + copy(outerScopeAttrs = newOuterScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): LateralSubquery = copy(hint = hint) @@ -795,9 +795,9 @@ case class ListQuery( override def withNewOuterAttrs(outerAttrs: Seq[Expression]): ListQuery = copy( outerAttrs = outerAttrs) - override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): ListQuery = { - validateOuterScopeAttrs() - copy(outerScopeAttrs = outerScopeAttrs) + override def withNewOuterScopeAttrs(newOuterScopeAttrs: Seq[Expression]): ListQuery = { + validateOuterScopeAttrs(newOuterScopeAttrs) + copy(outerScopeAttrs = newOuterScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): ListQuery = copy(hint = hint) @@ -872,9 +872,9 @@ case class Exists( override def withNewOuterAttrs(outerAttrs: Seq[Expression]): Exists = copy( outerAttrs = outerAttrs) - override def withNewOuterScopeAttrs(outerScopeAttrs: Seq[Expression]): Exists = { - validateOuterScopeAttrs() - copy(outerScopeAttrs = outerScopeAttrs) + override def withNewOuterScopeAttrs(newOuterScopeAttrs: Seq[Expression]): Exists = { + validateOuterScopeAttrs(newOuterScopeAttrs) + copy(outerScopeAttrs = newOuterScopeAttrs) } override def withNewHint(hint: Option[HintInfo]): Exists = copy(hint = hint) From 5565bb3c68ba9f5c68b083da7bdcf115af8f4d43 Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 28 Apr 2025 17:26:17 -0700 Subject: [PATCH 46/48] Fix errors for subqueries in the having clause --- .../sql/catalyst/analysis/Analyzer.scala | 39 +++++++++++++++++-- 1 file changed, 35 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4e7e7b5599986..19f7316ed5892 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2394,6 +2394,18 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor outerReferencesInSubquery.filter( _ match { case a: AttributeReference => !p.inputSet.contains(a) + case outer: AggregateExpression => + // For resolveSubquery, we only check if the references of the aggregate expression + // can be resolved in the p.inputSet as p might be changed after resolveAggregate. + // Currently we only allow subqueries in the Having clause + // to have aggregate expressions as outer references. + // So if p does not have Aggregate or the output of Aggregate does not have + // this outer reference, UpdateOuterReference won't trigger and we throw + // UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE. + !p.exists{ + case plan: LogicalPlan if outer.references.subsetOf(plan.inputSet) => true + case _ => false + } case _ => false } ) @@ -2903,7 +2915,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor expression.transformUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) { case sub: SubqueryExpression if sub.getOuterScopeAttrs.nonEmpty => val newOuterScopeAttrs = - sub.getOuterScopeAttrs.filter( outerExpr => outerExpr match { + sub.getOuterAttrs.filter( outerExpr => outerExpr match { case a: AttributeReference => !aggregate.outputSet.contains(a) case _ => true }) @@ -4253,7 +4265,7 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { private def updateOuterReferenceInSubquery( plan: LogicalPlan, refExprs: Seq[Expression]): LogicalPlan = { - plan resolveExpressions { case e => + val newPlan = plan resolveExpressions { case e => val outerAlias = refExprs.find(stripAlias(_).semanticEquals(stripOuterReference(e))) outerAlias match { @@ -4261,12 +4273,30 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { case _ => e } } + // The above step might modify the outerAttrs + // in any SubqueryExpressions in the plan. + // We need to make sure the outerAttrs and the outerScopeAttrs are aligned and + // don't contain any outer wrappers. + newPlan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case s: SubqueryExpression if s.getOuterAttrs.exists(containsOuter) => + val newOuterScopeAttrs = s.getOuterScopeAttrs.map { e => + val outerAlias = + refExprs.find(stripAlias(_).semanticEquals(stripOuterReference(e))) + outerAlias match { + case Some(a: Alias) => a.toAttribute + case _ => e + } + } + val newOuterAttrs = s.getOuterAttrs.map(stripOuterReference) + s.withNewOuterAttrs(newOuterAttrs).withNewOuterScopeAttrs(newOuterScopeAttrs) + } } def updateOuterReferenceInAllSubqueries( s: SubqueryExpression, outerAliases: Seq[Alias]): SubqueryExpression = { + val subPlan = s.plan val planWithNestedSubqueriesRewritten = - s.plan.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION), ruleId) { + subPlan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { // Only update the nested subqueries if they have outer scope references // And we don't collect new outerAliases along s.plan because this rule // will be fired multiple times for each subquery plan in the Analyzer, @@ -4274,7 +4304,8 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { case s: SubqueryExpression if s.getOuterScopeAttrs.nonEmpty => updateOuterReferenceInAllSubqueries(s, outerAliases) } - val newPlan = updateOuterReferenceInSubquery(planWithNestedSubqueriesRewritten, outerAliases) + val newPlan = + updateOuterReferenceInSubquery(planWithNestedSubqueriesRewritten, outerAliases) s.withNewPlan(newPlan) } From 08dbc74a09ae9aed5cdffcc14f9a6eb3d3a3cc1f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Mon, 28 Apr 2025 17:26:37 -0700 Subject: [PATCH 47/48] new testcases --- .../nestedcorrelation/scalar-subquery.sql.out | 240 ++++++++---------- .../nestedcorrelation/scalar-subquery.sql | 24 +- .../nestedcorrelation/scalar-subquery.sql.out | 185 ++++---------- .../org/apache/spark/sql/SubquerySuite.scala | 53 ++++ 4 files changed, 208 insertions(+), 294 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out index cd2a759c71451..52d5e286c2add 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -184,7 +184,7 @@ Project [a#x, b#x, c#x] -- !query SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -193,26 +193,21 @@ SELECT myt1.a, ( ) FROM myt1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 42, - "stopIndex" : 159, - "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" - } ] -} +Project [a#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#x] +: +- Project [scalar-subquery#x [b#x && c#x] AS scalarsubquery(b, c)#x] +: : +- Aggregate [max(a#x) AS max(a)#x] +: : +- Filter ((b#x > outer(b#x)) AND (c#x > outer(c#x))) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -221,26 +216,21 @@ SELECT myt1.a, ( ) FROM myt1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 42, - "stopIndex" : 159, - "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" - } ] -} +Project [a#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#x] +: +- Project [scalar-subquery#x [b#x && c#x] AS scalarsubquery(b, c)#x] +: : +- Aggregate [max(a#x) AS max(a)#x] +: : +- Filter ((b#x = outer(b#x)) AND (c#x = outer(c#x))) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -249,26 +239,21 @@ SELECT myt1.a, ( ) FROM myt1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 163, - "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" - } ] -} +Project [a#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#xL] +: +- Project [scalar-subquery#x [b#x && c#x] AS scalarsubquery(b, c)#xL] +: : +- Aggregate [count(a#x) AS count(a)#xL] +: : +- Filter ((b#x > outer(b#x)) AND (c#x > outer(c#x))) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -277,21 +262,16 @@ SELECT myt1.a, ( ) FROM myt1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 163, - "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" - } ] -} +Project [a#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#xL] +: +- Project [scalar-subquery#x [b#x && c#x] AS scalarsubquery(b, c)#xL] +: : +- Aggregate [count(a#x) AS count(a)#xL] +: : +- Filter ((b#x = outer(b#x)) AND (c#x = outer(c#x))) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query @@ -303,7 +283,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1 -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException @@ -326,7 +305,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1 -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException @@ -349,7 +327,6 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1 -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException @@ -372,7 +349,6 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1 -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException @@ -387,8 +363,9 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -399,28 +376,24 @@ HAVING ( ) AND myt2.b > myt1.b ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 203, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" - } ] -} +Filter cast(scalar-subquery#x [b#x && max(a)#x] as boolean) +: +- Aggregate [max(a#x) AS max(a)#x] +: +- Filter ((a#x = scalar-subquery#x [max(a)#x]) AND (b#x > outer(b#x))) +: : +- Aggregate [max(a#x) AS max(a)#x] +: : +- Filter (a#x > outer(max(a)#x)) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -431,28 +404,24 @@ HAVING ( ) AND myt2.b = myt1.b ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 203, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" - } ] -} +Filter cast(scalar-subquery#x [b#x && max(a)#x] as boolean) +: +- Aggregate [max(a#x) AS max(a)#x] +: +- Filter ((a#x = scalar-subquery#x [max(a)#x]) AND (b#x = outer(b#x))) +: : +- Aggregate [max(a#x) AS max(a)#x] +: : +- Filter (a#x = outer(max(a)#x)) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 @@ -463,28 +432,24 @@ HAVING ( ) AND myt2.b > myt1.b ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 207, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" - } ] -} +Filter cast(scalar-subquery#x [b#x && max(a)#x] as boolean) +: +- Aggregate [count(a#x) AS count(a)#xL] +: +- Filter ((cast(a#x as bigint) = scalar-subquery#x [max(a)#x]) AND (b#x > outer(b#x))) +: : +- Aggregate [count(a#x) AS count(a)#xL] +: : +- Filter (a#x > outer(max(a)#x)) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 @@ -495,23 +460,18 @@ HAVING ( ) AND myt2.b = myt1.b ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 207, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" - } ] -} +Filter cast(scalar-subquery#x [b#x && max(a)#x] as boolean) +: +- Aggregate [count(a#x) AS count(a)#xL] +: +- Filter ((cast(a#x as bigint) = scalar-subquery#x [max(a)#x]) AND (b#x = outer(b#x))) +: : +- Aggregate [count(a#x) AS count(a)#xL] +: : +- Filter (a#x = outer(max(a)#x)) +: : +- SubqueryAlias spark_catalog.default.myt3 +: : +- Relation spark_catalog.default.myt3[a#x,b#x,c#x] parquet +: +- SubqueryAlias spark_catalog.default.myt2 +: +- Relation spark_catalog.default.myt2[a#x,b#x,c#x] parquet ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.myt1 + +- Relation spark_catalog.default.myt1[a#x,b#x,c#x] parquet -- !query diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql index 4f7ac915a091f..ff3a217b5d92e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/nestedcorrelation/scalar-subquery.sql @@ -98,7 +98,7 @@ WHERE myt1.a = ( ); -- query 5 SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -108,7 +108,7 @@ SELECT myt1.a, ( FROM myt1; -- query 6 SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -118,7 +118,7 @@ SELECT myt1.a, ( FROM myt1; -- query 7 SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -128,7 +128,7 @@ SELECT myt1.a, ( FROM myt1; -- query 8 SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -145,7 +145,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1; -- query 10 SELECT MIN( @@ -156,7 +155,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1; -- query 11 SELECT COUNT( @@ -167,7 +165,6 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1; -- query 12 SELECT COUNT( @@ -178,11 +175,11 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1; -- query 13 -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -193,8 +190,9 @@ HAVING ( ) AND myt2.b > myt1.b ); -- query 14 -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -205,8 +203,9 @@ HAVING ( ) AND myt2.b = myt1.b ); -- query 15 -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 @@ -217,8 +216,9 @@ HAVING ( ) AND myt2.b > myt1.b ); -- query 16 -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index 0c999f3b2bce7..136ffc4732162 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -165,7 +165,7 @@ struct -- !query SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -176,26 +176,16 @@ FROM myt1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkRuntimeException { - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 42, - "stopIndex" : 159, - "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" - } ] + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" } -- !query SELECT myt1.a, ( - SELECT MAX(myt2.a), ( + SELECT ( SELECT MAX(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -204,28 +194,18 @@ SELECT myt1.a, ( ) FROM myt1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 42, - "stopIndex" : 159, - "fragment" : "(\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" - } ] -} +0 0 +1 1 +2 2 +3 3 +NULL NULL -- !query SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b > myt2.b AND myt3.c > myt1.c @@ -236,26 +216,16 @@ FROM myt1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkRuntimeException { - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 163, - "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b > myt2.b AND myt3.c > myt1.c\n )" - } ] + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" } -- !query SELECT myt1.a, ( - SELECT COUNT(myt2.a), ( + SELECT ( SELECT COUNT(myt3.a) FROM myt3 WHERE myt3.b = myt2.b AND myt3.c = myt1.c @@ -264,23 +234,13 @@ SELECT myt1.a, ( ) FROM myt1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION", - "sqlState" : "0A000", - "messageParameters" : { - "sqlExpr" : "\"scalarsubquery(b, c)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 163, - "fragment" : "(\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.b = myt2.b AND myt3.c = myt1.c\n )" - } ] -} +0 1 +1 1 +2 1 +3 1 +NULL NULL -- !query @@ -292,7 +252,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1 -- !query schema struct<> @@ -317,7 +276,6 @@ SELECT MIN( ) FROM myt2 ) -) FROM myt1 -- !query schema struct<> @@ -342,7 +300,6 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1 -- !query schema struct<> @@ -367,7 +324,6 @@ SELECT COUNT( ) FROM myt2 ) -) FROM myt1 -- !query schema struct<> @@ -384,8 +340,9 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -396,30 +353,17 @@ HAVING ( ) AND myt2.b > myt1.b ) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 203, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" - } ] -} +0 0 +1 1 +2 2 -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT MAX(myt2.a) FROM myt2 @@ -430,30 +374,17 @@ HAVING ( ) AND myt2.b = myt1.b ) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 203, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT MAX(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT MAX(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" - } ] -} +1 1 +2 2 +3 3 -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 @@ -464,30 +395,16 @@ HAVING ( ) AND myt2.b > myt1.b ) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 207, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a > MAX(myt1.a)\n ) AND myt2.b > myt1.b\n)" - } ] -} +0 0 +1 1 -- !query -SELECT MAX(myt1.a) +SELECT b, MAX(myt1.a) FROM myt1 +GROUP BY b HAVING ( SELECT COUNT(myt2.a) FROM myt2 @@ -498,25 +415,9 @@ HAVING ( ) AND myt2.b = myt1.b ) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", - "sqlState" : "XX000", - "messageParameters" : { - "input" : "\"max(a)\"", - "missingAttributes" : "\"b\"", - "operator" : "Filter cast(scalar-subquery#x [b#x] as boolean)" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 207, - "fragment" : "SELECT MAX(myt1.a)\nFROM myt1\nHAVING (\n SELECT COUNT(myt2.a)\n FROM myt2\n WHERE myt2.a = (\n SELECT COUNT(myt3.a)\n FROM myt3\n WHERE myt3.a = MAX(myt1.a)\n ) AND myt2.b = myt1.b\n)" - } ] -} +1 1 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 196460943fad3..8108b8a28f038 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2952,4 +2952,57 @@ class SubquerySuite extends QueryTest val df = sql(querySuperNested).collect() } } + + test("avery test") { + val query = + """ + |SELECT b, MAX(t1.a) + |FROM t1 + |GROUP BY b + |HAVING ( + | SELECT MAX(t2.a) + | FROM t2 + | WHERE t2.a = ( + | SELECT MAX(t3.a) + | FROM t3 + | WHERE t3.a > MAX(t1.a) + | ) AND t2.b > t1.b + |); + |""".stripMargin + sql("CREATE TABLE IF NOT EXISTS t1(a INT, b INT, c INT);") + sql("CREATE TABLE IF NOT EXISTS t2(a INT, b INT, c INT);") + sql("CREATE TABLE IF NOT EXISTS t3(a INT, b INT, c INT);") + sql("INSERT INTO t1 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL);") + sql("INSERT INTO t2 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL);") + sql("INSERT INTO t3 VALUES (0, 0, 0), (1, 1, 1), (2, 2, 2), (3, 3, 3), (NULL, NULL, NULL);") + withTable("t1", "t2", "t3") { + withSQLConf( + "spark.sql.planChangeLog.level" -> "info", + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true" + ) { + val df = sql(query).collect() + } + } + } + + test("postgresql test") { + val query = + """ + |select ten, sum(distinct four) filter (where four > 10) from onek a + |group by ten + |having exists (select 1 from onek b where sum(distinct a.four) = b.four); + |""".stripMargin + sql("create table if not exists onek(ten int, four int);") + sql("insert into onek values (1, 5), (1, 15), (2, 20), (2, 25), (3, 30);") + withTable("onek") { + withSQLConf( + "spark.sql.planChangeLog.level" -> "info", + "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true", + "spark.sql.optimizer.supportNestedCorrelatedSubqueriesForScalarSubqueries.enabled" -> "true" + ) { + val df = sql(query).collect() + } + } + } } From c7fe834cd4d870a24822f518a9868cd7c324979f Mon Sep 17 00:00:00 2001 From: Avery Qi Date: Tue, 29 Apr 2025 12:08:17 -0700 Subject: [PATCH 48/48] fix OptimizeOneRowRelationSubquery --- .../sql/catalyst/optimizer/subquery.scala | 73 ++++++++++++++++++- .../nestedcorrelation/scalar-subquery.sql.out | 28 +++---- .../org/apache/spark/sql/SubquerySuite.scala | 11 ++- 3 files changed, 85 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 4bd8e0e2f086b..5c49fdaa8cf16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -1008,11 +1008,64 @@ object RewriteLateralSubquery extends Rule[LogicalPlan] { } } +/** + * Recalculate outerAttrs and outerScopeAttrs in SubqueryExpressions. + */ +object RecalculateOuterAttrsAndOuterScopeAttrs extends Rule[LogicalPlan] { + /** + * Returns the outer scope attributes referenced in the subquery expressions + * in current plan and the children of the current plan. + */ + private def getOuterAttrsNeedToBePropagated(plan: LogicalPlan): Seq[Expression] = { + plan.expressions.flatMap { + case subExpr: SubqueryExpression => subExpr.getOuterScopeAttrs + case in: InSubquery => in.query.getOuterScopeAttrs + case expr if expr.containsPattern(PLAN_EXPRESSION) => + expr.collect { + case subExpr: SubqueryExpression => subExpr.getOuterScopeAttrs + }.flatten + case _ => Seq.empty + } ++ plan.children.flatMap{ + case p if p.containsPattern(PLAN_EXPRESSION) => + getOuterAttrsNeedToBePropagated(p) + case _ => Seq.empty + } + } + + private def getNestedOuterReferences( + outerAttrs: Seq[Expression], p: LogicalPlan + ): Seq[Expression] = { + outerAttrs.filter { + _ match { + case a: AttributeReference => !p.inputSet.contains(a) + case n: NamedExpression => !p.inputSet.contains(n.toAttribute) + case _ => false + } + } + } + + def apply0(plan: LogicalPlan): LogicalPlan = plan.transformExpressions { + case s: SubqueryExpression if s.children.nonEmpty && s.getJoinCond.isEmpty => + val newSubPlan = apply(s.plan) + val allOuterAttrs = getOuterReferences(newSubPlan) ++ + getOuterAttrsNeedToBePropagated(newSubPlan) + val nestedOuterAttrs = getNestedOuterReferences(allOuterAttrs, plan) + s.withNewOuterAttrs(allOuterAttrs).withNewOuterScopeAttrs(nestedOuterAttrs) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case p: LogicalPlan if p.expressions.exists(SubqueryExpression.hasCorrelatedSubquery) => + apply0(p) + } +} + /** * This rule optimizes subqueries with OneRowRelation as leaf nodes. */ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { + var needToRecalculateOuterScopeAttrs = false + object OneRowSubquery { def unapply(plan: LogicalPlan): Option[UnaryNode] = { // SPARK-40800: always inline expressions to support a broader range of correlated @@ -1060,10 +1113,18 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { case p: LogicalPlan => p.transformExpressionsUpWithPruning( _.containsPattern(SCALAR_SUBQUERY)) { - case s @ ScalarSubquery(OneRowSubquery(p @ Project(_, _: OneRowRelation)), _, _, _, _, _, _, _) + case s @ ScalarSubquery(OneRowSubquery(p @ Project(_, _: OneRowRelation)), outerAttrs, outerScopeAttrs, _, _, _, _, _) if !hasCorrelatedSubquery(s.plan) && s.joinCond.isEmpty => assert(p.projectList.size == 1) - stripOuterReferences(p.projectList).head + needToRecalculateOuterScopeAttrs = true + val originalOutput = p.projectList.head + // If the outer reference is a outerScopeAttr, even if current subquery + // is eliminated to one or multiple expressions, we can't strip its outer references. + // After the rule is applied, the outerAttrs and the outerScopeAttrs need to be reevaluated. + originalOutput.transform { + case OuterReference(a) if !outerScopeAttrs.contains(a) => + a + } } } @@ -1071,7 +1132,13 @@ object OptimizeOneRowRelationSubquery extends Rule[LogicalPlan] { if (!conf.getConf(SQLConf.OPTIMIZE_ONE_ROW_RELATION_SUBQUERY)) { plan } else { - rewrite(plan) + needToRecalculateOuterScopeAttrs = false + val newPlan = rewrite(plan) + if (needToRecalculateOuterScopeAttrs) { + RecalculateOuterAttrsAndOuterScopeAttrs(newPlan) + } else { + newPlan + } } } } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out index 136ffc4732162..08cc38763e9bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/nestedcorrelation/scalar-subquery.sql.out @@ -594,31 +594,23 @@ NULL NULL -- !query SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkException -{ - "errorClass" : "INTERNAL_ERROR", - "sqlState" : "XX000", - "messageParameters" : { - "message" : "Couldn't find i#x in []" - } -} +NULL NULL +1 86 +2 88 +3 90 -- !query SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1 LEFT OUTER JOIN (SELECT (SELECT 42+i1.i) AS k) s2 ON s1.k=s2.k) AS j FROM table_integers i1 ORDER BY i -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkException -{ - "errorClass" : "INTERNAL_ERROR", - "sqlState" : "XX000", - "messageParameters" : { - "message" : "Couldn't find i#x in []" - } -} +NULL NULL +1 86 +2 88 +3 90 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 8108b8a28f038..760e076e24973 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2989,13 +2989,12 @@ class SubquerySuite extends QueryTest test("postgresql test") { val query = """ - |select ten, sum(distinct four) filter (where four > 10) from onek a - |group by ten - |having exists (select 1 from onek b where sum(distinct a.four) = b.four); + |SELECT i, (SELECT s1.k+s2.k FROM (SELECT (SELECT 42+i1.i) AS k) s1, + | (SELECT (SELECT 42+i1.i) AS k) s2) AS j FROM table_integers i1 ORDER BY i; |""".stripMargin - sql("create table if not exists onek(ten int, four int);") - sql("insert into onek values (1, 5), (1, 15), (2, 20), (2, 25), (3, 30);") - withTable("onek") { + sql("CREATE TABLE table_integers(i INTEGER);") + sql("INSERT INTO table_integers VALUES (1), (2), (3), (NULL);") + withTable("table_integers") { withSQLConf( "spark.sql.planChangeLog.level" -> "info", "spark.sql.optimizer.supportNestedCorrelatedSubqueries.enabled" -> "true",