diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index cddb08487652c..e5516ed92d267 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1443,6 +1443,11 @@ "A correlated outer name reference within a subquery expression body was not found in the enclosing query: " ] }, + "CORRELATED_REFERENCE" : { + "message" : [ + "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: " + ] + }, "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC" : { "message" : [ "Lateral join condition cannot be non-deterministic: " @@ -1468,11 +1473,6 @@ "Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row" ] }, - "UNSUPPORTED_CORRELATED_REFERENCE" : { - "message" : [ - "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses" - ] - }, "UNSUPPORTED_CORRELATED_REFERENCE_DATA_TYPE" : { "message" : [ "Correlated column reference '' cannot be type" 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 12dac5c632a3b..fac82188a1282 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 @@ -1066,11 +1066,12 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // 2. Expressions containing outer references on plan nodes other than allowed operators. def failOnInvalidOuterReference(p: LogicalPlan): Unit = { p.expressions.foreach(checkMixedReferencesInsideAggregateExpr) - if (!canHostOuter(p) && p.expressions.exists(containsOuter)) { + val exprs = stripOuterReferences(p.expressions.filter(expr => containsOuter(expr))) + if (!canHostOuter(p) && !exprs.isEmpty) { p.failAnalysis( errorClass = - "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_REFERENCE", - messageParameters = Map("treeNode" -> planToString(p))) + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + messageParameters = Map("sqlExprs" -> exprs.map(toSQLExpr).mkString(","))) } } 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 577f663d8b1e5..7b99153acf9a8 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 @@ -51,11 +51,14 @@ class ResolveSubquerySuite extends AnalysisTest { test("SPARK-17251 Improve `OuterReference` to be `NamedExpression`") { val expr = Filter( InSubquery(Seq(a), ListQuery(Project(Seq(UnresolvedAttribute("a")), t2))), t1) - val m = intercept[AnalysisException] { - SimpleAnalyzer.checkAnalysis(SimpleAnalyzer.ResolveSubquery(expr)) - }.getMessage - assert(m.contains( - "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses")) + checkError( + exception = intercept[AnalysisException] { + SimpleAnalyzer.checkAnalysis(SimpleAnalyzer.ResolveSubquery(expr)) + }, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + parameters = Map("sqlExprs" -> "\"a\""), + matchPVals = true + ) } test("SPARK-29145 Support subquery in join condition") { 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 cc4349e05f228..26d402479de07 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 @@ -155,9 +155,9 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_REFERENCE", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "messageParameters" : { - "treeNode" : "Aggregate [min(outer(t2a#x)) AS min(outer(t2.t2a))#x]\n+- SubqueryAlias t3\n +- View (`t3`, [t3a#x,t3b#x,t3c#x])\n +- Project [cast(t3a#x as int) AS t3a#x, cast(t3b#x as int) AS t3b#x, cast(t3c#x as int) AS t3c#x]\n +- Project [t3a#x, t3b#x, t3c#x]\n +- SubqueryAlias t3\n +- LocalRelation [t3a#x, t3b#x, t3c#x]\n" + "sqlExprs" : "\"min(t2a) AS `min(outer(t2.t2a))`\"" }, "queryContext" : [ { "objectType" : "", 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 fe65e282c7712..65dd911df314e 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 @@ -964,17 +964,15 @@ class SubquerySuite extends QueryTest | WHERE t1.c1 = t2.c1) """.stripMargin) } - checkErrorMatchPVals( + checkError( exception1, - errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_REFERENCE", - parameters = Map("treeNode" -> "(?s).*"), - sqlState = None, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + parameters = Map("sqlExprs" -> "\"explode(arr_c2)\""), context = ExpectedContext( fragment = "LATERAL VIEW explode(t2.arr_c2) q AS c2", start = 68, - stop = 106)) - assert(exception1.getMessage.contains( - "Expressions referencing the outer query are not supported outside of WHERE/HAVING")) + stop = 106) + ) } }