From b38a21ef6146784e4b93ef4ce8c899f1eee14572 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 16 Nov 2015 18:30:26 -0800 Subject: [PATCH 1/9] SPARK-11633 --- .../spark/sql/catalyst/analysis/Analyzer.scala | 3 ++- .../spark/sql/hive/execution/SQLQuerySuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2f4670b55bdb..5a5b71e52dd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -425,7 +425,8 @@ class Analyzer( */ j case Some((oldRelation, newRelation)) => - val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) + val attributeRewrites = + AttributeMap(oldRelation.output.zip(newRelation.output).filter(x => x._1 != x._2)) val newRight = right transformUp { case r if r == oldRelation => newRelation } transformUp { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 3427152b2da0..5e00546a74c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -51,6 +51,8 @@ case class Order( state: String, month: Int) +case class Individual(F1: Integer, F2: Integer) + case class WindowData( month: Int, area: String, @@ -1479,4 +1481,18 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |FROM (SELECT '{"f1": "value1", "f2": 12}' json, 'hello' as str) test """.stripMargin), Row("value1", "12", 3.14, "hello")) } + + test ("SPARK-11633: HiveContext throws TreeNode Exception : Failed to Copy Node") { + val rdd1 = sparkContext.parallelize(Seq( Individual(1,3), Individual(2,1))) + val df = hiveContext.createDataFrame(rdd1) + df.registerTempTable("foo") + val df2 = sql("select f1, F2 as F2 from foo") + df2.registerTempTable("foo2") + df2.registerTempTable("foo3") + + checkAnswer(sql( + """ + SELECT a.F1 FROM foo2 a INNER JOIN foo3 b ON a.F2=b.F2 + """.stripMargin), Row(2) :: Row(1) :: Nil) + } } From 0546772f151f83d6d3cf4d000cbe341f52545007 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 20 Nov 2015 10:56:45 -0800 Subject: [PATCH 2/9] converge --- .../spark/sql/catalyst/analysis/Analyzer.scala | 3 +-- .../spark/sql/hive/execution/SQLQuerySuite.scala | 15 --------------- 2 files changed, 1 insertion(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7c9512fbd00a..47962ebe6ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -417,8 +417,7 @@ class Analyzer( */ j case Some((oldRelation, newRelation)) => - val attributeRewrites = - AttributeMap(oldRelation.output.zip(newRelation.output).filter(x => x._1 != x._2)) + val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) val newRight = right transformUp { case r if r == oldRelation => newRelation } transformUp { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 5e00546a74c0..61d9dcd37572 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -51,8 +51,6 @@ case class Order( state: String, month: Int) -case class Individual(F1: Integer, F2: Integer) - case class WindowData( month: Int, area: String, @@ -1481,18 +1479,5 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |FROM (SELECT '{"f1": "value1", "f2": 12}' json, 'hello' as str) test """.stripMargin), Row("value1", "12", 3.14, "hello")) } - - test ("SPARK-11633: HiveContext throws TreeNode Exception : Failed to Copy Node") { - val rdd1 = sparkContext.parallelize(Seq( Individual(1,3), Individual(2,1))) - val df = hiveContext.createDataFrame(rdd1) - df.registerTempTable("foo") - val df2 = sql("select f1, F2 as F2 from foo") - df2.registerTempTable("foo2") - df2.registerTempTable("foo3") - - checkAnswer(sql( - """ - SELECT a.F1 FROM foo2 a INNER JOIN foo3 b ON a.F2=b.F2 - """.stripMargin), Row(2) :: Row(1) :: Nil) } } From b37a64f13956b6ddd0e38ddfd9fe1caee611f1a8 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 20 Nov 2015 10:58:37 -0800 Subject: [PATCH 3/9] converge --- .../org/apache/spark/sql/hive/execution/SQLQuerySuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 61d9dcd37572..3427152b2da0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1479,5 +1479,4 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |FROM (SELECT '{"f1": "value1", "f2": 12}' json, 'hello' as str) test """.stripMargin), Row("value1", "12", 3.14, "hello")) } - } } From 34e44dac2c9f731fd69081ac0846800e3198de70 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 7 Apr 2016 14:11:04 -0700 Subject: [PATCH 4/9] fix 1 --- .../apache/spark/sql/execution/SparkSqlParser.scala | 9 ++++++--- .../org/apache/spark/sql/execution/command/ddl.scala | 10 ++++++---- 2 files changed, 12 insertions(+), 7 deletions(-) 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 3de8aa02766d..7fdc9abf75f9 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 @@ -378,7 +378,8 @@ class SparkSqlAstBuilder extends AstBuilder { override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { AlterTableRename( visitTableIdentifier(ctx.from), - visitTableIdentifier(ctx.to)) + visitTableIdentifier(ctx.to), + ctx.VIEW != null) } /** @@ -394,7 +395,8 @@ class SparkSqlAstBuilder extends AstBuilder { ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableSetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList)) + visitTablePropertyList(ctx.tablePropertyList), + ctx.VIEW != null) } /** @@ -411,7 +413,8 @@ class SparkSqlAstBuilder extends AstBuilder { AlterTableUnsetProperties( visitTableIdentifier(ctx.tableIdentifier), visitTablePropertyList(ctx.tablePropertyList).keys.toSeq, - ctx.EXISTS != null) + ctx.EXISTS != null, + ctx.VIEW != null) } /** 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 6d56a6fec86e..8745e2615d6c 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 @@ -185,7 +185,8 @@ case class DescribeDatabase( */ case class AlterTableRename( oldName: TableIdentifier, - newName: TableIdentifier) + newName: TableIdentifier, + isView: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { @@ -194,7 +195,6 @@ case class AlterTableRename( catalog.renameTable(oldName, newName) Seq.empty[Row] } - } /** @@ -208,7 +208,8 @@ case class AlterTableRename( */ case class AlterTableSetProperties( tableName: TableIdentifier, - properties: Map[String, String]) + properties: Map[String, String], + isView: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { @@ -238,7 +239,8 @@ case class AlterTableSetProperties( case class AlterTableUnsetProperties( tableName: TableIdentifier, propKeys: Seq[String], - ifExists: Boolean) + ifExists: Boolean, + isView: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { From c8811bca6ebda2b29d7efdd66bdc384e9c7b32ea Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 9 Apr 2016 23:45:01 -0700 Subject: [PATCH 5/9] correct compilation errors --- .../sql/execution/command/DDLCommandSuite.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index ac69518ddf68..811981f97d50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -205,10 +205,12 @@ class DDLCommandSuite extends PlanTest { val parsed_view = parser.parsePlan(sql_view) val expected_table = AlterTableRename( TableIdentifier("table_name", None), - TableIdentifier("new_table_name", None)) + TableIdentifier("new_table_name", None), + isView = false) val expected_view = AlterTableRename( TableIdentifier("table_name", None), - TableIdentifier("new_table_name", None)) + TableIdentifier("new_table_name", None), + isView = false) comparePlans(parsed_table, expected_table) comparePlans(parsed_view, expected_view) } @@ -235,11 +237,11 @@ class DDLCommandSuite extends PlanTest { val tableIdent = TableIdentifier("table_name", None) val expected1_table = AlterTableSetProperties( - tableIdent, Map("test" -> "test", "comment" -> "new_comment")) + tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) val expected2_table = AlterTableUnsetProperties( - tableIdent, Seq("comment", "test"), ifExists = false) + tableIdent, Seq("comment", "test"), ifExists = false, isView = false) val expected3_table = AlterTableUnsetProperties( - tableIdent, Seq("comment", "test"), ifExists = true) + tableIdent, Seq("comment", "test"), ifExists = true, isView = false) val expected1_view = expected1_table val expected2_view = expected2_table val expected3_view = expected3_table From c55ab123f016d166a4b4af98c5424967bac685b9 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 11 Apr 2016 21:20:50 -0700 Subject: [PATCH 6/9] impl --- .../spark/sql/execution/command/ddl.scala | 23 +++++++- .../sql/hive/execution/HiveDDLSuite.scala | 54 +++++++++++++++++++ 2 files changed, 76 insertions(+), 1 deletion(-) 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 89fe126222e4..1bc62e612931 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.types._ @@ -237,6 +237,7 @@ case class AlterTableRename( override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog + DDLUtils.verifyAlterTableOrAlterView(catalog, oldName, isView) catalog.invalidateTable(oldName) catalog.renameTable(oldName, newName) Seq.empty[Row] @@ -260,6 +261,7 @@ case class AlterTableSetProperties( override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog + DDLUtils.verifyAlterTableOrAlterView(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) val newProperties = table.properties ++ properties if (DDLUtils.isDatasourceTable(newProperties)) { @@ -291,6 +293,7 @@ case class AlterTableUnsetProperties( override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog + DDLUtils.verifyAlterTableOrAlterView(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( @@ -509,5 +512,23 @@ private object DDLUtils { def isDatasourceTable(table: CatalogTable): Boolean = { isDatasourceTable(table.properties) } + + // If the command ALTER VIEW is to alter a table or ALTER TABLE is to alter a view, + // issue an exception. + @throws[AnalysisException] + def verifyAlterTableOrAlterView( + catalog: SessionCatalog, + tableIdentifier: TableIdentifier, + isView: Boolean): Unit = { + catalog.getTableMetadataOption(tableIdentifier).map(_.tableType match { + case CatalogTableType.VIRTUAL_VIEW if !isView => + throw new AnalysisException( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") + case o if o != CatalogTableType.VIRTUAL_VIEW && isView => + throw new AnalysisException( + s"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead") + case _ => + }) + } } 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 78ccdc7adb17..4c456b979aac 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 @@ -131,6 +131,60 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("alter views - rename") { + val tabName = "tab1" + withTable(tabName) { + sqlContext.range(10).write.saveAsTable(tabName) + val oldViewName = "view1" + val newViewName = "view2" + withView(oldViewName, newViewName) { + val catalog = hiveContext.sessionState.catalog + sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") + + assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) + sql(s"ALTER VIEW $oldViewName RENAME TO $newViewName") + assert(!catalog.tableExists(TableIdentifier(oldViewName))) + assert(catalog.tableExists(TableIdentifier(newViewName))) + + sql(s"DROP VIEW $newViewName") + } + } + } + + test("alter views and alter table - misuse") { + val tabName = "tab1" + withTable(tabName) { + sqlContext.range(10).write.saveAsTable(tabName) + val oldViewName = "view1" + val newViewName = "view2" + withView(oldViewName, newViewName) { + val catalog = hiveContext.sessionState.catalog + sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") + + assert(catalog.tableExists(TableIdentifier(tabName))) + assert(catalog.tableExists(TableIdentifier(oldViewName))) + + var message = intercept[AnalysisException] { + sql(s"ALTER VIEW $tabName RENAME TO $newViewName") + }.getMessage + assert(message.contains( + "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + + message = intercept[AnalysisException] { + sql(s"ALTER TABLE $oldViewName RENAME TO $newViewName") + }.getMessage + assert(message.contains( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + + assert(catalog.tableExists(TableIdentifier(tabName))) + assert(catalog.tableExists(TableIdentifier(oldViewName))) + + sql(s"DROP VIEW $newViewName") + } + } + } + test("drop table using drop view") { withTable("tab1") { sql("CREATE TABLE tab1(c1 int)") From ed5a7d0bf4dc4d0f997fe27a274e2d9cfa1749cc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 11 Apr 2016 23:52:10 -0700 Subject: [PATCH 7/9] added more test cases. --- .../execution/command/DDLCommandSuite.scala | 8 +-- .../sql/hive/execution/HiveDDLSuite.scala | 64 +++++++++++++++++++ 2 files changed, 68 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 811981f97d50..74958794480e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -210,7 +210,7 @@ class DDLCommandSuite extends PlanTest { val expected_view = AlterTableRename( TableIdentifier("table_name", None), TableIdentifier("new_table_name", None), - isView = false) + isView = true) comparePlans(parsed_table, expected_table) comparePlans(parsed_view, expected_view) } @@ -242,9 +242,9 @@ class DDLCommandSuite extends PlanTest { tableIdent, Seq("comment", "test"), ifExists = false, isView = false) val expected3_table = AlterTableUnsetProperties( tableIdent, Seq("comment", "test"), ifExists = true, isView = false) - val expected1_view = expected1_table - val expected2_view = expected2_table - val expected3_view = expected3_table + val expected1_view = expected1_table.copy(isView = true) + val expected2_view = expected2_table.copy(isView = true) + val expected3_view = expected3_table.copy(isView = true) comparePlans(parsed1_table, expected1_table) comparePlans(parsed2_table, expected2_table) 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 4c456b979aac..64b7e6d42b3d 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 @@ -152,6 +152,46 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("alter views - set/unset tblproperties") { + val tabName = "tab1" + withTable(tabName) { + sqlContext.range(10).write.saveAsTable(tabName) + val viewName = "view1" + withView(viewName) { + val catalog = hiveContext.sessionState.catalog + sql(s"CREATE VIEW $viewName AS SELECT * FROM $tabName") + + assert(catalog.getTableMetadata(TableIdentifier(viewName)) + .properties.filter(_._1 != "transient_lastDdlTime") == Map()) + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + assert(catalog.getTableMetadata(TableIdentifier(viewName)) + .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an")) + + // no exception or message will be issued if we set it again + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + assert(catalog.getTableMetadata(TableIdentifier(viewName)) + .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an")) + + // the value will be updated if we set the same key to a different value + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'b')") + assert(catalog.getTableMetadata(TableIdentifier(viewName)) + .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "b")) + + sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + assert(catalog.getTableMetadata(TableIdentifier(viewName)) + .properties.filter(_._1 != "transient_lastDdlTime") == Map()) + + val message = intercept[AnalysisException] { + sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + }.getMessage + assert(message.contains( + "attempted to unset non-existent property 'p' in table '`view1`'")) + + sql(s"DROP VIEW $viewName") + } + } + } + test("alter views and alter table - misuse") { val tabName = "tab1" withTable(tabName) { @@ -171,12 +211,36 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(message.contains( "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + message = intercept[AnalysisException] { + sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')") + }.getMessage + assert(message.contains( + "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + + message = intercept[AnalysisException] { + sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')") + }.getMessage + assert(message.contains( + "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + message = intercept[AnalysisException] { sql(s"ALTER TABLE $oldViewName RENAME TO $newViewName") }.getMessage assert(message.contains( "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + message = intercept[AnalysisException] { + sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") + }.getMessage + assert(message.contains( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + + message = intercept[AnalysisException] { + sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") + }.getMessage + assert(message.contains( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + assert(catalog.tableExists(TableIdentifier(tabName))) assert(catalog.tableExists(TableIdentifier(oldViewName))) From e82aa477c9720c54ab847f76ffde357c63e821a1 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 13 Apr 2016 19:01:51 -0700 Subject: [PATCH 8/9] address comments. --- .../spark/sql/execution/command/ddl.scala | 37 ++++--------------- .../spark/sql/execution/command/tables.scala | 4 +- 2 files changed, 10 insertions(+), 31 deletions(-) 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 f6181dff3d56..fc37a142cda1 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 @@ -224,30 +224,6 @@ case class DropTable( } } -/** - * A command that renames a table/view. - * - * The syntax of this command is: - * {{{ - * ALTER TABLE table1 RENAME TO table2; - * ALTER VIEW view1 RENAME TO view2; - * }}} - */ -case class AlterTableRename( - oldName: TableIdentifier, - newName: TableIdentifier, - isView: Boolean) - extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - val catalog = sqlContext.sessionState.catalog - DDLUtils.verifyAlterTableOrAlterView(catalog, oldName, isView) - catalog.invalidateTable(oldName) - catalog.renameTable(oldName, newName) - Seq.empty[Row] - } -} - /** * A command that sets table/view properties. * @@ -265,7 +241,7 @@ case class AlterTableSetProperties( override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog - DDLUtils.verifyAlterTableOrAlterView(catalog, tableName, isView) + DDLUtils.verifyAlterTableType(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) val newProperties = table.properties ++ properties if (DDLUtils.isDatasourceTable(newProperties)) { @@ -297,7 +273,7 @@ case class AlterTableUnsetProperties( override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog - DDLUtils.verifyAlterTableOrAlterView(catalog, tableName, isView) + DDLUtils.verifyAlterTableType(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( @@ -542,10 +518,11 @@ private object DDLUtils { isDatasourceTable(table.properties) } - // If the command ALTER VIEW is to alter a table or ALTER TABLE is to alter a view, - // issue an exception. - @throws[AnalysisException] - def verifyAlterTableOrAlterView( + /** + * If the command ALTER VIEW is to alter a table or ALTER TABLE is to alter a view, + * issue an exception [[AnalysisException]]. + */ + def verifyAlterTableType( catalog: SessionCatalog, tableIdentifier: TableIdentifier, isView: Boolean): Unit = { 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 9c6030502dc2..e315598daaae 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 @@ -67,11 +67,13 @@ case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends Runnab */ case class AlterTableRename( oldName: TableIdentifier, - newName: TableIdentifier) + newName: TableIdentifier, + isView: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val catalog = sqlContext.sessionState.catalog + DDLUtils.verifyAlterTableType(catalog, oldName, isView) catalog.invalidateTable(oldName) catalog.renameTable(oldName, newName) Seq.empty[Row] From db7f9314bc72a85a157debd70012f0f2fce70a05 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 14 Apr 2016 05:46:04 -0700 Subject: [PATCH 9/9] address comments. --- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 6 ------ 1 file changed, 6 deletions(-) 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 64b7e6d42b3d..6ba8bd5a6a0f 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 @@ -146,8 +146,6 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql(s"ALTER VIEW $oldViewName RENAME TO $newViewName") assert(!catalog.tableExists(TableIdentifier(oldViewName))) assert(catalog.tableExists(TableIdentifier(newViewName))) - - sql(s"DROP VIEW $newViewName") } } } @@ -186,8 +184,6 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }.getMessage assert(message.contains( "attempted to unset non-existent property 'p' in table '`view1`'")) - - sql(s"DROP VIEW $viewName") } } } @@ -243,8 +239,6 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(catalog.tableExists(TableIdentifier(tabName))) assert(catalog.tableExists(TableIdentifier(oldViewName))) - - sql(s"DROP VIEW $newViewName") } } }