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 92ca416709a6..3108042a12ae 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 @@ -893,7 +893,7 @@ class Analyzer(override val catalogManager: CatalogManager) }.getOrElse(write) case _ => write } - case u @ UnresolvedTable(ident, cmd) => + case u @ UnresolvedTable(ident, cmd, _) => lookupTempView(ident).foreach { _ => throw QueryCompilationErrors.expectTableNotTempViewError(ident.quoted, cmd, u) } @@ -990,7 +990,7 @@ class Analyzer(override val catalogManager: CatalogManager) SubqueryAlias(catalog.get.name +: ident.namespace :+ ident.name, relation) }.getOrElse(u) - case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident), _) => + case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident), _, _) => CatalogV2Util.loadTable(catalog, ident) .map(table => ResolvedTable.create(catalog.asTableCatalog, ident, table)) .getOrElse(u) @@ -1144,18 +1144,20 @@ class Analyzer(override val catalogManager: CatalogManager) lookupRelation(u.multipartIdentifier, u.options, u.isStreaming) .map(resolveViews).getOrElse(u) - case u @ UnresolvedTable(identifier, cmd) => + case u @ UnresolvedTable(identifier, cmd, relationTypeMismatchHint) => lookupTableOrView(identifier).map { - case v: ResolvedView => throw QueryCompilationErrors.expectTableNotViewError(v, cmd, u) + case v: ResolvedView => + throw QueryCompilationErrors.expectTableNotViewError( + v, cmd, relationTypeMismatchHint, u) case table => table }.getOrElse(u) case u @ UnresolvedView(identifier, cmd, _, relationTypeMismatchHint) => lookupTableOrView(identifier).map { - case v: ResolvedView => v - case _ => - u.failAnalysis(s"${identifier.quoted} is a table. '$cmd' expects a view." + - relationTypeMismatchHint.map(" " + _).getOrElse("")) + case t: ResolvedTable => + throw QueryCompilationErrors.expectViewNotTableError( + t, cmd, relationTypeMismatchHint, u) + case view => view }.getOrElse(u) case u @ UnresolvedTableOrView(identifier, _, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 47263791dc50..f7e08bdb73ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -40,7 +40,8 @@ case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNod */ case class UnresolvedTable( multipartIdentifier: Seq[String], - commandName: String) extends LeafNode { + commandName: String, + relationTypeMismatchHint: Option[String]) extends LeafNode { override lazy val resolved: Boolean = false override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c276168a345f..c56a5225cba2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2166,8 +2166,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ private def createUnresolvedTable( ctx: MultipartIdentifierContext, - commandName: String): UnresolvedTable = withOrigin(ctx) { - UnresolvedTable(visitMultipartIdentifier(ctx), commandName) + commandName: String, + relationTypeMismatchHint: Option[String] = None): UnresolvedTable = withOrigin(ctx) { + UnresolvedTable(visitMultipartIdentifier(ctx), commandName, relationTypeMismatchHint) } /** @@ -3490,7 +3491,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg ctx.multipartIdentifier, commandName = "ALTER VIEW ... SET TBLPROPERTIES", allowTemp = false, - relationTypeMismatchHint = Some("Please use ALTER TABLE instead.")), + relationTypeMismatchHint = alterViewTypeMismatchHint), cleanedTableProperties) } else { AlterTableSetPropertiesStatement( @@ -3520,7 +3521,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg ctx.multipartIdentifier, commandName = "ALTER VIEW ... UNSET TBLPROPERTIES", allowTemp = false, - relationTypeMismatchHint = Some("Please use ALTER TABLE instead.")), + relationTypeMismatchHint = alterViewTypeMismatchHint), cleanedProperties, ifExists) } else { @@ -3543,7 +3544,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableSetLocation( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... SET LOCATION ..."), + "ALTER TABLE ... SET LOCATION ...", + alterTableTypeMismatchHint), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) } @@ -3810,7 +3812,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableRecoverPartitions( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... RECOVER PARTITIONS")) + "ALTER TABLE ... RECOVER PARTITIONS", + alterTableTypeMismatchHint)) } /** @@ -3839,7 +3842,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableAddPartition( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... ADD PARTITION ..."), + "ALTER TABLE ... ADD PARTITION ...", + alterTableTypeMismatchHint), specsAndLocs.toSeq, ctx.EXISTS != null) } @@ -3857,7 +3861,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableRenamePartition( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... RENAME TO PARTITION"), + "ALTER TABLE ... RENAME TO PARTITION", + alterTableTypeMismatchHint), UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)), UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.to))) } @@ -3885,7 +3890,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableDropPartition( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... DROP PARTITION ..."), + "ALTER TABLE ... DROP PARTITION ...", + alterTableTypeMismatchHint), partSpecs.toSeq, ifExists = ctx.EXISTS != null, purge = ctx.PURGE != null) @@ -3905,7 +3911,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg AlterTableSerDeProperties( createUnresolvedTable( ctx.multipartIdentifier, - "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", + alterTableTypeMismatchHint), Option(ctx.STRING).map(string), Option(ctx.tablePropertyList).map(visitPropertyKeyValues), // TODO a partition spec is allowed to have optional values. This is currently violated. @@ -4118,4 +4125,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg } CommentOnTable(createUnresolvedTable(ctx.multipartIdentifier, "COMMENT ON TABLE"), comment) } + + private def alterViewTypeMismatchHint: Option[String] = Some("Please use ALTER TABLE instead.") + + private def alterTableTypeMismatchHint: Option[String] = Some("Please use ALTER VIEW instead.") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 57b4802968af..bd3d51d3921b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedView} +import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable, ResolvedView} import org.apache.spark.sql.catalyst.catalog.InvalidUDFClassException import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateMap, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SerdeInfo} @@ -231,9 +231,18 @@ private[spark] object QueryCompilationErrors { t.origin.line, t.origin.startPosition) } - def expectTableNotViewError(v: ResolvedView, cmd: String, t: TreeNode[_]): Throwable = { + def expectTableNotViewError( + v: ResolvedView, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = { val viewStr = if (v.isTemp) "temp view" else "view" - new AnalysisException(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.", + val hintStr = mismatchHint.map(" " + _).getOrElse("") + new AnalysisException(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.$hintStr", + t.origin.line, t.origin.startPosition) + } + + def expectViewNotTableError( + v: ResolvedTable, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = { + val hintStr = mismatchHint.map(" " + _).getOrElse("") + new AnalysisException(s"${v.identifier.quoted} is a table. '$cmd' expects a view.$hintStr", t.origin.line, t.origin.startPosition) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index de90f55dc3eb..bbaffd3c22c0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -867,17 +867,18 @@ class DDLParserSuite extends AnalysisTest { } test("alter table: set location") { + val hint = Some("Please use ALTER VIEW instead.") comparePlans( parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), AlterTableSetLocation( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."), + UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint), None, "new location")) comparePlans( parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"), AlterTableSetLocation( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."), + UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", hint), Some(Map("ds" -> "2017-06-10")), "new location")) } @@ -1915,21 +1916,36 @@ class DDLParserSuite extends AnalysisTest { test("MSCK REPAIR TABLE") { comparePlans( parsePlan("MSCK REPAIR TABLE a.b.c"), - RepairTable(UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE"))) + RepairTable(UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE", None))) } test("LOAD DATA INTO table") { comparePlans( parsePlan("LOAD DATA INPATH 'filepath' INTO TABLE a.b.c"), - LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", false, false, None)) + LoadData( + UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None), + "filepath", + false, + false, + None)) comparePlans( parsePlan("LOAD DATA LOCAL INPATH 'filepath' INTO TABLE a.b.c"), - LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, false, None)) + LoadData( + UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None), + "filepath", + true, + false, + None)) comparePlans( parsePlan("LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE a.b.c"), - LoadData(UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), "filepath", true, true, None)) + LoadData( + UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None), + "filepath", + true, + true, + None)) comparePlans( parsePlan( @@ -1938,7 +1954,7 @@ class DDLParserSuite extends AnalysisTest { |PARTITION(ds='2017-06-10') """.stripMargin), LoadData( - UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"), + UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None), "filepath", true, true, @@ -2003,12 +2019,12 @@ class DDLParserSuite extends AnalysisTest { test("TRUNCATE table") { comparePlans( parsePlan("TRUNCATE TABLE a.b.c"), - TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE"), None)) + TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", None), None)) comparePlans( parsePlan("TRUNCATE TABLE a.b.c PARTITION(ds='2017-06-10')"), TruncateTable( - UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE"), + UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", None), Some(Map("ds" -> "2017-06-10")))) } @@ -2058,9 +2074,10 @@ class DDLParserSuite extends AnalysisTest { test("alter table: SerDe properties") { val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" + val hint = Some("Please use ALTER VIEW instead.") val parsed1 = parsePlan(sql1) val expected1 = AlterTableSerDeProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), None, None) @@ -2073,7 +2090,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed2 = parsePlan(sql2) val expected2 = AlterTableSerDeProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) @@ -2086,7 +2103,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed3 = parsePlan(sql3) val expected3 = AlterTableSerDeProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) @@ -2100,7 +2117,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed4 = parsePlan(sql4) val expected4 = AlterTableSerDeProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us"))) @@ -2113,7 +2130,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed5 = parsePlan(sql5) val expected5 = AlterTableSerDeProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us"))) @@ -2126,7 +2143,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed6 = parsePlan(sql6) val expected6 = AlterTableSerDeProperties( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) @@ -2139,7 +2156,7 @@ class DDLParserSuite extends AnalysisTest { """.stripMargin val parsed7 = parsePlan(sql7) val expected7 = AlterTableSerDeProperties( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"), + UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]", hint), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us"))) @@ -2470,7 +2487,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("COMMENT ON TABLE a.b.c IS 'xYz'"), - CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE"), "xYz")) + CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE", None), "xYz")) } test("create table - without using") { 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 0d8c52662480..d4f228538730 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 @@ -448,7 +448,8 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { override def recoverPartitions(tableName: String): Unit = { val multiPartIdent = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName) sparkSession.sessionState.executePlan( - AlterTableRecoverPartitions(UnresolvedTable(multiPartIdent, "recoverPartitions()"))).toRdd + AlterTableRecoverPartitions( + UnresolvedTable(multiPartIdent, "recoverPartitions()", None))).toRdd } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala index 5ebca8f65160..1ec0f45f6611 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala @@ -30,7 +30,10 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes |(dt='2009-09-09', country='uk')""".stripMargin val parsed = parsePlan(sql) val expected = AlterTableAddPartition( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."), + UnresolvedTable( + Seq("a", "b", "c"), + "ALTER TABLE ... ADD PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq( UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"), None)), @@ -42,7 +45,10 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes val sql = "ALTER TABLE a.b.c ADD PARTITION (dt='2008-08-08') LOCATION 'loc'" val parsed = parsePlan(sql) val expected = AlterTableAddPartition( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... ADD PARTITION ..."), + UnresolvedTable( + Seq("a", "b", "c"), + "ALTER TABLE ... ADD PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq(UnresolvedPartitionSpec(Map("dt" -> "2008-08-08"), Some("loc"))), ifNotExists = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala index 53edd5854f28..b48ca16a6bb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala @@ -30,7 +30,10 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') """.stripMargin val expected = AlterTableDropPartition( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP PARTITION ..."), + UnresolvedTable( + Seq("table_name"), + "ALTER TABLE ... DROP PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq( UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")), UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))), @@ -47,7 +50,10 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe |PARTITION (dt='2009-09-09', country='uk') """.stripMargin val expected = AlterTableDropPartition( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP PARTITION ..."), + UnresolvedTable( + Seq("table_name"), + "ALTER TABLE ... DROP PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq( UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")), UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))), @@ -59,7 +65,10 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe test("drop partition in a table with multi-part identifier") { val sql = "ALTER TABLE a.b.c DROP IF EXISTS PARTITION (ds='2017-06-10')" val expected = AlterTableDropPartition( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... DROP PARTITION ..."), + UnresolvedTable( + Seq("a", "b", "c"), + "ALTER TABLE ... DROP PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"))), ifExists = true, purge = false) @@ -70,7 +79,10 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe test("drop partition with PURGE") { val sql = "ALTER TABLE table_name DROP PARTITION (p=1) PURGE" val expected = AlterTableDropPartition( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP PARTITION ..."), + UnresolvedTable( + Seq("table_name"), + "ALTER TABLE ... DROP PARTITION ...", + Some("Please use ALTER VIEW instead.")), Seq(UnresolvedPartitionSpec(Map("p" -> "1"))), ifExists = false, purge = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala index 038f0a3adb83..04251b665c05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala @@ -36,21 +36,30 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa comparePlans( parsePlan("ALTER TABLE tbl RECOVER PARTITIONS"), AlterTableRecoverPartitions( - UnresolvedTable(Seq("tbl"), "ALTER TABLE ... RECOVER PARTITIONS"))) + UnresolvedTable( + Seq("tbl"), + "ALTER TABLE ... RECOVER PARTITIONS", + Some("Please use ALTER VIEW instead.")))) } test("recover partitions of a table in a database") { comparePlans( parsePlan("alter table db.tbl recover partitions"), AlterTableRecoverPartitions( - UnresolvedTable(Seq("db", "tbl"), "ALTER TABLE ... RECOVER PARTITIONS"))) + UnresolvedTable( + Seq("db", "tbl"), + "ALTER TABLE ... RECOVER PARTITIONS", + Some("Please use ALTER VIEW instead.")))) } test("recover partitions of a table spark_catalog") { comparePlans( parsePlan("alter table spark_catalog.db.TBL recover partitions"), AlterTableRecoverPartitions( - UnresolvedTable(Seq("spark_catalog", "db", "TBL"), "ALTER TABLE ... RECOVER PARTITIONS"))) + UnresolvedTable( + Seq("spark_catalog", "db", "TBL"), + "ALTER TABLE ... RECOVER PARTITIONS", + Some("Please use ALTER VIEW instead.")))) } test("recover partitions of a table in nested namespaces") { @@ -59,6 +68,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa AlterTableRecoverPartitions( UnresolvedTable( Seq("ns1", "ns2", "ns3", "ns4", "ns5", "ns6", "ns7", "ns8", "t"), - "ALTER TABLE ... RECOVER PARTITIONS"))) + "ALTER TABLE ... RECOVER PARTITIONS", + Some("Please use ALTER VIEW instead.")))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala index c9a673279672..5f2856f071df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala @@ -30,7 +30,10 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark """.stripMargin val parsed = parsePlan(sql) val expected = AlterTableRenamePartition( - UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO PARTITION"), + UnresolvedTable( + Seq("a", "b", "c"), + "ALTER TABLE ... RENAME TO PARTITION", + Some("Please use ALTER VIEW instead.")), UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")), UnresolvedPartitionSpec(Map("ds" -> "2018-06-10"))) comparePlans(parsed, expected) @@ -43,7 +46,10 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark """.stripMargin val parsed = parsePlan(sql) val expected = AlterTableRenamePartition( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME TO PARTITION"), + UnresolvedTable( + Seq("table_name"), + "ALTER TABLE ... RENAME TO PARTITION", + Some("Please use ALTER VIEW instead.")), UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")), UnresolvedPartitionSpec(Map("dt" -> "2008-09-09", "country" -> "uk"))) comparePlans(parsed, expected) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index 7b5cf8af4eea..1c7b1282fde4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -28,18 +28,18 @@ class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { test("SHOW PARTITIONS") { val commandName = "SHOW PARTITIONS" Seq( - "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTable(Seq("t1"), commandName), None), + "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTable(Seq("t1"), commandName, None), None), "SHOW PARTITIONS db1.t1" -> ShowPartitions( - UnresolvedTable(Seq("db1", "t1"), commandName), None), + UnresolvedTable(Seq("db1", "t1"), commandName, None), None), "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> ShowPartitions( - UnresolvedTable(Seq("t1"), commandName), + UnresolvedTable(Seq("t1"), commandName, None), Some(UnresolvedPartitionSpec(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))), "SHOW PARTITIONS a.b.c" -> ShowPartitions( - UnresolvedTable(Seq("a", "b", "c"), commandName), None), + UnresolvedTable(Seq("a", "b", "c"), commandName, None), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> ShowPartitions( - UnresolvedTable(Seq("a", "b", "c"), commandName), + UnresolvedTable(Seq("a", "b", "c"), commandName, None), Some(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) 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 a60660a7a189..e3e150661a80 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 @@ -808,6 +808,20 @@ class HiveDDLSuite assert(message.contains("Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) } + private def assertErrorForAlterTableOnView( + sqlText: String, viewName: String, cmdName: String): Unit = { + assertAnalysisError( + sqlText, + s"$viewName is a view. '$cmdName' expects a table. Please use ALTER VIEW instead.") + } + + private def assertErrorForAlterViewOnTable( + sqlText: String, tableName: String, cmdName: String): Unit = { + assertAnalysisError( + sqlText, + s"$tableName is a table. '$cmdName' expects a view. Please use ALTER TABLE instead.") + } + test("create table - SET TBLPROPERTIES EXTERNAL to TRUE") { val tabName = "tab1" withTable(tabName) { @@ -856,50 +870,59 @@ class HiveDDLSuite assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName RENAME TO $newViewName") - assertAnalysisError( + assertErrorForAlterViewOnTable( s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')", - s"$tabName is a table. 'ALTER VIEW ... SET TBLPROPERTIES' expects a view. " + - "Please use ALTER TABLE instead.") + tabName, + "ALTER VIEW ... SET TBLPROPERTIES") assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") - assertAnalysisError( + assertErrorForAlterViewOnTable( s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')", - s"$tabName is a table. 'ALTER VIEW ... UNSET TBLPROPERTIES' expects a view. " + - "Please use ALTER TABLE instead.") + tabName, + "ALTER VIEW ... UNSET TBLPROPERTIES") assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'", - s"$oldViewName is a view. 'ALTER TABLE ... SET LOCATION ...' expects a table.") + oldViewName, + "ALTER TABLE ... SET LOCATION ...") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName SET SERDE 'whatever'", - s"$oldViewName is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.") + oldViewName, + "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')", - s"$oldViewName is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.") + oldViewName, + "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')", - s"$oldViewName is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.") + oldViewName, + "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName RECOVER PARTITIONS", - s"$oldViewName is a view. 'ALTER TABLE ... RECOVER PARTITIONS' expects a table.") + oldViewName, + "ALTER TABLE ... RECOVER PARTITIONS") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')", - s"$oldViewName is a view. 'ALTER TABLE ... RENAME TO PARTITION' expects a table.") + oldViewName, + "ALTER TABLE ... RENAME TO PARTITION") - assertAnalysisError( + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')", - s"$oldViewName is a view. 'ALTER TABLE ... ADD PARTITION ...' expects a table.") - assertAnalysisError( + oldViewName, + "ALTER TABLE ... ADD PARTITION ...") + + assertErrorForAlterTableOnView( s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')", - s"$oldViewName is a view. 'ALTER TABLE ... DROP PARTITION ...' expects a table.") + oldViewName, + "ALTER TABLE ... DROP PARTITION ...") assert(catalog.tableExists(TableIdentifier(tabName))) assert(catalog.tableExists(TableIdentifier(oldViewName)))