You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2016/09/01 00:55:00 UTC
spark git commit: [SPARK-17180][SPARK-17309][SPARK-17323][SQL][2.0]
create AlterViewAsCommand to handle ALTER VIEW AS
Repository: spark
Updated Branches:
refs/heads/branch-2.0 8d15c1a6a -> 191d99692
[SPARK-17180][SPARK-17309][SPARK-17323][SQL][2.0] create AlterViewAsCommand to handle ALTER VIEW AS
## What changes were proposed in this pull request?
Currently we use `CreateViewCommand` to implement ALTER VIEW AS, which has 3 bugs:
1. SPARK-17180: ALTER VIEW AS should alter temp view if view name has no database part and temp view exists
2. SPARK-17309: ALTER VIEW AS should issue exception if view does not exist.
3. SPARK-17323: ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.
The root cause is, ALTER VIEW AS is quite different from CREATE VIEW, we need different code path to handle them. However, in `CreateViewCommand`, there is no way to distinguish ALTER VIEW AS and CREATE VIEW, we have to introduce extra flag. But instead of doing this, I think a more natural way is to separate the ALTER VIEW AS logic into a new command.
backport https://github.com/apache/spark/pull/14874 to 2.0
## How was this patch tested?
new tests in SQLViewSuite
Author: Wenchen Fan <we...@databricks.com>
Closes #14893 from cloud-fan/minor4.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/191d9969
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/191d9969
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/191d9969
Branch: refs/heads/branch-2.0
Commit: 191d99692dc4315c371b566e3a9c5b114876ee49
Parents: 8d15c1a
Author: Wenchen Fan <we...@databricks.com>
Authored: Thu Sep 1 08:54:59 2016 +0800
Committer: Wenchen Fan <we...@databricks.com>
Committed: Thu Sep 1 08:54:59 2016 +0800
----------------------------------------------------------------------
.../spark/sql/execution/SparkSqlParser.scala | 74 ++++++++------------
.../spark/sql/execution/command/views.scala | 71 +++++++++++++++++--
.../spark/sql/hive/execution/SQLViewSuite.scala | 71 +++++++++++++++++++
3 files changed, 167 insertions(+), 49 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/191d9969/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index 876b334..3072a6d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -1250,60 +1250,44 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
val schema = identifiers.map { ic =>
CatalogColumn(ic.identifier.getText, null, nullable = true, Option(ic.STRING).map(string))
}
- createView(
- ctx,
- ctx.tableIdentifier,
- comment = Option(ctx.STRING).map(string),
- schema,
- ctx.query,
- Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty),
- ctx.EXISTS != null,
- ctx.REPLACE != null,
- ctx.TEMPORARY != null
- )
+
+ val sql = Option(source(ctx.query))
+ val tableDesc = CatalogTable(
+ identifier = visitTableIdentifier(ctx.tableIdentifier),
+ tableType = CatalogTableType.VIEW,
+ schema = schema,
+ storage = CatalogStorageFormat.empty,
+ properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty),
+ viewOriginalText = sql,
+ viewText = sql,
+ comment = Option(ctx.STRING).map(string))
+
+ CreateViewCommand(
+ tableDesc,
+ plan(ctx.query),
+ allowExisting = ctx.EXISTS != null,
+ replace = ctx.REPLACE != null,
+ isTemporary = ctx.TEMPORARY != null)
}
}
/**
- * Alter the query of a view. This creates a [[CreateViewCommand]] command.
+ * Alter the query of a view. This creates a [[AlterViewAsCommand]] command.
+ *
+ * For example:
+ * {{{
+ * ALTER VIEW [db_name.]view_name AS SELECT ...;
+ * }}}
*/
override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
- createView(
- ctx,
- ctx.tableIdentifier,
- comment = None,
- Seq.empty,
- ctx.query,
- Map.empty,
- allowExist = false,
- replace = true,
- isTemporary = false)
- }
-
- /**
- * Create a [[CreateViewCommand]] command.
- */
- private def createView(
- ctx: ParserRuleContext,
- name: TableIdentifierContext,
- comment: Option[String],
- schema: Seq[CatalogColumn],
- query: QueryContext,
- properties: Map[String, String],
- allowExist: Boolean,
- replace: Boolean,
- isTemporary: Boolean): LogicalPlan = {
- val sql = Option(source(query))
val tableDesc = CatalogTable(
- identifier = visitTableIdentifier(name),
+ identifier = visitTableIdentifier(ctx.tableIdentifier),
tableType = CatalogTableType.VIEW,
- schema = schema,
storage = CatalogStorageFormat.empty,
- properties = properties,
- viewOriginalText = sql,
- viewText = sql,
- comment = comment)
- CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary)
+ schema = Nil,
+ viewOriginalText = Option(source(ctx.query)))
+
+ AlterViewAsCommand(tableDesc, plan(ctx.query))
}
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/191d9969/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
index 9e263d1..125b3d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
@@ -22,7 +22,7 @@ import scala.util.control.NonFatal
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute}
+import org.apache.spark.sql.catalyst.expressions.Alias
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
@@ -56,8 +56,6 @@ case class CreateViewCommand(
// TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is
// different from Hive and may not work for some cases like create view on self join.
- override def output: Seq[Attribute] = Seq.empty[Attribute]
-
require(tableDesc.tableType == CatalogTableType.VIEW,
"The type of the table to created with CREATE VIEW must be 'CatalogTableType.VIEW'.")
if (!isTemporary) {
@@ -191,7 +189,7 @@ case class CreateViewCommand(
sparkSession.sql(viewSQL).queryExecution.assertAnalyzed()
} catch {
case NonFatal(e) =>
- throw new RuntimeException(s"Failed to analyze the canonicalized SQL: ${viewSQL}", e)
+ throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e)
}
val viewSchema: Seq[CatalogColumn] = {
@@ -212,3 +210,68 @@ case class CreateViewCommand(
/** Escape backtick with double-backtick in column name and wrap it with backtick. */
private def quote(name: String) = s"`${name.replaceAll("`", "``")}`"
}
+
+/**
+ * Alter a view with given query plan. If the view name contains database prefix, this command will
+ * alter a permanent view matching the given name, or throw an exception if view not exist. Else,
+ * this command will try to alter a temporary view first, if view not exist, try permanent view
+ * next, if still not exist, throw an exception.
+ *
+ * @param tableDesc the catalog table
+ * @param query the logical plan that represents the view; this is used to generate a canonicalized
+ * version of the SQL that can be saved in the catalog.
+ */
+case class AlterViewAsCommand(
+ tableDesc: CatalogTable,
+ query: LogicalPlan) extends RunnableCommand {
+
+ override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query)
+
+ override def run(session: SparkSession): Seq[Row] = {
+ // If the plan cannot be analyzed, throw an exception and don't proceed.
+ val qe = session.sessionState.executePlan(query)
+ qe.assertAnalyzed()
+ val analyzedPlan = qe.analyzed
+
+ if (session.sessionState.catalog.isTemporaryTable(tableDesc.identifier)) {
+ session.sessionState.catalog.createTempView(
+ tableDesc.identifier.table,
+ analyzedPlan,
+ overrideIfExists = true)
+ } else {
+ alterPermanentView(session, analyzedPlan)
+ }
+
+ Seq.empty[Row]
+ }
+
+ private def alterPermanentView(session: SparkSession, analyzedPlan: LogicalPlan): Unit = {
+ val viewMeta = session.sessionState.catalog.getTableMetadata(tableDesc.identifier)
+ if (viewMeta.tableType != CatalogTableType.VIEW) {
+ throw new AnalysisException(s"${viewMeta.identifier} is not a view.")
+ }
+
+ val viewSQL: String = new SQLBuilder(analyzedPlan).toSQL
+ // Validate the view SQL - make sure we can parse it and analyze it.
+ // If we cannot analyze the generated query, there is probably a bug in SQL generation.
+ try {
+ session.sql(viewSQL).queryExecution.assertAnalyzed()
+ } catch {
+ case NonFatal(e) =>
+ throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e)
+ }
+
+ val viewSchema: Seq[CatalogColumn] = {
+ analyzedPlan.output.map { a =>
+ CatalogColumn(a.name, a.dataType.catalogString)
+ }
+ }
+
+ val updatedViewMeta = viewMeta.copy(
+ schema = viewSchema,
+ viewOriginalText = tableDesc.viewOriginalText,
+ viewText = Some(viewSQL))
+
+ session.sessionState.catalog.alterTable(updatedViewMeta)
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/191d9969/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
index 39846f1..c6eeb8a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala
@@ -18,6 +18,8 @@
package org.apache.spark.sql.hive.execution
import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils
@@ -174,6 +176,75 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
}
+ test("should not allow ALTER VIEW AS when the view does not exist") {
+ intercept[NoSuchTableException](
+ sql("ALTER VIEW testView AS SELECT 1, 2")
+ )
+
+ intercept[NoSuchTableException](
+ sql("ALTER VIEW default.testView AS SELECT 1, 2")
+ )
+ }
+
+ test("ALTER VIEW AS should try to alter temp view first if view name has no database part") {
+ withView("test_view") {
+ withTempView("test_view") {
+ sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b")
+ sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b")
+
+ sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j")
+
+ // The temporary view should be updated.
+ checkAnswer(spark.table("test_view"), Row(3, 4))
+
+ // The permanent view should stay same.
+ checkAnswer(spark.table("default.test_view"), Row(1, 2))
+ }
+ }
+ }
+
+ test("ALTER VIEW AS should alter permanent view if view name has database part") {
+ withView("test_view") {
+ withTempView("test_view") {
+ sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b")
+ sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b")
+
+ sql("ALTER VIEW default.test_view AS SELECT 3 AS i, 4 AS j")
+
+ // The temporary view should stay same.
+ checkAnswer(spark.table("test_view"), Row(1, 2))
+
+ // The permanent view should be updated.
+ checkAnswer(spark.table("default.test_view"), Row(3, 4))
+ }
+ }
+ }
+
+ test("ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.") {
+ withView("test_view") {
+ sql(
+ """
+ |CREATE VIEW test_view
+ |COMMENT 'test'
+ |TBLPROPERTIES ('key' = 'a')
+ |AS SELECT 1 AS a, 2 AS b
+ """.stripMargin)
+
+ val catalog = spark.sessionState.catalog
+ val viewMeta = catalog.getTableMetadata(TableIdentifier("test_view"))
+ assert(viewMeta.properties("comment") == "test")
+ assert(viewMeta.properties("key") == "a")
+
+ sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j")
+ val updatedViewMeta = catalog.getTableMetadata(TableIdentifier("test_view"))
+ assert(updatedViewMeta.properties("comment") == "test")
+ assert(updatedViewMeta.properties("key") == "a")
+ assert(updatedViewMeta.createTime == viewMeta.createTime)
+ // The view should be updated.
+ checkAnswer(spark.table("test_view"), Row(3, 4))
+ }
+ }
+
Seq(true, false).foreach { enabled =>
val prefix = (if (enabled) "With" else "Without") + " canonical native view: "
test(s"$prefix correctly handle CREATE OR REPLACE VIEW") {
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org