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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3806,7 +3806,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
}

/**
* Create an [[AlterTableSerDePropertiesStatement]]
* Create an [[AlterTableSerDeProperties]]
*
* For example:
* {{{
Expand All @@ -3816,8 +3816,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
* }}}
*/
override def visitSetTableSerDe(ctx: SetTableSerDeContext): LogicalPlan = withOrigin(ctx) {
AlterTableSerDePropertiesStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
AlterTableSerDeProperties(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

partitionSpec is not using UnresolvedPartitionSpec since v2 command is not supported.

UnresolvedTable(
visitMultipartIdentifier(ctx.multipartIdentifier),
"ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
Option(ctx.STRING).map(string),
Option(ctx.tablePropertyList).map(visitPropertyKeyValues),
// TODO a partition spec is allowed to have optional values. This is currently violated.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,15 +300,6 @@ case class AlterTableRenamePartitionStatement(
from: TablePartitionSpec,
to: TablePartitionSpec) extends ParsedStatement

/**
* ALTER TABLE ... SERDEPROPERTIES command, as parsed from SQL
*/
case class AlterTableSerDePropertiesStatement(
tableName: Seq[String],
serdeClassName: Option[String],
serdeProperties: Option[Map[String, String]],
partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement

/**
* An INSERT INTO statement, as parsed from SQL.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -777,6 +777,17 @@ case class AlterViewUnsetProperties(
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] command.
*/
case class AlterTableSerDeProperties(
child: LogicalPlan,
serdeClassName: Option[String],
serdeProperties: Option[Map[String, String]],
partitionSpec: Option[TablePartitionSpec]) extends Command {
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the CACHE TABLE command.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2134,8 +2134,11 @@ class DDLParserSuite extends AnalysisTest {
test("alter table: SerDe properties") {
val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'"
val parsed1 = parsePlan(sql1)
val expected1 = AlterTableSerDePropertiesStatement(
Seq("table_name"), Some("org.apache.class"), None, None)
val expected1 = AlterTableSerDeProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
Some("org.apache.class"),
None,
None)
comparePlans(parsed1, expected1)

val sql2 =
Expand All @@ -2144,8 +2147,8 @@ class DDLParserSuite extends AnalysisTest {
|WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed2 = parsePlan(sql2)
val expected2 = AlterTableSerDePropertiesStatement(
Seq("table_name"),
val expected2 = AlterTableSerDeProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
Some("org.apache.class"),
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
None)
Expand All @@ -2157,8 +2160,11 @@ class DDLParserSuite extends AnalysisTest {
|SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed3 = parsePlan(sql3)
val expected3 = AlterTableSerDePropertiesStatement(
Seq("table_name"), None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None)
val expected3 = AlterTableSerDeProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
None,
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
None)
comparePlans(parsed3, expected3)

val sql4 =
Expand All @@ -2168,8 +2174,8 @@ class DDLParserSuite extends AnalysisTest {
|WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed4 = parsePlan(sql4)
val expected4 = AlterTableSerDePropertiesStatement(
Seq("table_name"),
val expected4 = AlterTableSerDeProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
Some("org.apache.class"),
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
Expand All @@ -2181,8 +2187,8 @@ class DDLParserSuite extends AnalysisTest {
|SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed5 = parsePlan(sql5)
val expected5 = AlterTableSerDePropertiesStatement(
Seq("table_name"),
val expected5 = AlterTableSerDeProperties(
UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
None,
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
Expand All @@ -2194,8 +2200,8 @@ class DDLParserSuite extends AnalysisTest {
|WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed6 = parsePlan(sql6)
val expected6 = AlterTableSerDePropertiesStatement(
Seq("a", "b", "c"),
val expected6 = AlterTableSerDeProperties(
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
Some("org.apache.class"),
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
None)
Expand All @@ -2207,8 +2213,8 @@ class DDLParserSuite extends AnalysisTest {
|SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')
""".stripMargin
val parsed7 = parsePlan(sql7)
val expected7 = AlterTableSerDePropertiesStatement(
Seq("a", "b", "c"),
val expected7 = AlterTableSerDeProperties(
UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
None,
Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -476,10 +476,13 @@ class ResolveSessionCatalog(
purge,
retainData = false)

case AlterTableSerDePropertiesStatement(tbl, serdeClassName, serdeProperties, partitionSpec) =>
val v1TableName = parseV1Table(tbl, "ALTER TABLE SerDe Properties")
case AlterTableSerDeProperties(
ResolvedV1TableIdentifier(ident),
serdeClassName,
serdeProperties,
partitionSpec) =>
AlterTableSerDePropertiesCommand(
v1TableName.asTableIdentifier,
ident.asTableIdentifier,
serdeClassName,
serdeProperties,
partitionSpec)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
throw new AnalysisException(
"ALTER TABLE ... RECOVER PARTITIONS is not supported for v2 tables.")

case AlterTableSerDeProperties(_: ResolvedTable, _, _, _) =>
throw new AnalysisException(
"ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] is not supported for v2 tables.")

case LoadData(_: ResolvedTable, _, _, _, _) =>
throw new AnalysisException("LOAD DATA is not supported for v2 tables.")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2120,7 +2120,8 @@ class DataSourceV2SQLSuite
val e = intercept[AnalysisException] {
sql(s"ALTER TABLE $t SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')")
}
assert(e.message.contains("ALTER TABLE SerDe Properties is only supported with v1 tables"))
assert(e.message.contains(
"ALTER TABLE ... SET [SERDE|SERDEPROPERTIES] is not supported for v2 tables"))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,9 +140,15 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
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')")
assertAnalysisError(
s"ALTER TABLE $viewName SET SERDE 'whatever'",
s"$viewName is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table")
assertAnalysisError(
s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'",
s"$viewName is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table")
assertAnalysisError(
s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')",
s"$viewName is a temp view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table")
assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')")
assertAnalysisError(
s"ALTER TABLE $viewName RECOVER PARTITIONS",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -888,12 +888,17 @@ class HiveDDLSuite

assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'")

assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDE 'whatever'")
assertAnalysisError(
s"ALTER TABLE $oldViewName SET SERDE 'whatever'",
s"$oldViewName is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.")

assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')")
assertAnalysisError(
s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')",
s"$oldViewName is a view. 'ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]' expects a table.")

assertErrorForAlterTableOnView(
s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')")
assertAnalysisError(
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.")

assertAnalysisError(
s"ALTER TABLE $oldViewName RECOVER PARTITIONS",
Expand Down