Repository: spark Updated Branches: refs/heads/branch-2.0 c393d86d1 -> 22216d6bd
[SPARK-17502][17609][SQL][BACKPORT][2.0] Fix Multiple Bugs in DDL Statements on Temporary Views ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/15054 and https://github.com/apache/spark/pull/15160 to Spark 2.0. - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example, ``` Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`'; ``` - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example, ``` Attempted to unset non-existent property 'p' in table '`testView`'; ``` - When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error: ``` ANALYZE TABLE is not supported for Project ``` - When inserting into a temporary view that is generated from `Range`, we will get the following error message: ``` assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false +- Project [1 AS 1#20] +- OneRowRelation$ ``` This PR is to fix the above four issues. There is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks. ### How was this patch tested? Added multiple test cases Author: gatorsmile <gatorsm...@gmail.com> Closes #15174 from gatorsmile/PR15054Backport. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/22216d6b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/22216d6b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/22216d6b Branch: refs/heads/branch-2.0 Commit: 22216d6bd4270095f175d9f4333afe07e07a7303 Parents: c393d86 Author: gatorsmile <gatorsm...@gmail.com> Authored: Fri Sep 23 09:56:40 2016 +0800 Committer: Wenchen Fan <wenc...@databricks.com> Committed: Fri Sep 23 09:56:40 2016 +0800 ---------------------------------------------------------------------- .../sql/catalyst/analysis/CheckAnalysis.scala | 1 + .../sql/catalyst/catalog/SessionCatalog.scala | 101 +++++++++-------- .../catalyst/catalog/SessionCatalogSuite.scala | 37 +++---- .../org/apache/spark/sql/DataFrameWriter.scala | 9 +- .../execution/command/AnalyzeTableCommand.scala | 5 +- .../command/createDataSourceTables.scala | 13 +-- .../spark/sql/execution/command/ddl.scala | 73 +++++------- .../spark/sql/execution/command/tables.scala | 110 +++++++------------ .../apache/spark/sql/internal/CatalogImpl.scala | 2 +- .../spark/sql/execution/command/DDLSuite.scala | 4 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../sql/hive/execution/HiveCommandSuite.scala | 17 ++- .../spark/sql/hive/execution/HiveDDLSuite.scala | 6 +- .../spark/sql/hive/execution/SQLViewSuite.scala | 59 ++++++++-- 14 files changed, 219 insertions(+), 220 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala ---------------------------------------------------------------------- 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 8b87a4e..790566c 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 @@ -342,6 +342,7 @@ trait CheckAnalysis extends PredicateHelper { case InsertIntoTable(t, _, _, _, _) if !t.isInstanceOf[LeafNode] || + t.isInstanceOf[Range] || t == OneRowRelation || t.isInstanceOf[LocalRelation] => failAnalysis(s"Inserting into an RDD-based table is not allowed.") http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala ---------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ecb4dab..f455cc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -246,33 +246,26 @@ class SessionCatalog( } /** - * Retrieve the metadata of an existing metastore table. - * If no database is specified, assume the table is in the current database. - * If the specified table is not found in the database then a [[NoSuchTableException]] is thrown. + * Return whether a table/view with the specified name exists. If no database is specified, check + * with current database. + */ + def tableExists(name: TableIdentifier): Boolean = synchronized { + val db = formatDatabaseName(name.database.getOrElse(currentDb)) + val table = formatTableName(name.table) + externalCatalog.tableExists(db, table) + } + + /** + * Retrieve the metadata of an existing permanent table/view. If no database is specified, + * assume the table/view is in the current database. If the specified table/view is not found + * in the database then a [[NoSuchTableException]] is thrown. */ def getTableMetadata(name: TableIdentifier): CatalogTable = { val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) val table = formatTableName(name.table) - val tid = TableIdentifier(table) - if (isTemporaryTable(name)) { - CatalogTable( - identifier = tid, - tableType = CatalogTableType.VIEW, - storage = CatalogStorageFormat.empty, - schema = tempTables(table).output.map { c => - CatalogColumn( - name = c.name, - dataType = c.dataType.catalogString, - nullable = c.nullable - ) - }, - properties = Map(), - viewText = None) - } else { - requireDbExists(db) - requireTableExists(TableIdentifier(table, Some(db))) - externalCatalog.getTable(db, table) - } + requireDbExists(db) + requireTableExists(TableIdentifier(table, Some(db))) + externalCatalog.getTable(db, table) } /** @@ -369,6 +362,38 @@ class SessionCatalog( // ------------------------------------------------------------- /** + * Retrieve the metadata of an existing temporary view or permanent table/view. + * + * If a database is specified in `name`, this will return the metadata of table/view in that + * database. + * If no database is specified, this will first attempt to get the metadata of a temporary view + * with the same name, then, if that does not exist, return the metadata of table/view in the + * current database. + */ + def getTempViewOrPermanentTableMetadata(name: TableIdentifier): CatalogTable = synchronized { + val table = formatTableName(name.table) + if (name.database.isDefined) { + getTableMetadata(name) + } else { + getTempView(table).map { plan => + CatalogTable( + identifier = TableIdentifier(table), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = plan.output.map { c => + CatalogColumn( + name = c.name, + dataType = c.dataType.catalogString, + nullable = c.nullable + ) + }, + properties = Map(), + viewText = None) + }.getOrElse(getTableMetadata(name)) + } + } + + /** * Rename a table. * * If a database is specified in `oldName`, this will rename the table in that database. @@ -457,24 +482,6 @@ class SessionCatalog( } /** - * Return whether a table with the specified name exists. - * - * Note: If a database is explicitly specified, then this will return whether the table - * exists in that particular database instead. In that case, even if there is a temporary - * table with the same name, we will return false if the specified database does not - * contain the table. - */ - def tableExists(name: TableIdentifier): Boolean = synchronized { - val db = formatDatabaseName(name.database.getOrElse(currentDb)) - val table = formatTableName(name.table) - if (isTemporaryTable(name)) { - true - } else { - externalCatalog.tableExists(db, table) - } - } - - /** * Return whether a table with the specified name is a temporary table. * * Note: The temporary table cache is checked only when database is not @@ -544,11 +551,11 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { - requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) externalCatalog.createPartitions(db, table, parts, ignoreIfExists) } @@ -560,11 +567,11 @@ class SessionCatalog( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { - requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName)) externalCatalog.dropPartitions(db, table, specs, ignoreIfNotExists) } @@ -579,12 +586,12 @@ class SessionCatalog( specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { val tableMetadata = getTableMetadata(tableName) - requireExactMatchedPartitionSpec(specs, tableMetadata) - requireExactMatchedPartitionSpec(newSpecs, tableMetadata) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(specs, tableMetadata) + requireExactMatchedPartitionSpec(newSpecs, tableMetadata) externalCatalog.renamePartitions(db, table, specs, newSpecs) } @@ -598,11 +605,11 @@ class SessionCatalog( * this becomes a no-op. */ def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { - requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) externalCatalog.alterPartitions(db, table, parts) } @@ -611,11 +618,11 @@ class SessionCatalog( * If no database is specified, assume the table is in the current database. */ def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { - requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) externalCatalog.getPartition(db, table, spec) } http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---------------------------------------------------------------------- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 574ed05..399b706 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -423,46 +423,37 @@ class SessionCatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(TableIdentifier("tbl2", Some("db1")))) // If database is explicitly specified, do not check temporary tables val tempTable = Range(1, 10, 1, 10) - catalog.createTempView("tbl3", tempTable, overrideIfExists = false) assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) // If database is not explicitly specified, check the current database catalog.setCurrentDatabase("db2") assert(catalog.tableExists(TableIdentifier("tbl1"))) assert(catalog.tableExists(TableIdentifier("tbl2"))) - assert(catalog.tableExists(TableIdentifier("tbl3"))) - } - test("tableExists on temporary views") { - val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10) - assert(!catalog.tableExists(TableIdentifier("view1"))) - assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) - catalog.createTempView("view1", tempTable, overrideIfExists = false) - assert(catalog.tableExists(TableIdentifier("view1"))) - assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) + catalog.createTempView("tbl3", tempTable, overrideIfExists = false) + // tableExists should not check temp view. + assert(!catalog.tableExists(TableIdentifier("tbl3"))) } - test("getTableMetadata on temporary views") { + test("getTempViewOrPermanentTableMetadata on temporary views") { val catalog = new SessionCatalog(newBasicCatalog()) val tempTable = Range(1, 10, 2, 10) - val m = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1")) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1")) }.getMessage - assert(m.contains("Table or view 'view1' not found in database 'default'")) - val m2 = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }.getMessage - assert(m2.contains("Table or view 'view1' not found in database 'default'")) catalog.createTempView("view1", tempTable, overrideIfExists = false) - assert(catalog.getTableMetadata(TableIdentifier("view1")).identifier.table == "view1") - assert(catalog.getTableMetadata(TableIdentifier("view1")).schema(0).name == "id") + assert(catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier("view1")).identifier.table == "view1") + assert(catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier("view1")).schema(0).name == "id") - val m3 = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }.getMessage - assert(m3.contains("Table or view 'view1' not found in database 'default'")) } test("list tables without pattern") { http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index b448b9e..a4c4a5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -356,12 +356,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def saveAsTable(tableIdent: TableIdentifier): Unit = { - val sessionState = df.sparkSession.sessionState - val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val tableIdentWithDB = tableIdent.copy(database = Some(db)) - // Pass a table identifier with database part, so that `tableExists` won't check temp views - // unexpectedly. - val tableExists = sessionState.catalog.tableExists(tableIdentWithDB) + val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) (tableExists, mode) match { case (true, SaveMode.Ignore) => @@ -380,7 +375,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { mode, extraOptions.toMap, df.logicalPlan) - sessionState.executePlan(cmd).toRdd + df.sparkSession.sessionState.executePlan(cmd).toRdd } } http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 9509b66..0712753 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -22,6 +22,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable, SimpleCatalogRelation} @@ -38,7 +39,9 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val sessionState = sparkSession.sessionState val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) - val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentwithDB = TableIdentifier(tableIdent.table, Some(db)) + val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentwithDB)) relation match { case relation: CatalogRelation if !relation.isInstanceOf[SimpleCatalogRelation] => http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index f282d54..de7d1fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -74,15 +74,11 @@ case class CreateDataSourceTableCommand( } val sessionState = sparkSession.sessionState - val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val tableIdentWithDB = tableIdent.copy(database = Some(db)) - // Pass a table identifier with database part, so that `tableExists` won't check temp views - // unexpectedly. - if (sessionState.catalog.tableExists(tableIdentWithDB)) { + if (sessionState.catalog.tableExists(tableIdent)) { if (ignoreIfExists) { return Seq.empty[Row] } else { - throw new AnalysisException(s"Table ${tableIdentWithDB.unquotedString} already exists.") + throw new AnalysisException(s"Table ${tableIdent.unquotedString} already exists.") } } @@ -200,8 +196,9 @@ case class CreateDataSourceTableAsSelectCommand( // TODO: Check that options from the resolved relation match the relation that we are // inserting into (i.e. using the same compression). - EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(tableIdentWithDB)) match { + // Pass a table identifier with database part, so that `tableExists` won't check temp + // views unexpectedly. + EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentWithDB)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => // check if the file formats match l.relation match { http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 50ffcd4..16deee3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -190,32 +190,25 @@ case class DropTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(tableName)) { - if (!ifExists) { - val objectName = if (isView) "View" else "Table" - throw new AnalysisException(s"$objectName to drop '$tableName' does not exist") - } - } else { - // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view - // issue an exception. - catalog.getTableMetadataOption(tableName).map(_.tableType match { - case CatalogTableType.VIEW if !isView => - throw new AnalysisException( - "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") - case o if o != CatalogTableType.VIEW && isView => - throw new AnalysisException( - s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") - case _ => - }) - try { - sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession.table(tableName.quotedString)) - } catch { - case NonFatal(e) => log.warn(e.toString, e) - } - catalog.refreshTable(tableName) - catalog.dropTable(tableName, ifExists) + // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view + // issue an exception. + catalog.getTableMetadataOption(tableName).map(_.tableType match { + case CatalogTableType.VIEW if !isView => + throw new AnalysisException( + "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") + case o if o != CatalogTableType.VIEW && isView => + throw new AnalysisException( + s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") + case _ => + }) + try { + sparkSession.sharedState.cacheManager.uncacheQuery( + sparkSession.table(tableName.quotedString)) + } catch { + case NonFatal(e) => log.warn(e.toString, e) } + catalog.refreshTable(tableName) + catalog.dropTable(tableName, ifExists) Seq.empty[Row] } } @@ -275,7 +268,7 @@ case class AlterTableUnsetPropertiesCommand( propKeys.foreach { k => if (!table.properties.contains(k)) { throw new AnalysisException( - s"Attempted to unset non-existent property '$k' in table '$tableName'") + s"Attempted to unset non-existent property '$k' in table '${table.identifier}'") } } } @@ -330,11 +323,11 @@ case class AlterTableSerDePropertiesCommand( catalog.alterTable(newTable) } else { val spec = partSpec.get - val part = catalog.getPartition(tableName, spec) + val part = catalog.getPartition(table.identifier, spec) val newPart = part.copy(storage = part.storage.copy( serde = serdeClassName.orElse(part.storage.serde), serdeProperties = part.storage.serdeProperties ++ serdeProperties.getOrElse(Map()))) - catalog.alterPartitions(tableName, Seq(newPart)) + catalog.alterPartitions(table.identifier, Seq(newPart)) } Seq.empty[Row] } @@ -370,7 +363,7 @@ case class AlterTableAddPartitionCommand( // inherit table storage format (possibly except for location) CatalogTablePartition(spec, table.storage.copy(locationUri = location)) } - catalog.createPartitions(tableName, parts, ignoreIfExists = ifNotExists) + catalog.createPartitions(table.identifier, parts, ignoreIfExists = ifNotExists) Seq.empty[Row] } @@ -425,7 +418,7 @@ case class AlterTableDropPartitionCommand( throw new AnalysisException( "ALTER TABLE DROP PARTITIONS is not allowed for tables defined using the datasource API") } - catalog.dropPartitions(tableName, specs, ignoreIfNotExists = ifExists) + catalog.dropPartitions(table.identifier, specs, ignoreIfNotExists = ifExists) Seq.empty[Row] } @@ -474,25 +467,19 @@ case class AlterTableRecoverPartitionsCommand( override def run(spark: SparkSession): Seq[Row] = { val catalog = spark.sessionState.catalog - if (!catalog.tableExists(tableName)) { - throw new AnalysisException(s"Table $tableName in $cmd does not exist.") - } val table = catalog.getTableMetadata(tableName) - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"Operation not allowed: $cmd on temporary tables: $tableName") - } + val tableIdentWithDB = table.identifier.quotedString if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - s"Operation not allowed: $cmd on datasource tables: $tableName") + s"Operation not allowed: $cmd on datasource tables: $tableIdentWithDB") } if (!DDLUtils.isTablePartitioned(table)) { throw new AnalysisException( - s"Operation not allowed: $cmd only works on partitioned tables: $tableName") + s"Operation not allowed: $cmd only works on partitioned tables: $tableIdentWithDB") } if (table.storage.locationUri.isEmpty) { - throw new AnalysisException( - s"Operation not allowed: $cmd only works on table with location provided: $tableName") + throw new AnalysisException(s"Operation not allowed: $cmd only works on table with " + + s"location provided: $tableIdentWithDB") } val root = new Path(table.storage.locationUri.get) @@ -660,7 +647,7 @@ case class AlterTableSetLocationCommand( partitionSpec match { case Some(spec) => // Partition spec is specified, so we set the location only for this partition - val part = catalog.getPartition(tableName, spec) + val part = catalog.getPartition(table.identifier, spec) val newPart = if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( @@ -669,7 +656,7 @@ case class AlterTableSetLocationCommand( } else { part.copy(storage = part.storage.copy(locationUri = Some(location))) } - catalog.alterPartitions(tableName, Seq(newPart)) + catalog.alterPartitions(table.identifier, Seq(newPart)) case None => // No partition spec is specified, so we set the location for the table itself val newTable = http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 615d9dc..995feb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -78,11 +78,7 @@ case class CreateTableLikeCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(sourceTable)) { - throw new AnalysisException( - s"Source table in CREATE TABLE LIKE does not exist: '$sourceTable'") - } - val sourceTableDesc = catalog.getTableMetadata(sourceTable) + val sourceTableDesc = catalog.getTempViewOrPermanentTableMetadata(sourceTable) if (DDLUtils.isDatasourceTable(sourceTableDesc) || sourceTableDesc.tableType == CatalogTableType.VIEW) { @@ -188,13 +184,13 @@ case class AlterTableRenameCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - DDLUtils.verifyAlterTableType(catalog, oldName, isView) // If this is a temp view, just rename the view. // Otherwise, if this is a real table, we also need to uncache and invalidate the table. - val isTemporary = catalog.isTemporaryTable(oldName) - if (isTemporary) { + if (catalog.isTemporaryTable(oldName)) { catalog.renameTable(oldName, newName) } else { + val table = catalog.getTableMetadata(oldName) + DDLUtils.verifyAlterTableType(catalog, table.identifier, isView) // If an exception is thrown here we can just assume the table is uncached; // this can happen with Hive tables when the underlying catalog is in-memory. val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) @@ -206,7 +202,6 @@ case class AlterTableRenameCommand( } } // For datasource tables, we also need to update the "path" serde property - val table = catalog.getTableMetadata(oldName) if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { val newPath = catalog.defaultTablePath(newName) val newTable = table.withNewStorage( @@ -244,37 +239,34 @@ case class LoadDataCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(table)) { - throw new AnalysisException(s"Target table in LOAD DATA does not exist: '$table'") - } - val targetTable = catalog.getTableMetadataOption(table).getOrElse { - throw new AnalysisException(s"Target table in LOAD DATA cannot be temporary: '$table'") - } + val targetTable = catalog.getTableMetadata(table) + val tableIdentwithDB = targetTable.identifier.quotedString if (DDLUtils.isDatasourceTable(targetTable)) { - throw new AnalysisException(s"LOAD DATA is not supported for datasource tables: '$table'") + throw new AnalysisException( + s"LOAD DATA is not supported for datasource tables: '$tableIdentwithDB'") } if (targetTable.partitionColumnNames.nonEmpty) { if (partition.isEmpty) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is partitioned, " + s"but no partition spec is provided") } if (targetTable.partitionColumnNames.size != partition.get.size) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is partitioned, " + s"but number of columns in provided partition spec (${partition.get.size}) " + s"do not match number of partitioned columns in table " + s"(s${targetTable.partitionColumnNames.size})") } partition.get.keys.foreach { colName => if (!targetTable.partitionColumnNames.contains(colName)) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + - s"but the specified partition spec refers to a column that is not partitioned: " + - s"'$colName'") + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is " + + s"partitioned, but the specified partition spec refers to a column that is " + + s"not partitioned: '$colName'") } } } else { if (partition.nonEmpty) { - throw new AnalysisException(s"LOAD DATA target table '$table' is not partitioned, " + - s"but a partition spec was provided.") + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is not " + + s"partitioned, but a partition spec was provided.") } } @@ -363,32 +355,26 @@ case class TruncateTableCommand( override def run(spark: SparkSession): Seq[Row] = { val catalog = spark.sessionState.catalog - if (!catalog.tableExists(tableName)) { - throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") - } - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") - } val table = catalog.getTableMetadata(tableName) + val tableIdentwithDB = table.identifier.quotedString if (table.tableType == CatalogTableType.EXTERNAL) { throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableName'") + s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableIdentwithDB'") } if (table.tableType == CatalogTableType.VIEW) { throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'") + s"Operation not allowed: TRUNCATE TABLE on views: '$tableIdentwithDB'") } val isDatasourceTable = DDLUtils.isDatasourceTable(table) if (isDatasourceTable && partitionSpec.isDefined) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + - s"for tables created using the data sources API: '$tableName'") + s"for tables created using the data sources API: '$tableIdentwithDB'") } if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + - s"for tables that are not partitioned: '$tableName'") + s"for tables that are not partitioned: '$tableIdentwithDB'") } val locations = if (isDatasourceTable) { @@ -396,7 +382,7 @@ case class TruncateTableCommand( } else if (table.partitionColumnNames.isEmpty) { Seq(table.storage.locationUri) } else { - catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) + catalog.listPartitions(table.identifier, partitionSpec).map(_.storage.locationUri) } val hadoopConf = spark.sessionState.newHadoopConf() locations.foreach { location => @@ -409,8 +395,8 @@ case class TruncateTableCommand( } catch { case NonFatal(e) => throw new AnalysisException( - s"Failed to truncate table '$tableName' when removing data of the path: $path " + - s"because of ${e.toString}") + s"Failed to truncate table '$tableIdentwithDB' when removing data of the path: " + + s"$path because of ${e.toString}") } } } @@ -419,10 +405,10 @@ case class TruncateTableCommand( spark.sessionState.refreshTable(tableName.unquotedString) // Also try to drop the contents of the table from the columnar cache try { - spark.sharedState.cacheManager.uncacheQuery(spark.table(tableName.quotedString)) + spark.sharedState.cacheManager.uncacheQuery(spark.table(table.identifier)) } catch { case NonFatal(e) => - log.warn(s"Exception when attempting to uncache table '$tableName'", e) + log.warn(s"Exception when attempting to uncache table '$tableIdentwithDB'", e) } Seq.empty[Row] } @@ -666,14 +652,16 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio * SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; * }}} */ -case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand { +case class ShowColumnsCommand(tableName: TableIdentifier) extends RunnableCommand { // The result of SHOW COLUMNS has one column called 'result' override val output: Seq[Attribute] = { AttributeReference("result", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.catalog.getTableMetadata(table).schema.map { c => + val catalog = sparkSession.sessionState.catalog + val table = catalog.getTempViewOrPermanentTableMetadata(tableName) + table.schema.map { c => Row(c.name) } } @@ -695,7 +683,7 @@ case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand { * }}} */ case class ShowPartitionsCommand( - table: TableIdentifier, + tableName: TableIdentifier, spec: Option[TablePartitionSpec]) extends RunnableCommand { // The result of SHOW PARTITIONS has one column called 'result' override val output: Seq[Attribute] = { @@ -710,13 +698,8 @@ case class ShowPartitionsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a temporary table: ${table.unquotedString}") - } - - val tab = catalog.getTableMetadata(table) + val table = catalog.getTableMetadata(tableName) + val tableIdentWithDB = table.identifier.quotedString /** * Validate and throws an [[AnalysisException]] exception under the following conditions: @@ -724,19 +707,18 @@ case class ShowPartitionsCommand( * 2. If it is a datasource table. * 3. If it is a view. */ - if (tab.tableType == VIEW) { - throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a view: ${tab.qualifiedName}") + if (table.tableType == VIEW) { + throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: $tableIdentWithDB") } - if (!DDLUtils.isTablePartitioned(tab)) { + if (!DDLUtils.isTablePartitioned(table)) { throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${tab.qualifiedName}") + s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableIdentWithDB") } - if (DDLUtils.isDatasourceTable(tab)) { + if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a datasource table: ${tab.qualifiedName}") + s"SHOW PARTITIONS is not allowed on a datasource table: $tableIdentWithDB") } /** @@ -745,7 +727,7 @@ case class ShowPartitionsCommand( * thrown if the partitioning spec is invalid. */ if (spec.isDefined) { - val badColumns = spec.get.keySet.filterNot(tab.partitionColumns.map(_.name).contains) + val badColumns = spec.get.keySet.filterNot(table.partitionColumns.map(_.name).contains) if (badColumns.nonEmpty) { val badCols = badColumns.mkString("[", ", ", "]") throw new AnalysisException( @@ -753,8 +735,8 @@ case class ShowPartitionsCommand( } } - val partNames = catalog.listPartitions(table, spec).map { p => - getPartName(p.spec, tab.partitionColumnNames) + val partNames = catalog.listPartitions(tableName, spec).map { p => + getPartName(p.spec, table.partitionColumnNames) } partNames.map(Row(_)) @@ -768,16 +750,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"SHOW CREATE TABLE cannot be applied to temporary table") - } - - if (!catalog.tableExists(table)) { - throw new AnalysisException(s"Table $table doesn't exist") - } - val tableMetadata = catalog.getTableMetadata(table) val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 2067e7e..414a4a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -151,7 +151,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = { - val tableMetadata = sessionCatalog.getTableMetadata(tableIdentifier) + val tableMetadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdentifier) val partitionColumnNames = tableMetadata.partitionColumnNames.toSet val bucketColumnNames = tableMetadata.bucketColumnNames.toSet val columns = tableMetadata.schema.map { c => http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 3b2a660..1f5492e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1424,7 +1424,9 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") sql(s"CREATE EXTERNAL TABLE my_ext_tab LOCATION '$path'") sql(s"CREATE VIEW my_view AS SELECT 1") - assertUnsupported("TRUNCATE TABLE my_temp_tab") + intercept[NoSuchTableException] { + sql("TRUNCATE TABLE my_temp_tab") + } assertUnsupported("TRUNCATE TABLE my_ext_tab") assertUnsupported("TRUNCATE TABLE my_view") } http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala ---------------------------------------------------------------------- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index b758ab0..28c8139 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert( intercept[AnalysisException] { sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) - }.getMessage.contains("Table default.createdJsonTable already exists."), + }.getMessage.contains("Table createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala ---------------------------------------------------------------------- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 37e78b0..446029f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -404,25 +404,24 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto |USING org.apache.spark.sql.parquet.DefaultSource """.stripMargin) // An empty sequence of row is returned for session temporary table. - val message1 = intercept[AnalysisException] { + intercept[NoSuchTableException] { sql("SHOW PARTITIONS parquet_temp") - }.getMessage - assert(message1.contains("is not allowed on a temporary table")) + } - val message2 = intercept[AnalysisException] { + val message1 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_tab3") }.getMessage - assert(message2.contains("not allowed on a table that is not partitioned")) + assert(message1.contains("not allowed on a table that is not partitioned")) - val message3 = intercept[AnalysisException] { + val message2 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_tab4 PARTITION(abcd=2015, xyz=1)") }.getMessage - assert(message3.contains("Non-partitioning column(s) [abcd, xyz] are specified")) + assert(message2.contains("Non-partitioning column(s) [abcd, xyz] are specified")) - val message4 = intercept[AnalysisException] { + val message3 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_view1") }.getMessage - assert(message4.contains("is not allowed on a view")) + assert(message3.contains("is not allowed on a view")) } } http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala ---------------------------------------------------------------------- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 676c08b..0a6ccbe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -292,7 +292,7 @@ class HiveDDLSuite sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") }.getMessage assert(message.contains( - "Attempted to unset non-existent property 'p' in table '`view1`'")) + "Attempted to unset non-existent property 'p' in table '`default`.`view1`'")) } } } @@ -664,8 +664,8 @@ class HiveDDLSuite .createTempView(sourceViewName) sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName") - val sourceTable = spark.sessionState.catalog.getTableMetadata( - TableIdentifier(sourceViewName, None)) + val sourceTable = spark.sessionState.catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier(sourceViewName)) val targetTable = spark.sessionState.catalog.getTableMetadata( TableIdentifier(targetTabName, Some("default"))) http://git-wip-us.apache.org/repos/asf/spark/blob/22216d6b/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 cf63537..490fea2 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 @@ -57,6 +57,56 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("Issue exceptions for ALTER VIEW on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + } + } + + test("Issue exceptions for ALTER TABLE on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") + assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") + assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS") + } + } + + test("Issue exceptions for other table DDL on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + + val e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $viewName SELECT 1") + }.getMessage + assert(e.contains("Inserting into an RDD-based table is not allowed")) + + val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath + assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName""") + assertNoSuchTable(s"TRUNCATE TABLE $viewName") + assertNoSuchTable(s"SHOW CREATE TABLE $viewName") + assertNoSuchTable(s"SHOW PARTITIONS $viewName") + assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") + } + } + + private def assertNoSuchTable(query: String): Unit = { + intercept[NoSuchTableException] { + sql(query) + } + } + test("error handling: fail if the view sql itself is invalid") { // A table that does not exist intercept[AnalysisException] { @@ -177,13 +227,8 @@ 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") - ) + assertNoSuchTable("ALTER VIEW testView AS SELECT 1, 2") + assertNoSuchTable("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") { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org