Repository: spark Updated Branches: refs/heads/master 39fd46907 -> 257375019
[SPARK-15421][SQL] Validate DDL property values ## What changes were proposed in this pull request? When we parse DDLs involving table or database properties, we need to validate the values. E.g. if we alter a database's property without providing a value: ``` ALTER DATABASE my_db SET DBPROPERTIES('some_key') ``` Then we'll ignore it with Hive, but override the property with the in-memory catalog. Inconsistencies like these arise because we don't validate the property values. In such cases, we should throw exceptions instead. ## How was this patch tested? `DDLCommandSuite` Author: Andrew Or <and...@databricks.com> Closes #13205 from andrewor14/ddl-prop-values. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/25737501 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/25737501 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/25737501 Branch: refs/heads/master Commit: 257375019266ab9e3c320e33026318cc31f58ada Parents: 39fd469 Author: Andrew Or <and...@databricks.com> Authored: Thu May 19 23:43:01 2016 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu May 19 23:43:01 2016 -0700 ---------------------------------------------------------------------- .../spark/sql/execution/SparkSqlParser.scala | 45 ++++++++++++++++---- .../sql/execution/command/DDLCommandSuite.scala | 41 ++++++++++++++++++ 2 files changed, 77 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/25737501/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---------------------------------------------------------------------- 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 8af6d07..ee12bfa 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 @@ -293,7 +293,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (external) { throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } - val options = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText val partitionColumnNames = Option(ctx.partitionColumnNames) @@ -371,6 +371,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Convert a table property list into a key-value map. + * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. */ override def visitTablePropertyList( ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { @@ -382,6 +383,32 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** + * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. + */ + private def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v == null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props + } + + /** + * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. + */ + private def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v != null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props.keys.toSeq + } + + /** * A table property key can either be String or a collection of dot separated elements. This * function extracts the property key based on whether its a string literal or a table property * identifier. @@ -409,7 +436,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null, Option(ctx.locationSpec).map(visitLocationSpec), Option(ctx.comment).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -424,7 +451,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterDatabaseProperties( ctx.identifier.getText, - visitTablePropertyList(ctx.tablePropertyList)) + visitPropertyKeyValues(ctx.tablePropertyList)) } /** @@ -540,7 +567,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableSetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList), + visitPropertyKeyValues(ctx.tablePropertyList), ctx.VIEW != null) } @@ -557,7 +584,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableUnsetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList).keys.toSeq, + visitPropertyKeys(ctx.tablePropertyList), ctx.EXISTS != null, ctx.VIEW != null) } @@ -575,7 +602,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { AlterTableSerDeProperties( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.STRING).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues), // TODO a partition spec is allowed to have optional values. This is currently violated. Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } @@ -783,7 +810,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val comment = Option(ctx.STRING).map(string) val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitCatalogColumns) val cols = Option(ctx.columns).toSeq.flatMap(visitCatalogColumns) - val properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val selectQuery = Option(ctx.query).map(plan) // Note: Hive requires partition columns to be distinct from the schema, so we need @@ -944,7 +971,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { import ctx._ EmptyStorageFormat.copy( serde = Option(string(name)), - serdeProperties = Option(tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -1001,7 +1028,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { comment = Option(ctx.STRING).map(string), schema, ctx.query, - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), ctx.EXISTS != null, ctx.REPLACE != null, ctx.TEMPORARY != null http://git-wip-us.apache.org/repos/asf/spark/blob/25737501/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala ---------------------------------------------------------------------- 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 897170e..0925a51 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 @@ -57,6 +57,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed, expected) } + test("create database - property values must be set") { + assertUnsupported( + sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("drop database") { val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" @@ -121,6 +127,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } + test("alter database - property values must be set") { + assertUnsupported( + sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("describe database") { // DESCRIBE DATABASE [EXTENDED] db_name; val sql1 = "DESCRIBE DATABASE EXTENDED db_name" @@ -228,6 +240,16 @@ class DDLCommandSuite extends PlanTest { } } + test("create table - property values must be set") { + assertUnsupported( + sql = "CREATE TABLE my_tab TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + assertUnsupported( + sql = "CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { @@ -349,6 +371,18 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed3_view, expected3_view) } + test("alter table - property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("alter table unset properties - property values must NOT be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_with_value")) + } + test("alter table: SerDe properties") { val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" val sql2 = @@ -404,6 +438,13 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed5, expected5) } + test("alter table - SerDe property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION partition_spec // [LOCATION 'location1'] partition_spec [LOCATION 'location2'] ...; test("alter table: add partition") { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org