You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/01/26 20:23:16 UTC

[GitHub] [spark] imback82 commented on a change in pull request #31352: [SPARK-34252][SQL] Subquery with view in aggregate's grouping expression fails during the analysis check

imback82 commented on a change in pull request #31352:
URL: https://github.com/apache/spark/pull/31352#discussion_r564807595



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala
##########
@@ -53,38 +53,42 @@ import org.apache.spark.sql.catalyst.rules.Rule
  * completely resolved during the batch of Resolution.
  */
 object EliminateView extends Rule[LogicalPlan] with CastSupport {
-  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    // The child has the different output attributes with the View operator. Adds a Project over
-    // the child of the view.
-    case v @ View(desc, _, output, child) if child.resolved && !v.sameOutput(child) =>
-      val resolver = conf.resolver
-      val queryColumnNames = desc.viewQueryColumnNames
-      val queryOutput = if (queryColumnNames.nonEmpty) {
-        // Find the attribute that has the expected attribute name from an attribute list, the names
-        // are compared using conf.resolver.
-        // `CheckAnalysis` already guarantees the expected attribute can be found for sure.
-        desc.viewQueryColumnNames.map { colName =>
-          child.output.find(attr => resolver(attr.name, colName)).get
-        }
-      } else {
-        // For view created before Spark 2.2.0, the view text is already fully qualified, the plan
-        // output is the same with the view output.
-        child.output
-      }
-      // Map the attributes in the query output to the attributes in the view output by index.
-      val newOutput = output.zip(queryOutput).map {
-        case (attr, originAttr) if !attr.semanticEquals(originAttr) =>
-          // `CheckAnalysis` already guarantees that the cast is a up-cast for sure.
-          Alias(cast(originAttr, attr.dataType), attr.name)(exprId = attr.exprId,
-            qualifier = attr.qualifier, explicitMetadata = Some(attr.metadata))
-        case (_, originAttr) => originAttr
-      }
-      Project(newOutput, child)
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    AnalysisHelper.allowInvokingTransformsInAnalyzer {
+      plan transformUp {

Review comment:
       There is no change below this line. If you don't like the diff, I can change the PR to something like the following:
   ```scala
     override def apply(plan: LogicalPlan): LogicalPlan = {
       AnalysisHelper.allowInvokingTransformsInAnalyzer {
         applyInternal(plan)
       }
     }
   
     private def applyInternal(plan: LogicalPlan): LogicalPlan = plan transformUp {
   ```

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
##########
@@ -512,6 +512,18 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       Row(1, null) :: Row(2, 6.0) :: Row(3, 2.0) :: Row(null, null) :: Row(6, null) :: Nil)
   }
 
+  test("SPARK-34252: correlated scalar subquery with view in aggregate") {
+    withTable("tr") {
+      withView("vr") {
+        r.write.saveAsTable("tr")
+        sql("create view vr as select * from tr")
+        checkAnswer(
+          sql("select a, (select sum(d) from vr where a = c) sum_d from l l1 group by 1, 2"),

Review comment:
       Note that the existing test with the same query worked fine because `r` is a dataframe temp view, which doesn't have the `View` node.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org