Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -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] = {
Expand All @@ -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)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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") {
Expand Down