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/28 07:21:02 UTC

[GitHub] [spark] viirya commented on a change in pull request #31368: [SPARK-34269][SQL] Simplify SQL view resolution

viirya commented on a change in pull request #31368:
URL: https://github.com/apache/spark/pull/31368#discussion_r565867620



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##########
@@ -844,18 +844,41 @@ class SessionCatalog(
     }
   }
 
+  def getTempViewSchema(plan: LogicalPlan): StructType = {
+    plan match {
+      case viewInfo: TemporaryViewRelation => viewInfo.tableMeta.schema
+      case v => v.schema
+    }
+  }
+
   private def fromCatalogTable(metadata: CatalogTable, isTempView: Boolean): View = {
-    val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text."))
+    val viewText = metadata.viewText.getOrElse {
+      throw new IllegalStateException("Invalid view without text.")
+    }
     val viewConfigs = metadata.viewSQLConfigs
-    val viewPlan =
+    val parsedPlan =
       SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView = isTempView)) {
         parser.parsePlan(viewText)
       }
-    View(
-      desc = metadata,
-      isTempView = isTempView,
-      output = metadata.schema.toAttributes,
-      child = viewPlan)
+    val viewColumnNames = metadata.viewQueryColumnNames
+    val viewPlan = if (viewColumnNames.nonEmpty) {
+      assert(viewColumnNames.length == metadata.schema.length)
+      // For view queries like `SELECT * FROM t`, the schema of the referenced table/view may
+      // change after the view has been created. We need to add an extra SELECT to pick the columns
+      // according to the recorded column names (to get the correct view column ordering and omit
+      // the extra columns that we don't require), add UpCast (to make sure the type change is
+      // safe) and Alias according to the schema in the catalog.
+      val projectList = viewColumnNames.zip(metadata.schema).map { case (col, field) =>
+        Alias(UpCast(UnresolvedAttribute.quoted(col), field.dataType), field.name)(
+          explicitMetadata = Some(field.metadata))
+      }
+      Project(projectList, parsedPlan)
+    } 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.
+      parsedPlan

Review comment:
       For the issue "the schema of the referenced table/view is changed ...", doesn't this also suffer from it too? The view text is fully qualified doesn't mean it has no problem that the referenced table/view changes schema. Isn't?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##########
@@ -845,17 +845,33 @@ class SessionCatalog(
   }
 
   private def fromCatalogTable(metadata: CatalogTable, isTempView: Boolean): View = {
-    val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text."))
+    val viewText = metadata.viewText.getOrElse {
+      throw new IllegalStateException("Invalid view without text.")
+    }
     val viewConfigs = metadata.viewSQLConfigs
-    val viewPlan =
+    val parsedPlan =
       SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView = isTempView)) {
         parser.parsePlan(viewText)
       }
-    View(
-      desc = metadata,
-      isTempView = isTempView,
-      output = metadata.schema.toAttributes,
-      child = viewPlan)
+    val viewColumnNames = metadata.viewQueryColumnNames
+    val viewPlan = if (viewColumnNames.nonEmpty) {
+      assert(viewColumnNames.length == metadata.schema.length)
+      // For view queries like `SELECT * FROM t`, the schema of the referenced table/view may
+      // change after the view has been created. We need to add an extra SELECT to pick the columns

Review comment:
       Hm? Is the comment "For view queries like `SELECT * FROM t`..." copied in this PR? I don't see its original place here.
   
   This code seems copied from `EliminateView`, but its original comment is different. The `EliminateView`'s comment is more about resolution of attribute of view text.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##########
@@ -844,18 +844,41 @@ class SessionCatalog(
     }
   }
 
+  def getTempViewSchema(plan: LogicalPlan): StructType = {
+    plan match {
+      case viewInfo: TemporaryViewRelation => viewInfo.tableMeta.schema
+      case v => v.schema
+    }
+  }
+
   private def fromCatalogTable(metadata: CatalogTable, isTempView: Boolean): View = {
-    val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text."))
+    val viewText = metadata.viewText.getOrElse {
+      throw new IllegalStateException("Invalid view without text.")
+    }
     val viewConfigs = metadata.viewSQLConfigs
-    val viewPlan =
+    val parsedPlan =
       SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView = isTempView)) {
         parser.parsePlan(viewText)
       }
-    View(
-      desc = metadata,
-      isTempView = isTempView,
-      output = metadata.schema.toAttributes,
-      child = viewPlan)
+    val viewColumnNames = metadata.viewQueryColumnNames
+    val viewPlan = if (viewColumnNames.nonEmpty) {
+      assert(viewColumnNames.length == metadata.schema.length)
+      // For view queries like `SELECT * FROM t`, the schema of the referenced table/view may
+      // change after the view has been created. We need to add an extra SELECT to pick the columns
+      // according to the recorded column names (to get the correct view column ordering and omit
+      // the extra columns that we don't require), add UpCast (to make sure the type change is
+      // safe) and Alias according to the schema in the catalog.
+      val projectList = viewColumnNames.zip(metadata.schema).map { case (col, field) =>
+        Alias(UpCast(UnresolvedAttribute.quoted(col), field.dataType), field.name)(
+          explicitMetadata = Some(field.metadata))
+      }
+      Project(projectList, parsedPlan)

Review comment:
       If the child plan's output is same as view's schema, this projection will be removed by optimization, right?




----------------------------------------------------------------
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