From 090b89b1542ab6f0389e762c7ef3432a8d281c0e Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 8 Sep 2022 16:33:22 -0700 Subject: [PATCH 01/19] initial commit --- .../main/resources/error/error-classes.json | 77 ++++++++++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 110 +++++++++++++----- .../sql/catalyst/expressions/subquery.scala | 2 +- .../sql/errors/QueryCompilationErrors.scala | 12 +- .../sql-tests/results/join-lateral.sql.out | 29 ++--- .../postgreSQL/aggregates_part1.sql.out | 8 +- .../invalid-correlation.sql.out | 16 ++- .../org/apache/spark/sql/SubquerySuite.scala | 45 +++++-- 8 files changed, 235 insertions(+), 64 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3e15334931bd1..1c67759056ff0 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -327,6 +327,83 @@ ], "sqlState" : "42000" }, + "INVALID_SUBQUERY_EXPRESSION" : { + "message" : [ + "Invalid subquery expression" + ], + "subClass" : { + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { + "message" : [ + "Accessing outer query column is not allowed in this location" + ] + }, + "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES" : { + "message" : [ + "Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: " + ] + }, + "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE" : { + "message" : [ + "Correlated column is not allowed in predicate " + ] + }, + "CORRELATED_SCALAR_SUBQUERIES_IN_GROUP_BY_MUST_BE_IN_AGGREGATE_EXPRESSIONS" : { + "message" : [ + "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" + ] + }, + "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT" : { + "message" : [ + "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" + ] + }, + "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING" : { + "message" : [ + "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses" + ] + }, + "IN_EXISTS_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT" : { + "message" : [ + "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" + ] + }, + "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC" : { + "message" : [ + "Lateral join condition cannot be non-deterministic: " + ] + }, + "MORE_THAN_ONE_OUTPUT_COLUMN" : { + "message" : [ + "Scalar subquery must return only one column, but got " + ] + }, + "MUST_AGGREGATE_CORRELATED_SUBQUERY" : { + "message" : [ + "Correlated scalar subqueries must be aggregated" + ] + }, + "MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT" : { + "message" : [ + "The output of a correlated scalar subquery must be aggregated" + ] + }, + "NON_CORRELATED_COLUMNS_IN_GROUP_BY" : { + "message" : [ + "A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns: " + ] + }, + "NON_DETERMINISTIC_LATERAL_SUBQUERIES" : { + "message" : [ + "Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row" + ] + }, + "OUTER_ATTRIBUTE_NOT_FOUND" : { + "message" : [ + "Outer attribute not found: " + ] + } + } + }, "MISSING_STATIC_PARTITION_COLUMN" : { "message" : [ "Unknown static partition column: " 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 68ed8991553e3..45e2e3efb504c 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery, InlineCTE} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.catalyst.trees.{Origin, TreeNodeTag} import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_WINDOW_EXPRESSION import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} @@ -55,6 +55,17 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { throw new AnalysisException(msg) } + protected def failAnalysisForSubqueryExpression( + errorSubClass: String, + origin: Origin, + messageParameters: Array[String] = Array.empty[String]): Nothing = { + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = errorSubClass, + origin = origin, + messageParameters = messageParameters) + } + protected def containsMultipleGenerators(exprs: Seq[Expression]): Boolean = { exprs.flatMap(_.collect { case e: Generator => e @@ -743,7 +754,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case a: AggregateExpression => a }) if (aggregates.isEmpty) { - failAnalysis("The output of a correlated scalar subquery must be aggregated") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT", + origin = expr.origin, + messageParameters = Array.empty[String]) } // SPARK-18504/SPARK-18814: Block cases where GROUP BY columns @@ -756,10 +771,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { val invalidCols = groupByCols -- correlatedCols // GROUP BY columns must be a subset of columns in the predicates if (invalidCols.nonEmpty) { - failAnalysis( - "A GROUP BY clause in a scalar correlated subquery " + - "cannot contain non-correlated columns: " + - invalidCols.mkString(",")) + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "NON_CORRELATED_COLUMNS_IN_GROUP_BY", + origin = expr.origin, + messageParameters = Array(invalidCols.map(_.name).mkString(","))) } } @@ -784,7 +800,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case o: OuterReference => p.children.foreach(e => if (!e.output.exists(_.exprId == o.exprId)) { - failAnalysis("outer attribute not found") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "OUTER_ATTRIBUTE_NOT_FOUND", + origin = o.origin, + messageParameters = Array(o.name)) }) case _ => }) @@ -802,8 +822,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case ScalarSubquery(query, outerAttrs, _, _) => // Scalar subquery must return one column as output. if (query.output.size != 1) { - failAnalysis( - s"Scalar subquery must return only one column, but got ${query.output.size}") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "MORE_THAN_ONE_OUTPUT_COLUMN", + origin = expr.origin, + messageParameters = Array(query.output.size.toString)) } if (outerAttrs.nonEmpty) { @@ -811,7 +834,12 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case a: Aggregate => checkAggregateInScalarSubquery(outerAttrs, query, a) case Filter(_, a: Aggregate) => checkAggregateInScalarSubquery(outerAttrs, query, a) case p: LogicalPlan if p.maxRows.exists(_ <= 1) => // Ok - case fail => failAnalysis(s"Correlated scalar subqueries must be aggregated: $fail") + case _ => + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SUBQUERY", + origin = expr.origin, + messageParameters = Array.empty[String]) } // Only certain operators are allowed to host subquery expression containing @@ -823,12 +851,19 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // it must also be in the aggregate expressions to be rewritten in the optimization // phase. if (containsExpr(a.groupingExpressions) && !containsExpr(a.aggregateExpressions)) { - failAnalysis("Correlated scalar subqueries in the group by clause " + - s"must also be in the aggregate expressions:\n$a") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = + "CORRELATED_SCALAR_SUBQUERIES_IN_GROUP_BY_MUST_BE_IN_AGGREGATE_EXPRESSIONS", + origin = a.origin, + messageParameters = Array.empty[String]) } - case other => failAnalysis( - "Correlated scalar sub-queries can only be used in a " + - s"Filter/Aggregate/Project and a few commands: $plan") + case other => + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT", + origin = other.origin, + messageParameters = Array.empty[String]) } } // Validate to make sure the correlations appearing in the query are valid and @@ -841,14 +876,19 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // A lateral join with a multi-row outer query and a non-deterministic lateral subquery // cannot be decorrelated. Otherwise it may produce incorrect results. if (!expr.deterministic && !join.left.maxRows.exists(_ <= 1)) { - expr.failAnalysis( - s"Non-deterministic lateral subqueries are not supported when joining with " + - s"outer relations that produce more than one row\n${expr.plan}") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "NON_DETERMINISTIC_LATERAL_SUBQUERIES", + origin = expr.origin, + messageParameters = Array.empty[String]) } // Check if the lateral join's join condition is deterministic. if (join.condition.exists(!_.deterministic)) { - join.failAnalysis( - s"Lateral join condition cannot be non-deterministic: ${join.condition.get.sql}") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", + origin = join.origin, + messageParameters = Array(join.condition.get.sql)) } // Validate to make sure the correlations appearing in the query are valid and // allowed by spark. @@ -859,8 +899,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _: Filter | _: SupportsSubquery | _: Join | _: Project | _: Aggregate | _: Window => // Ok case _ => - failAnalysis(s"IN/EXISTS predicate sub-queries can only be used in" + - s" Filter/Join/Project/Aggregate/Window and a few commands: $plan") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT", + origin = expr.origin, + messageParameters = Array.empty[String]) } // Validate to make sure the correlations appearing in the query are valid and // allowed by spark. @@ -914,7 +957,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.foreach { case a: AggregateExpression if containsOuter(a) => if (a.references.nonEmpty) { - throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql) + throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql, a.origin) } case _ => } @@ -923,7 +966,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // Make sure a plan's subtree does not contain outer references def failOnOuterReferenceInSubTree(p: LogicalPlan): Unit = { if (hasOuterReferences(p)) { - failAnalysis(s"Accessing outer query column is not allowed in:\n$p") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + origin = p.origin, + messageParameters = Array.empty[String]) } } @@ -942,9 +989,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def failOnInvalidOuterReference(p: LogicalPlan): Unit = { p.expressions.foreach(checkMixedReferencesInsideAggregateExpr) if (!canHostOuter(p) && p.expressions.exists(containsOuter)) { - failAnalysis( - "Expressions referencing the outer query are not supported outside of WHERE/HAVING " + - s"clauses:\n$p") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING", + origin = p.origin, + messageParameters = Array.empty[String]) } } @@ -1007,8 +1056,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def failOnUnsupportedCorrelatedPredicate(predicates: Seq[Expression], p: LogicalPlan): Unit = { if (predicates.nonEmpty) { // Report a non-supported case as an exception - failAnalysis("Correlated column is not allowed in predicate " + - s"${predicates.map(_.sql).mkString}:\n$p") + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + origin = p.origin, + messageParameters = Array(s"${predicates.map(_.sql).mkString}")) } } 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 71b36fa8ef9ba..fae196307b2fa 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 @@ -206,7 +206,7 @@ object SubExprUtils extends PredicateHelper { def collectOutRefs(input: Expression): Unit = input match { case a: AggregateExpression if containsOuter(a) => if (a.references.nonEmpty) { - throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql) + throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql, a.origin) } else { // Collect and update the sub-tree so that outer references inside this aggregate // expression will not be collected. For example: min(outer(a)) -> min(a). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index b62ae2a5900c9..1e26c3d4a40a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1543,14 +1543,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException(s"'$operation' does not support partitioning") } - def mixedRefsInAggFunc(funcStr: String): Throwable = { - val msg = "Found an aggregate function in a correlated predicate that has both " + - "outer and local references, which is not supported: " + funcStr - new AnalysisException(msg) + def mixedRefsInAggFunc(funcStr: String, origin: Origin): Throwable = { + throw new AnalysisException( + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + origin = origin, + messageParameters = Array(funcStr)) } def functionCannotProcessInputError( - unbound: UnboundFunction, + unbound: UnboundFunction, arguments: Seq[Expression], unsupported: UnsupportedOperationException): Throwable = { new AnalysisException(s"Function '${unbound.name}' cannot process " + 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 5180df6fcdee5..531bdce1d7422 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 @@ -310,11 +310,10 @@ SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3) struct<> -- !query output org.apache.spark.sql.AnalysisException -Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row -SubqueryAlias __auto_generated_subquery_name -+- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(0)) AS c3#x] - +- OneRowRelation -; line 1 pos 9 +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES" +} -- !query @@ -323,14 +322,10 @@ SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2) struct<> -- !query output org.apache.spark.sql.AnalysisException -Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row -SubqueryAlias __auto_generated_subquery_name -+- Project [rand(0) AS rand(0)#x] - +- SubqueryAlias spark_catalog.default.t2 - +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) - +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] - +- LocalRelation [col1#x, col2#x] -; line 1 pos 9 +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES" +} -- !query @@ -339,7 +334,13 @@ SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Lateral join condition cannot be non-deterministic: ((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE)); line 1 pos 17 +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", + "messageParameters" : { + "condition" : "((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE))" + } +} -- !query 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 1a6f38699844f..a2ad5259c1848 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 @@ -484,7 +484,13 @@ having exists (select 1 from onek b struct<> -- !query output org.apache.spark.sql.AnalysisException -Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: sum(DISTINCT (outer(a.four) + b.four)) +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + } +} -- !query 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 0b7581afc25aa..743094b4b083e 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 @@ -75,7 +75,13 @@ HAVING EXISTS (SELECT t2a struct<> -- !query output org.apache.spark.sql.AnalysisException -Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: min((outer(t1.t1a) + t2.t2a)) +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "messageParameters" : { + "function" : "min((outer(t1.t1a) + t2.t2a))" + } +} -- !query @@ -91,7 +97,13 @@ WHERE t1a IN (SELECT t2a struct<> -- !query output org.apache.spark.sql.AnalysisException -Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: min((outer(t2.t2a) + t3.t3a)) +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "messageParameters" : { + "function" : "min((outer(t2.t2a) + t3.t3a))" + } +} -- !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 a0ccfb10ca8ac..68b5189c66496 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 @@ -523,8 +523,11 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark val errMsg = intercept[AnalysisException] { sql("select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1") } - assert(errMsg.getMessage.contains( - "A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns:")) + checkError( + errMsg, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("NON_CORRELATED_COLUMNS_IN_GROUP_BY"), + parameters = Map("value" -> "c2")) } } @@ -532,21 +535,28 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark val msg1 = intercept[AnalysisException] { sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") } - assert(msg1.getMessage.contains("Correlated scalar subqueries must be aggregated")) - + checkError( + msg1, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SUBQUERY")) val msg2 = intercept[AnalysisException] { sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") } - assert(msg2.getMessage.contains( - "The output of a correlated scalar subquery must be aggregated")) + checkError( + msg2, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT")) } test("non-equal correlated scalar subquery") { val msg1 = intercept[AnalysisException] { sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") } - assert(msg1.getMessage.contains( - "Correlated column is not allowed in predicate (l2.a < outer(l1.a))")) + checkError( + msg1, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), + parameters = Map("predicate" -> "(l2.a < outer(l1.a))")) } test("disjunctive correlated scalar subquery") { @@ -1983,9 +1993,14 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark withTempView("t1", "t2") { Seq((0, 1)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 2), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") - assert(intercept[AnalysisException] { + val msg1 = intercept[AnalysisException] { sql("SELECT * FROM t1 JOIN LATERAL (SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1)") - }.getMessage.contains("Correlated column is not allowed in predicate")) + } + checkError( + msg1, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), + parameters = Map("predicate" -> "(t2.c1 > outer(t1.c1))")) } } @@ -2005,13 +2020,19 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark |FROM (SELECT CAST(c1 AS STRING) a FROM t1) |""".stripMargin), Row(5) :: Row(null) :: Nil) - assert(intercept[AnalysisException] { + val msg1 = intercept[AnalysisException] { sql( """ |SELECT (SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a) |FROM (SELECT CAST(c1 AS SHORT) a FROM t1) |""".stripMargin) - }.getMessage.contains("Correlated column is not allowed in predicate")) + } + checkError( + msg1, + errorClass = "INVALID_SUBQUERY_EXPRESSION", + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), + parameters = Map( + "predicate" -> "(CAST(t2.c1 AS SMALLINT) = outer(__auto_generated_subquery_name.a))")) } } From e52e6565723d02486956094b5b4d17faa39f8acb Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 8 Sep 2022 16:40:30 -0700 Subject: [PATCH 02/19] initial commit --- .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 1e26c3d4a40a6..3eedac2352ce8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1552,7 +1552,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def functionCannotProcessInputError( - unbound: UnboundFunction, + unbound: UnboundFunction, arguments: Seq[Expression], unsupported: UnsupportedOperationException): Throwable = { new AnalysisException(s"Function '${unbound.name}' cannot process " + From 39fe505ee273cd1fcc5a26cb80c365971e06a277 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 8 Sep 2022 16:54:21 -0700 Subject: [PATCH 03/19] update tests --- .../describe-part-after-analyze.sql.out | 191 ++------ .../sql-tests/results/describe.sql.out | 429 ++++-------------- .../sql-tests/results/explain-aqe.sql.out | 13 +- .../sql-tests/results/explain.sql.out | 13 +- 4 files changed, 150 insertions(+), 496 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out index b04ad638dd2c0..52b7bf1cebb5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -5,7 +5,8 @@ CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. -- !query @@ -14,7 +15,8 @@ VALUES ('k1', 100), ('k2', 200), ('k3', 300) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 18 -- !query @@ -23,7 +25,8 @@ VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 18 -- !query @@ -32,33 +35,17 @@ VALUES ('k1', 102), ('k2', 202) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 18 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 -Created Time [not included in comparison] -Last Access [not included in comparison] - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -66,34 +53,17 @@ ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or permanent view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 3 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -101,61 +71,26 @@ ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or permanent view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 3 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 4 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -163,88 +98,35 @@ ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or permanent view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 3 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 4 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) -- !query schema -struct +struct<> -- !query output -key string -value string -ds string -hr int -# Partition Information -# col_name data_type comment -ds string -hr int - -# Detailed Partition Information -Database default -Table t -Partition Values [ds=2017-09-01, hr=5] -Location [not included in comparison]/{warehouse_dir}/t/ds=2017-09-01/hr=5 -Created Time [not included in comparison] -Last Access [not included in comparison] -Partition Statistics [not included in comparison] bytes, 2 rows - -# Storage Information -Location [not included in comparison]/{warehouse_dir}/t +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -252,4 +134,5 @@ DROP TABLE t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 11 diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 45df378495ccb..f41610b348ada 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -8,7 +8,8 @@ CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. -- !query @@ -16,7 +17,8 @@ CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 46 -- !query @@ -37,7 +39,8 @@ CREATE VIEW v AS SELECT * FROM t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 31 -- !query @@ -45,7 +48,8 @@ ALTER TABLE t SET TBLPROPERTIES (e = '3') -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 12 -- !query @@ -53,118 +57,53 @@ ALTER TABLE t ADD PARTITION (c='Us', d=1) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 12 -- !query DESCRIBE t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 9 -- !query DESC default.t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: default.t; line 1 pos 5 -- !query DESC TABLE t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 11 -- !query DESC FORMATTED t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table t -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Comment table_comment -Table Properties [e=3, password=*********(redacted), t=test] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] -Partition Provider Catalog +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 15 -- !query DESC EXTENDED t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table t -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Comment table_comment -Table Properties [e=3, password=*********(redacted), t=test] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] -Partition Provider Catalog +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -172,40 +111,17 @@ ALTER TABLE t UNSET TBLPROPERTIES (e) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 12 -- !query DESC EXTENDED t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table t -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Comment table_comment -Table Properties [password=*********(redacted), t=test] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] -Partition Provider Catalog +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -213,147 +129,53 @@ ALTER TABLE t UNSET TBLPROPERTIES (comment) -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table not found: t; line 1 pos 12 -- !query DESC EXTENDED t -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table t -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Table Properties [password=*********(redacted), t=test] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] -Partition Provider Catalog +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query DESC t PARTITION (c='Us', d=1) -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 5 -- !query DESC EXTENDED t PARTITION (c='Us', d=1) -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Partition Information -Database default -Table t -Partition Values [c=Us, d=1] -Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 -Storage Properties [a=1, b=2, password=*********(redacted)] -Created Time [not included in comparison] -Last Access [not included in comparison] - -# Storage Information -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query DESC FORMATTED t PARTITION (c='Us', d=1) -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Partition Information -Database default -Table t -Partition Values [c=Us, d=1] -Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 -Storage Properties [a=1, b=2, password=*********(redacted)] -Created Time [not included in comparison] -Last Access [not included in comparison] - -# Storage Information -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 15 -- !query DESC EXTENDED t PARTITION (C='Us', D=1) -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string -# Partition Information -# col_name data_type comment -c string -d string - -# Detailed Partition Information -Database default -Table t -Partition Values [c=Us, d=1] -Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 -Storage Properties [a=1, b=2, password=*********(redacted)] -Created Time [not included in comparison] -Last Access [not included in comparison] - -# Storage Information -Num Buckets 2 -Bucket Columns [`a`] -Sort Columns [`b`] -Location [not included in comparison]/{warehouse_dir}/t -Storage Properties [a=1, b=2, password=*********(redacted)] +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 14 -- !query @@ -361,10 +183,8 @@ DESC t PARTITION (c='Us', d=2) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException -Partition not found in table 't' database 'default': -c -> Us -d -> 2 +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 5 -- !query @@ -373,7 +193,7 @@ DESC t PARTITION (c='Us') struct<> -- !query output org.apache.spark.sql.AnalysisException -Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`spark_catalog`.`default`.`t`' +Table or view not found: t; line 1 pos 5 -- !query @@ -401,45 +221,37 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query DESC temp_v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: temp_v; line 1 pos 5 -- !query DESC TABLE temp_v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: temp_v; line 1 pos 11 -- !query DESC FORMATTED temp_v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: temp_v; line 1 pos 15 -- !query DESC EXTENDED temp_v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: temp_v; line 1 pos 14 -- !query @@ -471,84 +283,43 @@ DESC temp_v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -{ - "errorClass" : "FORBIDDEN_OPERATION", - "messageParameters" : { - "statement" : "DESC PARTITION", - "objectType" : "TEMPORARY VIEW", - "objectName" : "`temp_v`" - } -} +Table or view not found: temp_v; line 1 pos 5 -- !query DESC v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: v; line 1 pos 5 -- !query DESC TABLE v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string +org.apache.spark.sql.AnalysisException +Table or view not found: v; line 1 pos 11 -- !query DESC FORMATTED v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table v -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type VIEW -View Text SELECT * FROM t -View Original Text SELECT * FROM t -View Catalog and Namespace spark_catalog.default -View Query Output Columns [a, b, c, d] +org.apache.spark.sql.AnalysisException +Table or view not found: v; line 1 pos 15 -- !query DESC EXTENDED v -- !query schema -struct +struct<> -- !query output -a string -b int -c string -d string - -# Detailed Table Information -Catalog spark_catalog -Database default -Table v -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type VIEW -View Text SELECT * FROM t -View Original Text SELECT * FROM t -View Catalog and Namespace spark_catalog.default -View Query Output Columns [a, b, c, d] +org.apache.spark.sql.AnalysisException +Table or view not found: v; line 1 pos 14 -- !query @@ -557,14 +328,7 @@ DESC v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -{ - "errorClass" : "FORBIDDEN_OPERATION", - "messageParameters" : { - "statement" : "DESC PARTITION", - "objectType" : "VIEW", - "objectName" : "`v`" - } -} +Table or view not found: v; line 1 pos 5 -- !query @@ -573,8 +337,9 @@ EXPLAIN DESC t struct -- !query output == Physical Plan == -Execute DescribeTableCommand - +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 13; +'DescribeRelation false, [col_name#x, data_type#x, comment#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true -- !query @@ -583,8 +348,9 @@ EXPLAIN DESC EXTENDED t struct -- !query output == Physical Plan == -Execute DescribeTableCommand - +- DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; +'DescribeRelation true, [col_name#x, data_type#x, comment#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true -- !query @@ -597,15 +363,9 @@ struct +- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true == Analyzed Logical Plan == -col_name: string, data_type: string, comment: string -DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] - -== Optimized Logical Plan == -DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] - -== Physical Plan == -Execute DescribeTableCommand - +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; +'DescribeRelation false, [col_name#x, data_type#x, comment#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true -- !query @@ -614,8 +374,9 @@ EXPLAIN DESCRIBE t b struct -- !query output == Physical Plan == -Execute DescribeColumnCommand - +- DescribeColumnCommand `spark_catalog`.`default`.`t`, [spark_catalog, default, t, b], false, [info_name#x, info_value#x] +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 17; +'DescribeColumn 'b, false, [info_name#x, info_value#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true -- !query @@ -624,8 +385,9 @@ EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) struct -- !query output == Physical Plan == -Execute DescribeTableCommand - +- DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 17; +'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true -- !query @@ -633,7 +395,8 @@ DROP TABLE t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 11 -- !query @@ -641,7 +404,8 @@ DROP VIEW temp_v -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +View not found: temp_v; line 1 pos 10 -- !query @@ -657,4 +421,5 @@ DROP VIEW v -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +View not found: v; line 1 pos 10 diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 7e237d1267897..faf85670c0b3d 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -1104,7 +1104,8 @@ CREATE table t(v array) USING PARQUET -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. -- !query @@ -1113,9 +1114,10 @@ EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) struct -- !query output == Physical Plan == -*Filter v#x IN ([a],null) -+- *ColumnarToRow - +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; +'Project [*] ++- 'Filter 'v IN (array(a),null) + +- 'UnresolvedRelation [t], [], false -- !query @@ -1123,4 +1125,5 @@ DROP TABLE t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 11 diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index cdfeba62e7238..32dfa4150fb88 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -1046,7 +1046,8 @@ CREATE table t(v array) USING PARQUET -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. -- !query @@ -1055,9 +1056,10 @@ EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) struct -- !query output == Physical Plan == -*Filter v#x IN ([a],null) -+- *ColumnarToRow - +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> +org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; +'Project [*] ++- 'Filter 'v IN (array(a),null) + +- 'UnresolvedRelation [t], [], false -- !query @@ -1065,4 +1067,5 @@ DROP TABLE t -- !query schema struct<> -- !query output - +org.apache.spark.sql.AnalysisException +Table or view not found: t; line 1 pos 11 From 5f98b1e0bb7c3fd81babf0b7c6e4aa0b64bdf805 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Thu, 8 Sep 2022 16:57:12 -0700 Subject: [PATCH 04/19] fix tests --- .../describe-part-after-analyze.sql.out | 191 ++++++-- .../sql-tests/results/describe.sql.out | 429 ++++++++++++++---- .../sql-tests/results/explain-aqe.sql.out | 13 +- .../sql-tests/results/explain.sql.out | 13 +- 4 files changed, 496 insertions(+), 150 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out index 52b7bf1cebb5d..b04ad638dd2c0 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -5,8 +5,7 @@ CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. + -- !query @@ -15,8 +14,7 @@ VALUES ('k1', 100), ('k2', 200), ('k3', 300) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 18 + -- !query @@ -25,8 +23,7 @@ VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 18 + -- !query @@ -35,17 +32,33 @@ VALUES ('k1', 102), ('k2', 202) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 18 + -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query @@ -53,17 +66,34 @@ ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or permanent view not found: t; line 1 pos 14 + -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query @@ -71,26 +101,61 @@ ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or permanent view not found: t; line 1 pos 14 + -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 4 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query @@ -98,35 +163,88 @@ ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or permanent view not found: t; line 1 pos 14 + -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 4 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-09-01, hr=5] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-09-01/hr=5 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 2 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t -- !query @@ -134,5 +252,4 @@ DROP TABLE t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 11 + diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index f41610b348ada..45df378495ccb 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -8,8 +8,7 @@ CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. + -- !query @@ -17,8 +16,7 @@ CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 46 + -- !query @@ -39,8 +37,7 @@ CREATE VIEW v AS SELECT * FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 31 + -- !query @@ -48,8 +45,7 @@ ALTER TABLE t SET TBLPROPERTIES (e = '3') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 12 + -- !query @@ -57,53 +53,118 @@ ALTER TABLE t ADD PARTITION (c='Us', d=1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 12 + -- !query DESCRIBE t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 9 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string -- !query DESC default.t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: default.t; line 1 pos 5 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string -- !query DESC TABLE t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 11 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string -- !query DESC FORMATTED t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 15 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [e=3, password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog -- !query DESC EXTENDED t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [e=3, password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog -- !query @@ -111,17 +172,40 @@ ALTER TABLE t UNSET TBLPROPERTIES (e) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 12 + -- !query DESC EXTENDED t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog -- !query @@ -129,53 +213,147 @@ ALTER TABLE t UNSET TBLPROPERTIES (comment) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table not found: t; line 1 pos 12 + -- !query DESC EXTENDED t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Table Properties [password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog -- !query DESC t PARTITION (c='Us', d=1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 5 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string -- !query DESC EXTENDED t PARTITION (c='Us', d=1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] -- !query DESC FORMATTED t PARTITION (c='Us', d=1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 15 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] -- !query DESC EXTENDED t PARTITION (C='Us', D=1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 14 +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] -- !query @@ -183,8 +361,10 @@ DESC t PARTITION (c='Us', d=2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 5 +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +Partition not found in table 't' database 'default': +c -> Us +d -> 2 -- !query @@ -193,7 +373,7 @@ DESC t PARTITION (c='Us') struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 5 +Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`spark_catalog`.`default`.`t`' -- !query @@ -221,37 +401,45 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query DESC temp_v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: temp_v; line 1 pos 5 +a string +b int +c string +d string -- !query DESC TABLE temp_v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: temp_v; line 1 pos 11 +a string +b int +c string +d string -- !query DESC FORMATTED temp_v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: temp_v; line 1 pos 15 +a string +b int +c string +d string -- !query DESC EXTENDED temp_v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: temp_v; line 1 pos 14 +a string +b int +c string +d string -- !query @@ -283,43 +471,84 @@ DESC temp_v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: temp_v; line 1 pos 5 +{ + "errorClass" : "FORBIDDEN_OPERATION", + "messageParameters" : { + "statement" : "DESC PARTITION", + "objectType" : "TEMPORARY VIEW", + "objectName" : "`temp_v`" + } +} -- !query DESC v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: v; line 1 pos 5 +a string +b int +c string +d string -- !query DESC TABLE v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: v; line 1 pos 11 +a string +b int +c string +d string -- !query DESC FORMATTED v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: v; line 1 pos 15 +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Catalog and Namespace spark_catalog.default +View Query Output Columns [a, b, c, d] -- !query DESC EXTENDED v -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: v; line 1 pos 14 +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Catalog and Namespace spark_catalog.default +View Query Output Columns [a, b, c, d] -- !query @@ -328,7 +557,14 @@ DESC v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: v; line 1 pos 5 +{ + "errorClass" : "FORBIDDEN_OPERATION", + "messageParameters" : { + "statement" : "DESC PARTITION", + "objectType" : "VIEW", + "objectName" : "`v`" + } +} -- !query @@ -337,9 +573,8 @@ EXPLAIN DESC t struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 13; -'DescribeRelation false, [col_name#x, data_type#x, comment#x] -+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] -- !query @@ -348,9 +583,8 @@ EXPLAIN DESC EXTENDED t struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; -'DescribeRelation true, [col_name#x, data_type#x, comment#x] -+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] -- !query @@ -363,9 +597,15 @@ struct +- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true == Analyzed Logical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; -'DescribeRelation false, [col_name#x, data_type#x, comment#x] -+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true +col_name: string, data_type: string, comment: string +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + +== Optimized Logical Plan == +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] -- !query @@ -374,9 +614,8 @@ EXPLAIN DESCRIBE t b struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 17; -'DescribeColumn 'b, false, [info_name#x, info_value#x] -+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true +Execute DescribeColumnCommand + +- DescribeColumnCommand `spark_catalog`.`default`.`t`, [spark_catalog, default, t, b], false, [info_name#x, info_value#x] -- !query @@ -385,9 +624,8 @@ EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 17; -'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] -+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] -- !query @@ -395,8 +633,7 @@ DROP TABLE t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 11 + -- !query @@ -404,8 +641,7 @@ DROP VIEW temp_v -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -View not found: temp_v; line 1 pos 10 + -- !query @@ -421,5 +657,4 @@ DROP VIEW v -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -View not found: v; line 1 pos 10 + diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index faf85670c0b3d..7e237d1267897 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -1104,8 +1104,7 @@ CREATE table t(v array) USING PARQUET -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. + -- !query @@ -1114,10 +1113,9 @@ EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; -'Project [*] -+- 'Filter 'v IN (array(a),null) - +- 'UnresolvedRelation [t], [], false +*Filter v#x IN ([a],null) ++- *ColumnarToRow + +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> -- !query @@ -1125,5 +1123,4 @@ DROP TABLE t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 11 + diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 32dfa4150fb88..cdfeba62e7238 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -1046,8 +1046,7 @@ CREATE table t(v array) USING PARQUET -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Can not create the managed table('`spark_catalog`.`default`.`t`'). The associated location('file:/Users/Daniel.Tenedorio/spark-2/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t') already exists. + -- !query @@ -1056,10 +1055,9 @@ EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) struct -- !query output == Physical Plan == -org.apache.spark.sql.AnalysisException: Table or view not found: t; line 1 pos 22; -'Project [*] -+- 'Filter 'v IN (array(a),null) - +- 'UnresolvedRelation [t], [], false +*Filter v#x IN ([a],null) ++- *ColumnarToRow + +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> -- !query @@ -1067,5 +1065,4 @@ DROP TABLE t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException -Table or view not found: t; line 1 pos 11 + From 3448b5b98b48590ce43317cedfa3db10d0c606b9 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 9 Sep 2022 11:52:38 -0700 Subject: [PATCH 05/19] respond to code review comments --- core/src/main/resources/error/error-classes.json | 12 ++++++------ .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 12 ++++++------ .../negative-cases/invalid-correlation.sql.out | 12 ++++-------- .../scala/org/apache/spark/sql/SubquerySuite.scala | 4 ++-- 4 files changed, 18 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 1c67759056ff0..ca526c47fba60 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -329,7 +329,7 @@ }, "INVALID_SUBQUERY_EXPRESSION" : { "message" : [ - "Invalid subquery expression" + "Invalid subquery:" ], "subClass" : { "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { @@ -352,7 +352,7 @@ "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" ] }, - "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT" : { + "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_CERTAIN_PLACES" : { "message" : [ "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" ] @@ -362,7 +362,7 @@ "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses" ] }, - "IN_EXISTS_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT" : { + "IN_EXISTS_SUBQUERIES_ONLY_CERTAIN_PLACES" : { "message" : [ "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" ] @@ -372,17 +372,17 @@ "Lateral join condition cannot be non-deterministic: " ] }, - "MORE_THAN_ONE_OUTPUT_COLUMN" : { + "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN" : { "message" : [ "Scalar subquery must return only one column, but got " ] }, - "MUST_AGGREGATE_CORRELATED_SUBQUERY" : { + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY" : { "message" : [ "Correlated scalar subqueries must be aggregated" ] }, - "MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT" : { + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT" : { "message" : [ "The output of a correlated scalar subquery must be aggregated" ] 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 45e2e3efb504c..226f8e4e18fc4 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 @@ -756,7 +756,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { if (aggregates.isEmpty) { throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT", origin = expr.origin, messageParameters = Array.empty[String]) } @@ -824,7 +824,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { if (query.output.size != 1) { throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "MORE_THAN_ONE_OUTPUT_COLUMN", + errorSubClass = "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", origin = expr.origin, messageParameters = Array(query.output.size.toString)) } @@ -837,7 +837,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _ => throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "MUST_AGGREGATE_CORRELATED_SUBQUERY", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", origin = expr.origin, messageParameters = Array.empty[String]) } @@ -861,7 +861,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case other => throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT", + errorSubClass = "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_CERTAIN_PLACES", origin = other.origin, messageParameters = Array.empty[String]) } @@ -901,7 +901,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _ => throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_IN_FILTER_AGGREGATE_PROJECT", + errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_IN_CERTAIN_PLACES", origin = expr.origin, messageParameters = Array.empty[String]) } @@ -968,7 +968,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { if (hasOuterReferences(p)) { throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + errorSubClass = "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", origin = p.origin, messageParameters = Array.empty[String]) } 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 743094b4b083e..07be33f74af61 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 @@ -117,14 +117,10 @@ WHERE t1a IN (SELECT t2a struct<> -- !query output org.apache.spark.sql.AnalysisException -Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: -Aggregate [min(outer(t2a#x)) AS min(outer(t2.t2a))#x] -+- SubqueryAlias t3 - +- View (`t3`, [t3a#x,t3b#x,t3c#x]) - +- 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] - +- Project [t3a#x, t3b#x, t3c#x] - +- SubqueryAlias t3 - +- LocalRelation [t3a#x, t3b#x, t3c#x] +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING" +} -- !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 68b5189c66496..1c61c1a368f47 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 @@ -538,14 +538,14 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkError( msg1, errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SUBQUERY")) + errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY")) val msg2 = intercept[AnalysisException] { sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") } checkError( msg2, errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SUBQUERY_OUTPUT")) + errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT")) } test("non-equal correlated scalar subquery") { From 9445f609e4f8b2732d623ac9c5271a3fc09b2d3f Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 9 Sep 2022 12:00:13 -0700 Subject: [PATCH 06/19] remove unused code --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 11 ----------- 1 file changed, 11 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 226f8e4e18fc4..07cddcf2fac4e 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 @@ -55,17 +55,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { throw new AnalysisException(msg) } - protected def failAnalysisForSubqueryExpression( - errorSubClass: String, - origin: Origin, - messageParameters: Array[String] = Array.empty[String]): Nothing = { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = errorSubClass, - origin = origin, - messageParameters = messageParameters) - } - protected def containsMultipleGenerators(exprs: Seq[Expression]): Boolean = { exprs.flatMap(_.collect { case e: Generator => e From bde1b6d0eca714bd4c0376790d3d66cd135cf9ec Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 9 Sep 2022 12:05:23 -0700 Subject: [PATCH 07/19] remove unused code --- .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 07cddcf2fac4e..eabc19ebaea80 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, DecorrelateInnerQuery, InlineCTE} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.{Origin, TreeNodeTag} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_WINDOW_EXPRESSION import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, TypeUtils} import org.apache.spark.sql.connector.catalog.{LookupCatalog, SupportsPartitionManagement} From dfce9b14ded789bd8508d13b32bd8e3860ed8b2f Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 9 Sep 2022 16:34:43 -0700 Subject: [PATCH 08/19] fix tests --- core/src/main/resources/error/error-classes.json | 10 +++++----- .../catalyst/analysis/AnalysisErrorSuite.scala | 2 +- .../negative-cases/subq-input-typecheck.sql.out | 16 ++++++++++++++-- .../udf/postgreSQL/udf-aggregates_part1.sql.out | 8 +++++++- .../sql-tests/results/udf/udf-except.sql.out | 16 +++++++--------- 5 files changed, 34 insertions(+), 18 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ca526c47fba60..8770f0f290e47 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -372,11 +372,6 @@ "Lateral join condition cannot be non-deterministic: " ] }, - "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN" : { - "message" : [ - "Scalar subquery must return only one column, but got " - ] - }, "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY" : { "message" : [ "Correlated scalar subqueries must be aggregated" @@ -401,6 +396,11 @@ "message" : [ "Outer attribute not found: " ] + }, + "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN" : { + "message" : [ + "Scalar subquery must return only one column, but got " + ] } } }, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 553f46a00c27c..892f05981f1a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -723,7 +723,7 @@ class AnalysisErrorSuite extends AnalysisTest { Seq(SortOrder(InSubquery(Seq(a), ListQuery(LocalRelation(b))), Ascending)), global = true, LocalRelation(a)) - assertAnalysisError(plan, "Predicate sub-queries can only be used in " :: Nil) + assertAnalysisError(plan, "Predicate subqueries can only be used in " :: Nil) } test("PredicateSubQuery correlated predicate is nested in an illegal plan") { diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index 8a8334fb57d68..6251a2f48a042 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -61,7 +61,13 @@ FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Scalar subquery must return only one column, but got 2 +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "messageParameters" : { + "number" : "2" + } +} -- !query @@ -76,7 +82,13 @@ FROM t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Scalar subquery must return only one column, but got 2 +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "messageParameters" : { + "number" : "2" + } +} -- !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 fc11234112b40..8e97e0f0e2d3f 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 @@ -475,7 +475,13 @@ having exists (select 1 from onek b struct<> -- !query output org.apache.spark.sql.AnalysisException -Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: sum(DISTINCT (outer(a.four) + b.four)) +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 0134ad74f5239..bff3758ce01ca 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -100,12 +100,10 @@ WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) struct<> -- !query output org.apache.spark.sql.AnalysisException -Correlated column is not allowed in predicate (CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#x) as string)) AS STRING)): -Aggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS udf(max(udf(v)))#x] -+- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string)) - +- SubqueryAlias t2 - +- View (`t2`, [k#x,v#x]) - +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] - +- Project [k#x, v#x] - +- SubqueryAlias t2 - +- LocalRelation [k#x, v#x] +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + "messageParameters" : { + "predicate" : "(CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#440) as string)) AS STRING))" + } +} From 5e204a67dcd1b941061721f3ff9fc19a85934012 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 9 Sep 2022 19:22:57 -0700 Subject: [PATCH 09/19] fix tests --- core/src/main/resources/error/error-classes.json | 2 +- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- .../resources/sql-tests/results/udf/udf-except.sql.out | 3 --- .../scala/org/apache/spark/sql/SubquerySuite.scala | 10 +++------- 4 files changed, 6 insertions(+), 13 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8770f0f290e47..6c8e8e83a6645 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -344,7 +344,7 @@ }, "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE" : { "message" : [ - "Correlated column is not allowed in predicate " + "Correlated column is not allowed in predicate" ] }, "CORRELATED_SCALAR_SUBQUERIES_IN_GROUP_BY_MUST_BE_IN_AGGREGATE_EXPRESSIONS" : { 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 eabc19ebaea80..ae05f7d636eb9 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 @@ -890,7 +890,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _ => throw new AnalysisException( errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_IN_CERTAIN_PLACES", + errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_CERTAIN_PLACES", origin = expr.origin, messageParameters = Array.empty[String]) } @@ -1049,7 +1049,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { errorClass = "INVALID_SUBQUERY_EXPRESSION", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", origin = p.origin, - messageParameters = Array(s"${predicates.map(_.sql).mkString}")) + messageParameters = Array.empty[String]) } } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index bff3758ce01ca..072a966cfda63 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -103,7 +103,4 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_SUBQUERY_EXPRESSION", "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - "messageParameters" : { - "predicate" : "(CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#440) as string)) AS STRING))" - } } 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 1c61c1a368f47..d2b21ff7623bd 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 @@ -555,8 +555,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkError( msg1, errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), - parameters = Map("predicate" -> "(l2.a < outer(l1.a))")) + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) } test("disjunctive correlated scalar subquery") { @@ -1999,8 +1998,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkError( msg1, errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), - parameters = Map("predicate" -> "(t2.c1 > outer(t1.c1))")) + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) } } @@ -2030,9 +2028,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkError( msg1, errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE"), - parameters = Map( - "predicate" -> "(CAST(t2.c1 AS SMALLINT) = outer(__auto_generated_subquery_name.a))")) + errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) } } From 868b3a265d9a0965e43ccb811b51533abc96c50d Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Sat, 10 Sep 2022 13:21:06 -0700 Subject: [PATCH 10/19] respond to code review comments stash respond to code review comments stash respond to code review comments update tests --- .../main/resources/error/error-classes.json | 136 +++++++++--------- .../apache/spark/SparkThrowableHelper.scala | 7 +- .../org/apache/spark/SparkFunSuite.scala | 10 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 100 ++++++------- .../spark/sql/catalyst/analysis/package.scala | 39 ++++- .../sql/errors/QueryCompilationErrors.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 11 ++ .../sql-tests/results/join-lateral.sql.out | 35 ++++- .../postgreSQL/aggregates_part1.sql.out | 11 +- .../invalid-correlation.sql.out | 34 ++++- .../subq-input-typecheck.sql.out | 18 ++- .../postgreSQL/udf-aggregates_part1.sql.out | 11 +- .../sql-tests/results/udf/udf-except.sql.out | 10 +- .../org/apache/spark/sql/SubquerySuite.scala | 106 ++++++++------ 14 files changed, 343 insertions(+), 189 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 6c8e8e83a6645..aaac0374b305f 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -332,73 +332,8 @@ "Invalid subquery:" ], "subClass" : { - "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { - "message" : [ - "Accessing outer query column is not allowed in this location" - ] - }, - "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES" : { - "message" : [ - "Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: " - ] - }, - "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE" : { - "message" : [ - "Correlated column is not allowed in predicate" - ] - }, - "CORRELATED_SCALAR_SUBQUERIES_IN_GROUP_BY_MUST_BE_IN_AGGREGATE_EXPRESSIONS" : { - "message" : [ - "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" - ] - }, - "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_CERTAIN_PLACES" : { - "message" : [ - "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" - ] - }, - "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING" : { - "message" : [ - "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses" - ] - }, - "IN_EXISTS_SUBQUERIES_ONLY_CERTAIN_PLACES" : { - "message" : [ - "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" - ] - }, - "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC" : { - "message" : [ - "Lateral join condition cannot be non-deterministic: " - ] - }, - "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY" : { - "message" : [ - "Correlated scalar subqueries must be aggregated" - ] - }, - "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT" : { - "message" : [ - "The output of a correlated scalar subquery must be aggregated" - ] - }, - "NON_CORRELATED_COLUMNS_IN_GROUP_BY" : { - "message" : [ - "A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns: " - ] - }, - "NON_DETERMINISTIC_LATERAL_SUBQUERIES" : { - "message" : [ - "Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row" - ] - }, - "OUTER_ATTRIBUTE_NOT_FOUND" : { - "message" : [ - "Outer attribute not found: " - ] - }, "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN" : { - "message" : [ + "message": [ "Scalar subquery must return only one column, but got " ] } @@ -778,6 +713,73 @@ } } }, + "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY" : { + "message" : [ + "Invalid subquery:" + ], + "subClass" : { + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { + "message" : [ + "Accessing outer query column is not allowed in this location" + ] + }, + "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES" : { + "message" : [ + "Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: " + ] + }, + "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE" : { + "message" : [ + "Correlated column is not allowed in predicate" + ] + }, + "CORRELATED_COLUMN_NOT_FOUND" : { + "message" : [ + "A correlated outer name reference within a subquery expression body was not found in the enclosing query: " + ] + }, + "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC" : { + "message" : [ + "Lateral join condition cannot be non-deterministic: " + ] + }, + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY" : { + "message" : [ + "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" + ] + }, + "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT" : { + "message" : [ + "The output of a correlated scalar subquery must be aggregated" + ] + }, + "NON_CORRELATED_COLUMNS_IN_GROUP_BY" : { + "message" : [ + "A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns: " + ] + }, + "NON_DETERMINISTIC_LATERAL_SUBQUERIES" : { + "message" : [ + "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_SCALAR_SUBQUERY" : { + "message" : [ + "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" + ] + }, + "UNSUPPORTED_IN_EXISTS_SUBQUERY" : { + "message" : [ + "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" + ] + } + } + }, "UNTYPED_SCALA_UDF" : { "message" : [ "You're using untyped Scala UDF, which does not have the input type information. Spark may blindly pass null to the Scala closure with primitive-type argument, and the closure will see the default value of the Java type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, the result is 0 for null input. To get rid of this error, you could:", @@ -792,4 +794,4 @@ ], "sqlState" : "40000" } -} +}, diff --git a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 92128f2b48b87..95d37308bc0bd 100644 --- a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -176,7 +176,12 @@ private[spark] object SparkThrowableHelper { if (!parameterNames.isEmpty) { g.writeObjectFieldStart("messageParameters") (parameterNames zip e.getMessageParameters).foreach { case (name, value) => - g.writeStringField(name, value) + // Skip parameters with the name "planString" in the output to keep results + // deterministic since these strings usually contain expression IDs which change from + // one test run to another. + if (name != "planString") { + g.writeStringField(name, value) + } } g.writeEndObject() } diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 95d13b17ee418..1cb8fa5cf2222 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -380,6 +380,16 @@ abstract class SparkFunSuite checkError(exception, errorClass, Some(errorSubClass), sqlState, parameters, false, Array(context)) + protected def checkErrorMatchPVals( + exception: SparkThrowable, + errorClass: String, + errorSubClass: String, + sqlState: Option[String], + parameters: Map[String, String], + context: QueryContext): Unit = + checkError(exception, errorClass, Some(errorSubClass), sqlState, parameters, + matchPVals = true, Array(context)) + protected def checkError( exception: SparkThrowable, errorClass: String, 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 ae05f7d636eb9..7d730d7c4924e 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 @@ -743,11 +743,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case a: AggregateExpression => a }) if (aggregates.isEmpty) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT", - origin = expr.origin, - messageParameters = Array.empty[String]) + expr.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT") } // SPARK-18504/SPARK-18814: Block cases where GROUP BY columns @@ -760,10 +758,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { val invalidCols = groupByCols -- correlatedCols // GROUP BY columns must be a subset of columns in the predicates if (invalidCols.nonEmpty) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + expr.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "NON_CORRELATED_COLUMNS_IN_GROUP_BY", - origin = expr.origin, messageParameters = Array(invalidCols.map(_.name).mkString(","))) } } @@ -789,10 +786,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case o: OuterReference => p.children.foreach(e => if (!e.output.exists(_.exprId == o.exprId)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "OUTER_ATTRIBUTE_NOT_FOUND", - origin = o.origin, + o.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "CORRELATED_COLUMN_NOT_FOUND", messageParameters = Array(o.name)) }) case _ => @@ -811,10 +807,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case ScalarSubquery(query, outerAttrs, _, _) => // Scalar subquery must return one column as output. if (query.output.size != 1) { - throw new AnalysisException( + expr.failAnalysis( errorClass = "INVALID_SUBQUERY_EXPRESSION", errorSubClass = "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", - origin = expr.origin, messageParameters = Array(query.output.size.toString)) } @@ -823,12 +818,11 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case a: Aggregate => checkAggregateInScalarSubquery(outerAttrs, query, a) case Filter(_, a: Aggregate) => checkAggregateInScalarSubquery(outerAttrs, query, a) case p: LogicalPlan if p.maxRows.exists(_ <= 1) => // Ok - case _ => - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + case other => + expr.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - origin = expr.origin, - messageParameters = Array.empty[String]) + planString = other.toString) } // Only certain operators are allowed to host subquery expression containing @@ -840,19 +834,16 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // it must also be in the aggregate expressions to be rewritten in the optimization // phase. if (containsExpr(a.groupingExpressions) && !containsExpr(a.aggregateExpressions)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = - "CORRELATED_SCALAR_SUBQUERIES_IN_GROUP_BY_MUST_BE_IN_AGGREGATE_EXPRESSIONS", - origin = a.origin, - messageParameters = Array.empty[String]) + a.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", + planString = a.toString) } case other => - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "CORRELATED_SCALAR_SUBQUERIES_ONLY_IN_CERTAIN_PLACES", - origin = other.origin, - messageParameters = Array.empty[String]) + other.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + planString = other.toString) } } // Validate to make sure the correlations appearing in the query are valid and @@ -865,18 +856,16 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // A lateral join with a multi-row outer query and a non-deterministic lateral subquery // cannot be decorrelated. Otherwise it may produce incorrect results. if (!expr.deterministic && !join.left.maxRows.exists(_ <= 1)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + expr.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "NON_DETERMINISTIC_LATERAL_SUBQUERIES", - origin = expr.origin, - messageParameters = Array.empty[String]) + planString = plan.toString) } // Check if the lateral join's join condition is deterministic. if (join.condition.exists(!_.deterministic)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + join.condition.get.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", - origin = join.origin, messageParameters = Array(join.condition.get.sql)) } // Validate to make sure the correlations appearing in the query are valid and @@ -888,11 +877,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _: Filter | _: SupportsSubquery | _: Join | _: Project | _: Aggregate | _: Window => // Ok case _ => - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "IN_EXISTS_SUBQUERIES_ONLY_CERTAIN_PLACES", - origin = expr.origin, - messageParameters = Array.empty[String]) + expr.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "UNSUPPORTED_IN_EXISTS_SUBQUERY", + planString = plan.toString) } // Validate to make sure the correlations appearing in the query are valid and // allowed by spark. @@ -946,7 +934,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.foreach { case a: AggregateExpression if containsOuter(a) => if (a.references.nonEmpty) { - throw QueryCompilationErrors.mixedRefsInAggFunc(a.sql, a.origin) + a.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + messageParameters = Array(a.sql)) } case _ => } @@ -955,11 +946,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { // Make sure a plan's subtree does not contain outer references def failOnOuterReferenceInSubTree(p: LogicalPlan): Unit = { if (hasOuterReferences(p)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + p.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", - origin = p.origin, - messageParameters = Array.empty[String]) + planString = p.toString) } } @@ -978,11 +968,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def failOnInvalidOuterReference(p: LogicalPlan): Unit = { p.expressions.foreach(checkMixedReferencesInsideAggregateExpr) if (!canHostOuter(p) && p.expressions.exists(containsOuter)) { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING", - origin = p.origin, - messageParameters = Array.empty[String]) + p.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "UNSUPPORTED_CORRELATED_REFERENCE", + planString = p.toString) } } @@ -1045,11 +1034,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { def failOnUnsupportedCorrelatedPredicate(predicates: Seq[Expression], p: LogicalPlan): Unit = { if (predicates.nonEmpty) { // Report a non-supported case as an exception - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + p.failAnalysis( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - origin = p.origin, - messageParameters = Array.empty[String]) + planString = s"${predicates.map(_.sql).mkString}:\n$p") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index be6dcfac269a5..f9c169bca2d6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.errors.QueryErrorsBase +import org.apache.spark.sql.internal.SQLConf /** * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. @@ -45,11 +46,15 @@ package object analysis { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition) } - /** Fails the analysis at the point where a specific tree node was parsed. */ + /** Fails the analysis at the point where a specific tree node was parsed with a given cause. */ def failAnalysis(msg: String, cause: Throwable): Nothing = { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition, cause = Some(cause)) } + /** + * Fails the analysis at the point where a specific tree node was parsed using a provided + * error class and message parameters. + */ def failAnalysis(errorClass: String, messageParameters: Array[String]): Nothing = { throw new AnalysisException( errorClass = errorClass, @@ -57,6 +62,38 @@ package object analysis { origin = t.origin) } + /** + * Fails the analysis at the point where a specific tree node was parsed using a provided + * error class and subclass and message parameters. + */ + def failAnalysis( + errorClass: String, + errorSubClass: String, + messageParameters: Array[String] = Array.empty[String]): Nothing = { + throw new AnalysisException( + errorClass = errorClass, + errorSubClass = errorSubClass, + messageParameters = messageParameters, + origin = t.origin) + } + + /** + * Fails the analysis at the point where a specific tree node was parsed using a provided + * error class and subclass and one message parameter comprising a plan string. The plan string + * will be printed in the error message if and only if the corresponding Spark configuration is + * enabled. + */ + def failAnalysis( + errorClass: String, + errorSubClass: String, + planString: String): Nothing = { + throw new AnalysisException( + errorClass = errorClass, + errorSubClass = errorSubClass, + messageParameters = Array(if (SQLConf.get.includePlansInErrors) s": $planString" else ""), + origin = t.origin) + } + def dataTypeMismatch(expr: Expression, mismatch: DataTypeMismatch): Nothing = { throw new AnalysisException( errorClass = "DATATYPE_MISMATCH", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 3eedac2352ce8..5ce94e124416e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1544,8 +1544,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def mixedRefsInAggFunc(funcStr: String, origin: Origin): Throwable = { - throw new AnalysisException( - errorClass = "INVALID_SUBQUERY_EXPRESSION", + new AnalysisException( + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", origin = origin, messageParameters = Array(funcStr)) 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 de25c19a26eb8..ed4e18b0e85ed 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 @@ -3911,6 +3911,15 @@ object SQLConf { .checkValues(ErrorMessageFormat.values.map(_.toString)) .createWithDefault(ErrorMessageFormat.PRETTY.toString) + val INCLUDE_PLANS_IN_ERRORS = buildConf("spark.sql.error.includePlans") + .doc("If true, include the string representation of query plans or expressions in error " + + "messages when the origins of these entities are also available. Otherwise, elide these " + + "strings from the error messages since these origins convey the source locations of the " + + "erroneous behavior.") + .version("3.4.0") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * @@ -4705,6 +4714,8 @@ class SQLConf extends Serializable with Logging { def errorMessageFormat: ErrorMessageFormat.Value = ErrorMessageFormat.withName(getConf(SQLConf.ERROR_MESSAGE_FORMAT)) + def includePlansInErrors: Boolean = getConf(SQLConf.INCLUDE_PLANS_IN_ERRORS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 718c21d29dfee..42ebad51ae35d 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 @@ -318,8 +318,16 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", - "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES" + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "messageParameters" : { }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 58, + "fragment" : "SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)" + } ] } @@ -330,8 +338,16 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", - "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES" + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "messageParameters" : { }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2)" + } ] } @@ -342,11 +358,18 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", "messageParameters" : { "condition" : "((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE))" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.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 eb203461e2e2c..7d9d2af9f6d26 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 @@ -485,11 +485,18 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", "messageParameters" : { "function" : "sum(DISTINCT (outer(a.four) + b.four))" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] } 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 281f6eb6b6a59..9725e0d67b0b9 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,11 +76,18 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", "messageParameters" : { "function" : "min((outer(t1.t1a) + t2.t2a))" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 139, + "stopIndex" : 152, + "fragment" : "min(t1a + t2a)" + } ] } @@ -98,11 +105,18 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", "messageParameters" : { "function" : "min((outer(t2.t2a) + t3.t3a))" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 233, + "stopIndex" : 246, + "fragment" : "min(t2a + t3a)" + } ] } @@ -118,8 +132,16 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", - "errorSubClass" : "EXPRESSIONS_REFERENCING_OUTER_QUERY_COLUMN_ONLY_ALLOWED_IN_WHERE_HAVING" + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + "errorSubClass" : "UNSUPPORTED_CORRELATED_REFERENCE", + "messageParameters" : { }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 162, + "fragment" : "SELECT t1a \nFROM t1\nWHERE t1a IN (SELECT t2a \n FROM t2\n WHERE EXISTS (SELECT min(t2a) \n FROM t3))" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index 6251a2f48a042..30d50b6fe1ff4 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -66,7 +66,14 @@ org.apache.spark.sql.AnalysisException "errorSubClass" : "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "messageParameters" : { "number" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 109, + "fragment" : "SELECT \n ( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b = t1.t1b\n GROUP BY t2.t2b\n )\nFROM t1" + } ] } @@ -87,7 +94,14 @@ org.apache.spark.sql.AnalysisException "errorSubClass" : "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "messageParameters" : { "number" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 104, + "fragment" : "SELECT \n ( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b > 0\n GROUP BY t2.t2b\n )\nFROM t1" + } ] } 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 d6d427197a6fd..e12f12384edf7 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 @@ -476,11 +476,18 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", "messageParameters" : { "function" : "sum(DISTINCT (outer(a.four) + b.four))" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 072a966cfda63..6d744f41aed94 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -101,6 +101,14 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_SUBQUERY_EXPRESSION", + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + "messageParameters" : { }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 281, + "fragment" : "SELECT t1.k\nFROM t1\nWHERE t1.v <= (SELECT udf(max(udf(t2.v)))\n FROM t2\n WHERE udf(t2.k) = udf(t1.k))\nMINUS\nSELECT t1.k\nFROM t1\nWHERE udf(t1.v) >= (SELECT min(udf(t2.v))\n FROM t2\n WHERE t2.k = t1.k)" + } ] } 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 4ae3b5e8253df..35fec5223bc02 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 @@ -524,42 +524,56 @@ class SubquerySuite extends QueryTest withTempView("t") { Seq((1, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") - val errMsg = intercept[AnalysisException] { - sql("select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1") + val str = "select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1" + val exception = intercept[AnalysisException] { + sql(str) } checkError( - errMsg, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("NON_CORRELATED_COLUMNS_IN_GROUP_BY"), - parameters = Map("value" -> "c2")) - } + exception, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "NON_CORRELATED_COLUMNS_IN_GROUP_BY", + parameters = Map("value" -> "c2"), + sqlState = None, + context = ExpectedContext(fragment = str, start = 0, stop = 81)) } } test("non-aggregated correlated scalar subquery") { - val msg1 = intercept[AnalysisException] { - sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") + val str1 = "select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1" + val exception1 = intercept[AnalysisException] { + sql(str1) } - checkError( - msg1, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY")) - val msg2 = intercept[AnalysisException] { - sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") + checkErrorMatchPVals( + exception1, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", + parameters = Map("planString" -> "(?s): Filter .*"), + sqlState = None, + context = ExpectedContext(fragment = str1, start = 0, stop = 63)) + val str2 = "select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1" + val exception2 = intercept[AnalysisException] { + sql(str2) } - checkError( - msg2, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT")) + checkErrorMatchPVals( + exception2, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT", + parameters = Map.empty[String, String], + sqlState = None, + context = ExpectedContext(fragment = str2, start = 0, stop = 74)) } test("non-equal correlated scalar subquery") { - val msg1 = intercept[AnalysisException] { - sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") + val str = "select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1" + val exception = intercept[AnalysisException] { + sql(str) } - checkError( - msg1, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) + checkErrorMatchPVals( + exception, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + parameters = Map("planString" -> "(?s): .*"), + sqlState = None, + context = ExpectedContext(fragment = str, start = 0, stop = 68)) } test("disjunctive correlated scalar subquery") { @@ -857,7 +871,7 @@ class SubquerySuite extends QueryTest WHERE t1.c1 = t2.c1)""".stripMargin), Row(1) :: Row(0) :: Nil) - val msg1 = intercept[AnalysisException] { + val exception1 = intercept[AnalysisException] { sql( """ | SELECT c1 @@ -867,7 +881,7 @@ class SubquerySuite extends QueryTest | WHERE t1.c1 = t2.c1) """.stripMargin) } - assert(msg1.getMessage.contains( + assert(exception1.getMessage.contains( "Expressions referencing the outer query are not supported outside of WHERE/HAVING")) } } @@ -2000,13 +2014,17 @@ class SubquerySuite extends QueryTest withTempView("t1", "t2") { Seq((0, 1)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 2), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") - val msg1 = intercept[AnalysisException] { - sql("SELECT * FROM t1 JOIN LATERAL (SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1)") + val str = "SELECT * FROM t1 JOIN LATERAL (SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1)" + val exception = intercept[AnalysisException] { + sql(str) } - checkError( - msg1, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) + checkErrorMatchPVals( + exception, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + parameters = Map("planString" -> "(?s) .*"), + sqlState = None, + context = ExpectedContext(fragment = str, start = 0, stop = 74)) } } @@ -2026,17 +2044,19 @@ class SubquerySuite extends QueryTest |FROM (SELECT CAST(c1 AS STRING) a FROM t1) |""".stripMargin), Row(5) :: Row(null) :: Nil) - val msg1 = intercept[AnalysisException] { - sql( - """ - |SELECT (SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a) - |FROM (SELECT CAST(c1 AS SHORT) a FROM t1) - |""".stripMargin) + val str = + """SELECT (SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a) + |FROM (SELECT CAST(c1 AS SHORT) a FROM t1)""".stripMargin + val exception1 = intercept[AnalysisException] { + sql(str) } - checkError( - msg1, - errorClass = "INVALID_SUBQUERY_EXPRESSION", - errorSubClass = Some("CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE")) + checkErrorMatchPVals( + exception1, + errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", + errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", + parameters = Map("planString" -> "(?s) .*"), + sqlState = None, + context = ExpectedContext(fragment = str, start = 0, stop = 100)) } } From 7f1d2010ec3a7a42f309959c4d1106e071947faf Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 12 Sep 2022 13:06:03 -0700 Subject: [PATCH 11/19] respond to code review comments --- core/src/main/resources/error/error-classes.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index aaac0374b305f..419f79b6abe71 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -794,4 +794,4 @@ ], "sqlState" : "40000" } -}, +} From fd1ae375bc6fdb80840f03dcc155d243af96e4a2 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 12 Sep 2022 15:36:30 -0700 Subject: [PATCH 12/19] fix tests --- core/src/main/resources/error/error-classes.json | 4 ++-- .../spark/sql/catalyst/analysis/AnalysisErrorSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 419f79b6abe71..28f72724e0e19 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -333,7 +333,7 @@ ], "subClass" : { "SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN" : { - "message": [ + "message" : [ "Scalar subquery must return only one column, but got " ] } @@ -715,7 +715,7 @@ }, "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY" : { "message" : [ - "Invalid subquery:" + "Invalid subquery expression:" ], "subClass" : { "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 892f05981f1a6..26023d7dc71b3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -870,7 +870,7 @@ class AnalysisErrorSuite extends AnalysisTest { Filter(cond, t1)) ).as("sub") :: Nil, t2) - assertAnalysisError(plan, s"Correlated column is not allowed in predicate ($msg)" :: Nil) + assertAnalysisError(plan, s"Correlated column is not allowed in predicate: ($msg)" :: Nil) } } From c2c1d33b4bb1f6aeb2e9006e06951db65de7255b Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 13 Sep 2022 09:10:42 -0700 Subject: [PATCH 13/19] fix test --- .../src/test/scala/org/apache/spark/sql/SubquerySuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 35fec5223bc02..3c5c9b5404540 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 @@ -2022,7 +2022,7 @@ class SubquerySuite extends QueryTest exception, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s) .*"), + parameters = Map("planString" -> "(?s): .*"), sqlState = None, context = ExpectedContext(fragment = str, start = 0, stop = 74)) } @@ -2054,7 +2054,7 @@ class SubquerySuite extends QueryTest exception1, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s) .*"), + parameters = Map("planString" -> "(?s): .*"), sqlState = None, context = ExpectedContext(fragment = str, start = 0, stop = 100)) } From 5cb4accb3366f0c4baef0c8c04de46c1aece5d2b Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 13 Sep 2022 10:11:50 -0700 Subject: [PATCH 14/19] respond to code review comments respond to code review comments respond to code review comments update update update update --- .../main/resources/error/error-classes.json | 2 +- .../apache/spark/SparkThrowableHelper.scala | 7 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 16 ++--- .../spark/sql/catalyst/analysis/package.scala | 59 +++++++++++++++++-- .../expressions/namedExpressions.scala | 8 +++ .../sql-tests/results/join-lateral.sql.out | 8 ++- .../invalid-correlation.sql.out | 4 +- .../sql-tests/results/udf/udf-except.sql.out | 4 +- .../org/apache/spark/sql/SubquerySuite.scala | 6 +- 9 files changed, 88 insertions(+), 26 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index d7576a7f257ed..9cea2abdb722c 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -709,7 +709,7 @@ }, "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY" : { "message" : [ - "Invalid subquery expression:" + "Unsupported subquery expression:" ], "subClass" : { "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { diff --git a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 9e3454ab072bc..93e88261cb672 100644 --- a/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/core/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -204,12 +204,7 @@ private[spark] object SparkThrowableHelper { if (!parameterNames.isEmpty) { g.writeObjectFieldStart("messageParameters") (parameterNames zip e.getMessageParameters).foreach { case (name, value) => - // Skip parameters with the name "planString" in the output to keep results - // deterministic since these strings usually contain expression IDs which change from - // one test run to another. - if (name != "planString") { - g.writeStringField(name, value) - } + g.writeStringField(name, value) } g.writeEndObject() } 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 85b3355ef50d3..1711289b3193a 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 @@ -822,7 +822,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - planString = other.toString) + treeNodes = Seq(other)) } // Only certain operators are allowed to host subquery expression containing @@ -837,13 +837,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { a.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - planString = a.toString) + treeNodes = Seq(a)) } case other => other.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", - planString = other.toString) + treeNodes = Seq(other)) } } // Validate to make sure the correlations appearing in the query are valid and @@ -859,7 +859,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "NON_DETERMINISTIC_LATERAL_SUBQUERIES", - planString = plan.toString) + treeNodes = Seq(plan)) } // Check if the lateral join's join condition is deterministic. if (join.condition.exists(!_.deterministic)) { @@ -880,7 +880,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_IN_EXISTS_SUBQUERY", - planString = plan.toString) + treeNodes = Seq(plan)) } // Validate to make sure the correlations appearing in the query are valid and // allowed by spark. @@ -949,7 +949,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", - planString = p.toString) + treeNodes = Seq(p)) } } @@ -971,7 +971,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_CORRELATED_REFERENCE", - planString = p.toString) + treeNodes = Seq(p)) } } @@ -1037,7 +1037,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - planString = s"${predicates.map(_.sql).mkString}:\n$p") + treeNodes = predicates) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 4d1fea6f5f1d6..d918597d78aa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.catalyst +import java.util.UUID + +import scala.collection.mutable + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.errors.QueryErrorsBase import org.apache.spark.sql.internal.SQLConf @@ -86,12 +91,58 @@ package object analysis { def failAnalysis( errorClass: String, errorSubClass: String, - planString: String): Nothing = { + treeNodes: Seq[TreeNode[_]]): Nothing = { + // Normalize expression IDs in the query plan to keep tests deterministic. + object IdGen { + private val curId = new java.util.concurrent.atomic.AtomicLong() + private val jvmId = UUID.randomUUID() + private val idMap = mutable.Map.empty[ExprId, ExprId] + def get(previous: ExprId): ExprId = { + if (!idMap.contains(previous)) idMap.put(previous, ExprId(curId.getAndIncrement(), jvmId)) + idMap.get(previous).get + } + } + def normalizeExpr(expr: Expression): Expression = { + expr.withNewChildren(expr.children.map(normalizeExpr)) match { + case a: Attribute => a.withExprId(IdGen.get(a.exprId)) + case a: Alias => a.withExprId(IdGen.get(a.exprId)) + case n: NamedLambdaVariable => n.copy(exprId = IdGen.get(n.exprId)) + case o: OuterReference => o.copy(e = normalizeExpr(o.e).asInstanceOf[NamedExpression]) + case s: ScalarSubquery => s.copy(plan = normalizePlan(s.plan), + exprId = IdGen.get(s.exprId), outerAttrs = s.outerAttrs.map(normalizeExpr), + joinCond = s.joinCond.map(normalizeExpr)) + case s: Exists => s.copy(plan = normalizePlan(s.plan), exprId = IdGen.get(s.exprId), + outerAttrs = s.outerAttrs.map(normalizeExpr), joinCond = s.joinCond.map(normalizeExpr)) + case s: LateralSubquery => s.copy(plan = normalizePlan(s.plan), + exprId = IdGen.get(s.exprId), outerAttrs = s.outerAttrs.map(normalizeExpr), + joinCond = s.joinCond.map(normalizeExpr)) + case s: InSubquery => s.copy(values = s.values.map(normalizeExpr), + query = s.query.copy(plan = normalizePlan(s.query.plan), + exprId = IdGen.get(s.query.exprId), + childOutputs = s.query.childOutputs.map(a => a.withExprId(IdGen.get(a.exprId))), + outerAttrs = s.query.outerAttrs.map(normalizeExpr), + joinCond = s.query.joinCond.map(normalizeExpr))) + case other => other + } + } + def normalizePlan(node: LogicalPlan): LogicalPlan = { + node.withNewChildren(node.children.map(normalizePlan)).mapExpressions(normalizeExpr) + } + val planString = + if (SQLConf.get.includePlansInErrors) { + s": ${ + treeNodes.map { + case plan: LogicalPlan => normalizePlan(plan) + case expr: Expression => normalizeExpr(expr) + }.mkString("\n") + }" + } else { + "" + } throw new AnalysisException( errorClass = errorClass, errorSubClass = errorSubClass, - messageParameters = - Map("planString" -> (if (SQLConf.get.includePlansInErrors) s": $planString" else "")), + messageParameters = Map("planString" -> planString), origin = t.origin) } 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 4181edcb8c601..a7eec4774eae0 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 @@ -182,6 +182,14 @@ case class Alias(child: Expression, name: String)( nonInheritableMetadataKeys = nonInheritableMetadataKeys) } + def withExprId(newExprId: ExprId): NamedExpression = { + Alias(child, name)( + exprId = newExprId, + qualifier = qualifier, + explicitMetadata = explicitMetadata, + nonInheritableMetadataKeys = nonInheritableMetadataKeys) + } + def newInstance(): NamedExpression = Alias(child, name)( qualifier = qualifier, 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 42ebad51ae35d..8325f200f639d 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 @@ -320,7 +320,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", - "messageParameters" : { }, + "messageParameters" : { + "planString" : ": !LateralJoin lateral-subquery#5 [c1#6 && c2#7], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#2) + outer(c2#3)) as double) + rand(0)) AS c3#4]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#2,c2#3])\n +- Project [cast(col1#0 as int) AS c1#2, cast(col2#1 as int) AS c2#3]\n +- LocalRelation [col1#0, col2#1]\n" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -340,7 +342,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", - "messageParameters" : { }, + "messageParameters" : { + "planString" : ": LateralJoin lateral-subquery#9 [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(0) AS rand(0)#8]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#6,c2#7])\n: +- Project [cast(col1#4 as int) AS c1#6, cast(col2#5 as int) AS c2#7]\n: +- LocalRelation [col1#4, col2#5]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#2,c2#3])\n +- Project [cast(col1#0 as int) AS c1#2, cast(col2#1 as int) AS c2#3]\n +- LocalRelation [col1#0, col2#1]\n" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", 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 9725e0d67b0b9..32cd1682287b0 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 @@ -134,7 +134,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "UNSUPPORTED_CORRELATED_REFERENCE", - "messageParameters" : { }, + "messageParameters" : { + "planString" : ": Aggregate [min(outer(t2a#6)) AS min(outer(t2.t2a))#7]\n+- SubqueryAlias t3\n +- View (`t3`, [t3a#3,t3b#4,t3c#5])\n +- Project [cast(t3a#0 as int) AS t3a#3, cast(t3b#1 as int) AS t3b#4, cast(t3c#2 as int) AS t3c#5]\n +- Project [t3a#0, t3b#1, t3c#2]\n +- SubqueryAlias t3\n +- LocalRelation [t3a#0, t3b#1, t3c#2]\n" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 6d744f41aed94..0873e979fa48e 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -103,7 +103,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - "messageParameters" : { }, + "messageParameters" : { + "planString" : ": (cast(udf(cast(k#0 as string)) as string) = cast(udf(cast(outer(k#1) as string)) as string))" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", 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 5798d250d20b8..1ce4f99af3ee7 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 @@ -569,7 +569,7 @@ class SubquerySuite extends QueryTest exception, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s): .*"), + parameters = Map("planString" -> "(?s).*"), sqlState = None, context = ExpectedContext(fragment = str, start = 0, stop = 68)) } @@ -2020,7 +2020,7 @@ class SubquerySuite extends QueryTest exception, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s): .*"), + parameters = Map("planString" -> "(?s).*"), sqlState = None, context = ExpectedContext(fragment = str, start = 0, stop = 74)) } @@ -2052,7 +2052,7 @@ class SubquerySuite extends QueryTest exception1, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s): .*"), + parameters = Map("planString" -> "(?s).*"), sqlState = None, context = ExpectedContext(fragment = str, start = 0, stop = 100)) } From fd38ae740029eeac7bf830d29b3fcf78461c2f02 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 13 Sep 2022 16:31:53 -0700 Subject: [PATCH 15/19] fix test --- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 68fec2d518048..bd433a29e46d9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -857,12 +857,12 @@ class AnalysisErrorSuite extends AnalysisTest { val t1 = LocalRelation(a, b, d) val t2 = LocalRelation(c) val conditions = Seq( - (abs($"a") === $"c", "abs(a) = outer(c)"), - (abs($"a") <=> $"c", "abs(a) <=> outer(c)"), - ($"a" + 1 === $"c", "(a + 1) = outer(c)"), - ($"a" + $"b" === $"c", "(a + b) = outer(c)"), - ($"a" + $"c" === $"b", "(a + outer(c)) = b"), - (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d AS INT) = outer(c)")) + (abs($"a") === $"c", "abs(a#0) = outer(c#1)"), + (abs($"a") <=> $"c", "abs(a#0) <=> outer(c#1)"), + ($"a" + 1 === $"c", "(a#0 + 1) = outer(c#1)"), + ($"a" + $"b" === $"c", "(a#0 + b#1) = outer(c#2)"), + ($"a" + $"c" === $"b", "(a#0 + outer(c#1)) = b#2"), + (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d#0 AS INT) = outer(c#1)")) conditions.foreach { case (cond, msg) => val plan = Project( ScalarSubquery( From acd1ae2c387dc7f68292c318d0089f061f5208c5 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Wed, 14 Sep 2022 09:37:50 -0700 Subject: [PATCH 16/19] reuse existing plan canonicalize method respond to code review comments update update update simplify reuse existing plan canonicalize method respond to code review comments --- .../main/resources/error/error-classes.json | 14 ++-- .../sql/catalyst/analysis/CheckAnalysis.scala | 27 ++++--- .../spark/sql/catalyst/analysis/package.scala | 70 ------------------- .../expressions/namedExpressions.scala | 8 --- .../analysis/AnalysisErrorSuite.scala | 12 ++-- .../sql-tests/results/join-lateral.sql.out | 16 ++--- .../invalid-correlation.sql.out | 8 +-- .../subq-input-typecheck.sql.out | 12 ++-- .../sql-tests/results/udf/udf-except.sql.out | 8 +-- .../org/apache/spark/sql/SubquerySuite.scala | 49 ++++++------- 10 files changed, 78 insertions(+), 146 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 4e442988863a9..13a4565de6da0 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -719,7 +719,7 @@ "subClass" : { "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED" : { "message" : [ - "Accessing outer query column is not allowed in this location" + "Accessing outer query column is not allowed in this location" ] }, "AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES" : { @@ -729,7 +729,7 @@ }, "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE" : { "message" : [ - "Correlated column is not allowed in predicate" + "Correlated column is not allowed in predicate: " ] }, "CORRELATED_COLUMN_NOT_FOUND" : { @@ -744,7 +744,7 @@ }, "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY" : { "message" : [ - "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" + "Correlated scalar subqueries in the GROUP BY clause must also be in the aggregate expressions" ] }, "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT" : { @@ -759,22 +759,22 @@ }, "NON_DETERMINISTIC_LATERAL_SUBQUERIES" : { "message" : [ - "Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row" + "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" + "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses" ] }, "UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY" : { "message" : [ - "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" + "Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands" ] }, "UNSUPPORTED_IN_EXISTS_SUBQUERY" : { "message" : [ - "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" + "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands" ] } } 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 1711289b3193a..a1b37bd3f5b8f 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.util.Utils /** * Throws user facing errors when passed invalid queries that fail to analyze. @@ -224,7 +225,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case c: Cast if !c.resolved => failAnalysis(s"invalid cast from ${c.child.dataType.catalogString} to " + c.dataType.catalogString) - case e: RuntimeReplaceable if !e.replacement.resolved => throw new IllegalStateException("Illegal RuntimeReplaceable: " + e + "\nReplacement is unresolved: " + e.replacement) @@ -730,6 +730,14 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { } } + private def planToString(expr: LogicalPlan): String = + if (Utils.isTesting) { + expr.toString.replaceAll("#\\d+", "#x") + .replaceAll("operator id = \\d+", "operator id = #x") + } else { + expr.toString + } + /** * Validates subquery expressions in the plan. Upon failure, returns an user facing error. */ @@ -822,7 +830,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - treeNodes = Seq(other)) + messageParameters = Map("treeNode" -> planToString(other))) } // Only certain operators are allowed to host subquery expression containing @@ -837,13 +845,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { a.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - treeNodes = Seq(a)) + messageParameters = Map("treeNode" -> planToString(a))) } case other => other.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", - treeNodes = Seq(other)) + messageParameters = Map("treeNode" -> planToString(other))) } } // Validate to make sure the correlations appearing in the query are valid and @@ -859,7 +867,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "NON_DETERMINISTIC_LATERAL_SUBQUERIES", - treeNodes = Seq(plan)) + messageParameters = Map("treeNode" -> planToString(plan))) } // Check if the lateral join's join condition is deterministic. if (join.condition.exists(!_.deterministic)) { @@ -880,7 +888,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { expr.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_IN_EXISTS_SUBQUERY", - treeNodes = Seq(plan)) + messageParameters = Map("treeNode" -> planToString(plan))) } // Validate to make sure the correlations appearing in the query are valid and // allowed by spark. @@ -949,7 +957,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", - treeNodes = Seq(p)) + messageParameters = Map("treeNode" -> planToString(p))) } } @@ -971,7 +979,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "UNSUPPORTED_CORRELATED_REFERENCE", - treeNodes = Seq(p)) + messageParameters = Map("treeNode" -> planToString(p))) } } @@ -1037,7 +1045,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { p.failAnalysis( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - treeNodes = predicates) + messageParameters = + Map("treeNode" -> s"${predicates.map(_.sql).mkString}:\n${planToString(p)}")) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index d918597d78aa1..9be771c912467 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -17,17 +17,11 @@ package org.apache.spark.sql.catalyst -import java.util.UUID - -import scala.collection.mutable - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.errors.QueryErrorsBase -import org.apache.spark.sql.internal.SQLConf /** * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. @@ -82,70 +76,6 @@ package object analysis { origin = t.origin) } - /** - * Fails the analysis at the point where a specific tree node was parsed using a provided - * error class and subclass and one message parameter comprising a plan string. The plan string - * will be printed in the error message if and only if the corresponding Spark configuration is - * enabled. - */ - def failAnalysis( - errorClass: String, - errorSubClass: String, - treeNodes: Seq[TreeNode[_]]): Nothing = { - // Normalize expression IDs in the query plan to keep tests deterministic. - object IdGen { - private val curId = new java.util.concurrent.atomic.AtomicLong() - private val jvmId = UUID.randomUUID() - private val idMap = mutable.Map.empty[ExprId, ExprId] - def get(previous: ExprId): ExprId = { - if (!idMap.contains(previous)) idMap.put(previous, ExprId(curId.getAndIncrement(), jvmId)) - idMap.get(previous).get - } - } - def normalizeExpr(expr: Expression): Expression = { - expr.withNewChildren(expr.children.map(normalizeExpr)) match { - case a: Attribute => a.withExprId(IdGen.get(a.exprId)) - case a: Alias => a.withExprId(IdGen.get(a.exprId)) - case n: NamedLambdaVariable => n.copy(exprId = IdGen.get(n.exprId)) - case o: OuterReference => o.copy(e = normalizeExpr(o.e).asInstanceOf[NamedExpression]) - case s: ScalarSubquery => s.copy(plan = normalizePlan(s.plan), - exprId = IdGen.get(s.exprId), outerAttrs = s.outerAttrs.map(normalizeExpr), - joinCond = s.joinCond.map(normalizeExpr)) - case s: Exists => s.copy(plan = normalizePlan(s.plan), exprId = IdGen.get(s.exprId), - outerAttrs = s.outerAttrs.map(normalizeExpr), joinCond = s.joinCond.map(normalizeExpr)) - case s: LateralSubquery => s.copy(plan = normalizePlan(s.plan), - exprId = IdGen.get(s.exprId), outerAttrs = s.outerAttrs.map(normalizeExpr), - joinCond = s.joinCond.map(normalizeExpr)) - case s: InSubquery => s.copy(values = s.values.map(normalizeExpr), - query = s.query.copy(plan = normalizePlan(s.query.plan), - exprId = IdGen.get(s.query.exprId), - childOutputs = s.query.childOutputs.map(a => a.withExprId(IdGen.get(a.exprId))), - outerAttrs = s.query.outerAttrs.map(normalizeExpr), - joinCond = s.query.joinCond.map(normalizeExpr))) - case other => other - } - } - def normalizePlan(node: LogicalPlan): LogicalPlan = { - node.withNewChildren(node.children.map(normalizePlan)).mapExpressions(normalizeExpr) - } - val planString = - if (SQLConf.get.includePlansInErrors) { - s": ${ - treeNodes.map { - case plan: LogicalPlan => normalizePlan(plan) - case expr: Expression => normalizeExpr(expr) - }.mkString("\n") - }" - } else { - "" - } - throw new AnalysisException( - errorClass = errorClass, - errorSubClass = errorSubClass, - messageParameters = Map("planString" -> planString), - origin = t.origin) - } - def dataTypeMismatch(expr: Expression, mismatch: DataTypeMismatch): Nothing = { throw new AnalysisException( errorClass = "DATATYPE_MISMATCH", 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 a7eec4774eae0..4181edcb8c601 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 @@ -182,14 +182,6 @@ case class Alias(child: Expression, name: String)( nonInheritableMetadataKeys = nonInheritableMetadataKeys) } - def withExprId(newExprId: ExprId): NamedExpression = { - Alias(child, name)( - exprId = newExprId, - qualifier = qualifier, - explicitMetadata = explicitMetadata, - nonInheritableMetadataKeys = nonInheritableMetadataKeys) - } - def newInstance(): NamedExpression = Alias(child, name)( qualifier = qualifier, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index bd433a29e46d9..68fec2d518048 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -857,12 +857,12 @@ class AnalysisErrorSuite extends AnalysisTest { val t1 = LocalRelation(a, b, d) val t2 = LocalRelation(c) val conditions = Seq( - (abs($"a") === $"c", "abs(a#0) = outer(c#1)"), - (abs($"a") <=> $"c", "abs(a#0) <=> outer(c#1)"), - ($"a" + 1 === $"c", "(a#0 + 1) = outer(c#1)"), - ($"a" + $"b" === $"c", "(a#0 + b#1) = outer(c#2)"), - ($"a" + $"c" === $"b", "(a#0 + outer(c#1)) = b#2"), - (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d#0 AS INT) = outer(c#1)")) + (abs($"a") === $"c", "abs(a) = outer(c)"), + (abs($"a") <=> $"c", "abs(a) <=> outer(c)"), + ($"a" + 1 === $"c", "(a + 1) = outer(c)"), + ($"a" + $"b" === $"c", "(a + b) = outer(c)"), + ($"a" + $"c" === $"b", "(a + outer(c)) = b"), + (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d AS INT) = outer(c)")) conditions.foreach { case (cond, msg) => val plan = Project( ScalarSubquery( 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 c08d48eb802b0..a3d9a6a6c1db3 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 @@ -321,14 +321,14 @@ org.apache.spark.sql.AnalysisException "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", "messageParameters" : { - "planString" : ": !LateralJoin lateral-subquery#5 [c1#6 && c2#7], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#2) + outer(c2#3)) as double) + rand(0)) AS c3#4]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#2,c2#3])\n +- Project [cast(col1#0 as int) AS c1#2, cast(col2#1 as int) AS c2#3]\n +- LocalRelation [col1#0, col2#1]\n" + "treeNode" : "LateralJoin lateral-subquery#x [c1#x && c2#x], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(0)) AS c3#x]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 10, "stopIndex" : 58, - "fragment" : "SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)" + "fragment" : "FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)" } ] } @@ -343,14 +343,14 @@ org.apache.spark.sql.AnalysisException "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "NON_DETERMINISTIC_LATERAL_SUBQUERIES", "messageParameters" : { - "planString" : ": LateralJoin lateral-subquery#9 [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(0) AS rand(0)#8]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#6,c2#7])\n: +- Project [cast(col1#4 as int) AS c1#6, cast(col2#5 as int) AS c2#7]\n: +- LocalRelation [col1#4, col2#5]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#2,c2#3])\n +- Project [cast(col1#0 as int) AS c1#2, cast(col2#1 as int) AS c2#3]\n +- LocalRelation [col1#0, col2#1]\n" + "treeNode" : "LateralJoin lateral-subquery#x [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(0) AS rand(0)#x]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 10, "stopIndex" : 50, - "fragment" : "SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2)" + "fragment" : "FROM t1, LATERAL (SELECT rand(0) FROM t2)" } ] } @@ -370,9 +370,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 55, "stopIndex" : 76, - "fragment" : "SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1" + "fragment" : "t1.c1 + rand(0) = s.c1" } ] } 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 9766add3dbb76..f86c6784da535 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 @@ -135,14 +135,14 @@ org.apache.spark.sql.AnalysisException "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "UNSUPPORTED_CORRELATED_REFERENCE", "messageParameters" : { - "planString" : ": Aggregate [min(outer(t2a#6)) AS min(outer(t2.t2a))#7]\n+- SubqueryAlias t3\n +- View (`t3`, [t3a#3,t3b#4,t3c#5])\n +- Project [cast(t3a#0 as int) AS t3a#3, cast(t3b#1 as int) AS t3b#4, cast(t3c#2 as int) AS t3c#5]\n +- Project [t3a#0, t3b#1, t3c#2]\n +- SubqueryAlias t3\n +- LocalRelation [t3a#0, t3b#1, t3c#2]\n" + "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" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 162, - "fragment" : "SELECT t1a \nFROM t1\nWHERE t1a IN (SELECT t2a \n FROM t2\n WHERE EXISTS (SELECT min(t2a) \n FROM t3))" + "startIndex" : 105, + "stopIndex" : 160, + "fragment" : "SELECT min(t2a) \n FROM t3" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index 30d50b6fe1ff4..34e6bbf19fd8f 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -70,9 +70,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 109, - "fragment" : "SELECT \n ( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b = t1.t1b\n GROUP BY t2.t2b\n )\nFROM t1" + "startIndex" : 11, + "stopIndex" : 101, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b = t1.t1b\n GROUP BY t2.t2b\n )" } ] } @@ -98,9 +98,9 @@ org.apache.spark.sql.AnalysisException "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 104, - "fragment" : "SELECT \n ( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b > 0\n GROUP BY t2.t2b\n )\nFROM t1" + "startIndex" : 11, + "stopIndex" : 96, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b > 0\n GROUP BY t2.t2b\n )" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 0873e979fa48e..7393bb7f3e567 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -104,13 +104,13 @@ org.apache.spark.sql.AnalysisException "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", "messageParameters" : { - "planString" : ": (cast(udf(cast(k#0 as string)) as string) = cast(udf(cast(outer(k#1) as string)) as string))" + "treeNode" : "(CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#440) as string)) AS STRING)):\nAggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS udf(max(udf(v)))#x]\n+- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string))\n +- SubqueryAlias t2\n +- View (`t2`, [k#x,v#x])\n +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x]\n +- Project [k#x, v#x]\n +- SubqueryAlias t2\n +- LocalRelation [k#x, v#x]\n" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, - "stopIndex" : 281, - "fragment" : "SELECT t1.k\nFROM t1\nWHERE t1.v <= (SELECT udf(max(udf(t2.v)))\n FROM t2\n WHERE udf(t2.k) = udf(t1.k))\nMINUS\nSELECT t1.k\nFROM t1\nWHERE udf(t1.v) >= (SELECT min(udf(t2.v))\n FROM t2\n WHERE t2.k = t1.k)" + "startIndex" : 39, + "stopIndex" : 141, + "fragment" : "SELECT udf(max(udf(t2.v)))\n FROM t2\n WHERE udf(t2.k) = udf(t1.k)" } ] } 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 df4e90050db30..b6b65c66bef9a 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 @@ -522,9 +522,8 @@ class SubquerySuite extends QueryTest withTempView("t") { Seq((1, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") - val str = "select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1" val exception = intercept[AnalysisException] { - sql(str) + sql("select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1") } checkError( exception, @@ -532,24 +531,25 @@ class SubquerySuite extends QueryTest errorSubClass = "NON_CORRELATED_COLUMNS_IN_GROUP_BY", parameters = Map("value" -> "c2"), sqlState = None, - context = ExpectedContext(fragment = str, start = 0, stop = 81)) } + context = ExpectedContext( + fragment = "(select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2)", + start = 7, stop = 67)) } } test("non-aggregated correlated scalar subquery") { - val str1 = "select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1" val exception1 = intercept[AnalysisException] { - sql(str1) + sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") } checkErrorMatchPVals( exception1, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY", - parameters = Map("planString" -> "(?s): Filter .*"), + parameters = Map("treeNode" -> "(?s)Filter .*"), sqlState = None, - context = ExpectedContext(fragment = str1, start = 0, stop = 63)) - val str2 = "select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1" + context = ExpectedContext( + fragment = "(select b from l l2 where l2.a = l1.a)", start = 10, stop = 47)) val exception2 = intercept[AnalysisException] { - sql(str2) + sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") } checkErrorMatchPVals( exception2, @@ -557,21 +557,22 @@ class SubquerySuite extends QueryTest errorSubClass = "MUST_AGGREGATE_CORRELATED_SCALAR_SUBQUERY_OUTPUT", parameters = Map.empty[String, String], sqlState = None, - context = ExpectedContext(fragment = str2, start = 0, stop = 74)) + context = ExpectedContext( + fragment = "(select b from l l2 where l2.a = l1.a group by 1)", start = 10, stop = 58)) } test("non-equal correlated scalar subquery") { - val str = "select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1" val exception = intercept[AnalysisException] { - sql(str) + sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") } checkErrorMatchPVals( exception, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s).*"), + parameters = Map("treeNode" -> "(?s).*"), sqlState = None, - context = ExpectedContext(fragment = str, start = 0, stop = 68)) + context = ExpectedContext( + fragment = "select sum(b) from l l2 where l2.a < l1.a", start = 11, stop = 51)) } test("disjunctive correlated scalar subquery") { @@ -2014,17 +2015,17 @@ class SubquerySuite extends QueryTest withTempView("t1", "t2") { Seq((0, 1)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 2), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") - val str = "SELECT * FROM t1 JOIN LATERAL (SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1)" val exception = intercept[AnalysisException] { - sql(str) + sql("SELECT * FROM t1 JOIN LATERAL (SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1)") } checkErrorMatchPVals( exception, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s).*"), + parameters = Map("treeNode" -> "(?s).*"), sqlState = None, - context = ExpectedContext(fragment = str, start = 0, stop = 74)) + context = ExpectedContext( + fragment = "SELECT DISTINCT c2 FROM t2 WHERE c1 > t1.c1", start = 31, stop = 73)) } } @@ -2044,19 +2045,19 @@ class SubquerySuite extends QueryTest |FROM (SELECT CAST(c1 AS STRING) a FROM t1) |""".stripMargin), Row(5) :: Row(null) :: Nil) - val str = - """SELECT (SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a) - |FROM (SELECT CAST(c1 AS SHORT) a FROM t1)""".stripMargin val exception1 = intercept[AnalysisException] { - sql(str) + sql( + """SELECT (SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a) + |FROM (SELECT CAST(c1 AS SHORT) a FROM t1)""".stripMargin) } checkErrorMatchPVals( exception1, errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", - parameters = Map("planString" -> "(?s).*"), + parameters = Map("treeNode" -> "(?s).*"), sqlState = None, - context = ExpectedContext(fragment = str, start = 0, stop = 100)) + context = ExpectedContext( + fragment = "SELECT SUM(c2) FROM t2 WHERE CAST(c1 AS SHORT) = a", start = 8, stop = 57)) } } From eec1ed07321b0ed409ffca43f138fff34085fa27 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 19 Sep 2022 14:53:18 -0700 Subject: [PATCH 17/19] fix test again --- .../sql/catalyst/analysis/CheckAnalysis.scala | 21 ++++++++++++------- .../sql-tests/results/udf/udf-except.sql.out | 2 +- 2 files changed, 14 insertions(+), 9 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 a1b37bd3f5b8f..63a51106d7931 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 @@ -730,13 +730,18 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { } } - private def planToString(expr: LogicalPlan): String = - if (Utils.isTesting) { - expr.toString.replaceAll("#\\d+", "#x") - .replaceAll("operator id = \\d+", "operator id = #x") - } else { - expr.toString - } + private def scrubOutIds(string: String): String = + string.replaceAll("#\\d+", "#x") + .replaceAll("operator id = \\d+", "operator id = #x") + + private def planToString(plan: LogicalPlan): String = { + if (Utils.isTesting) scrubOutIds(plan.toString) else plan.toString + } + + private def exprsToString(exprs: Seq[Expression]): String = { + val result = exprs.map(_.toString).mkString("\n") + if (Utils.isTesting) scrubOutIds(result) else result + } /** * Validates subquery expressions in the plan. Upon failure, returns an user facing error. @@ -1046,7 +1051,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", errorSubClass = "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", messageParameters = - Map("treeNode" -> s"${predicates.map(_.sql).mkString}:\n${planToString(p)}")) + Map("treeNode" -> s"${exprsToString(predicates)}\n${planToString(p)}")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 7393bb7f3e567..03e4ac3d57b66 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -104,7 +104,7 @@ org.apache.spark.sql.AnalysisException "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY", "errorSubClass" : "CORRELATED_COLUMN_IS_NOT_ALLOWED_IN_PREDICATE", "messageParameters" : { - "treeNode" : "(CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#440) as string)) AS STRING)):\nAggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS udf(max(udf(v)))#x]\n+- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string))\n +- SubqueryAlias t2\n +- View (`t2`, [k#x,v#x])\n +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x]\n +- Project [k#x, v#x]\n +- SubqueryAlias t2\n +- LocalRelation [k#x, v#x]\n" + "treeNode" : "(cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string))\nAggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS udf(max(udf(v)))#x]\n+- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string))\n +- SubqueryAlias t2\n +- View (`t2`, [k#x,v#x])\n +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x]\n +- Project [k#x, v#x]\n +- SubqueryAlias t2\n +- LocalRelation [k#x, v#x]\n" }, "queryContext" : [ { "objectType" : "", From 9b33b03f5e4e687597aa5c9fcbafe8eaa0eef37a Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Mon, 19 Sep 2022 14:57:38 -0700 Subject: [PATCH 18/19] fix test again --- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 68fec2d518048..72b721a3ff3a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -857,12 +857,12 @@ class AnalysisErrorSuite extends AnalysisTest { val t1 = LocalRelation(a, b, d) val t2 = LocalRelation(c) val conditions = Seq( - (abs($"a") === $"c", "abs(a) = outer(c)"), - (abs($"a") <=> $"c", "abs(a) <=> outer(c)"), - ($"a" + 1 === $"c", "(a + 1) = outer(c)"), - ($"a" + $"b" === $"c", "(a + b) = outer(c)"), - ($"a" + $"c" === $"b", "(a + outer(c)) = b"), - (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d AS INT) = outer(c)")) + (abs($"a") === $"c", "abs(a#x) = outer(c#x)"), + (abs($"a") <=> $"c", "abs(a#x) <=> outer(c#x)"), + ($"a" + 1 === $"c", "(a#x + 1) = outer(c#x)"), + ($"a" + $"b" === $"c", "(a#x + b#x) = outer(c#x)"), + ($"a" + $"c" === $"b", "(a#x + outer(c#x)) = b#x"), + (And($"a" === $"c", Cast($"d", IntegerType) === $"c"), "CAST(d#x AS INT) = outer(c#x)")) conditions.foreach { case (cond, msg) => val plan = Project( ScalarSubquery( From 263b36b0da19a7341ca91a4f228b85de99a0734a Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Tue, 20 Sep 2022 09:14:41 -0700 Subject: [PATCH 19/19] respond to code review comments --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 11 ----------- 1 file changed, 11 deletions(-) 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 ed4e18b0e85ed..de25c19a26eb8 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 @@ -3911,15 +3911,6 @@ object SQLConf { .checkValues(ErrorMessageFormat.values.map(_.toString)) .createWithDefault(ErrorMessageFormat.PRETTY.toString) - val INCLUDE_PLANS_IN_ERRORS = buildConf("spark.sql.error.includePlans") - .doc("If true, include the string representation of query plans or expressions in error " + - "messages when the origins of these entities are also available. Otherwise, elide these " + - "strings from the error messages since these origins convey the source locations of the " + - "erroneous behavior.") - .version("3.4.0") - .booleanConf - .createWithDefault(true) - /** * Holds information about keys that have been deprecated. * @@ -4714,8 +4705,6 @@ class SQLConf extends Serializable with Logging { def errorMessageFormat: ErrorMessageFormat.Value = ErrorMessageFormat.withName(getConf(SQLConf.ERROR_MESSAGE_FORMAT)) - def includePlansInErrors: Boolean = getConf(SQLConf.INCLUDE_PLANS_IN_ERRORS) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */