From fc3c1684ad8e24ab9b05f0f7e02659ea2e365ebd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 4 Mar 2016 14:20:07 -0800 Subject: [PATCH 01/12] Move things into new ParserUtils object --- .../sql/catalyst/parser/CatalystQl.scala | 142 +-------------- .../sql/catalyst/parser/ParserUtils.scala | 170 ++++++++++++++++++ .../apache/spark/sql/execution/SparkQl.scala | 4 +- .../org/apache/spark/sql/hive/HiveQl.scala | 4 +- 4 files changed, 177 insertions(+), 143 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala index d2318417e3e68..44f7d8a056bcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler @@ -36,12 +35,7 @@ import org.apache.spark.util.random.RandomSampler * This class translates SQL to Catalyst [[LogicalPlan]]s or [[Expression]]s. */ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends ParserInterface { - object Token { - def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { - CurrentOrigin.setPosition(node.line, node.positionInLine) - node.pattern - } - } + import ParserUtils._ /** * The safeParse method allows a user to focus on the parsing/AST transformation logic. This @@ -82,102 +76,6 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends def parseTableIdentifier(sql: String): TableIdentifier = safeParse(sql, ParseDriver.parseTableName(sql, conf))(extractTableIdent) - def parseDdl(sql: String): Seq[Attribute] = { - safeParse(sql, ParseDriver.parseExpression(sql, conf)) { ast => - val Token("TOK_CREATETABLE", children) = ast - children - .find(_.text == "TOK_TABCOLLIST") - .getOrElse(sys.error("No columnList!")) - .flatMap(_.children.map(nodeToAttribute)) - } - } - - protected def getClauses( - clauseNames: Seq[String], - nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { - var remainingNodes = nodeList - val clauses = clauseNames.map { clauseName => - val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName) - remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) - matches.headOption - } - - if (remainingNodes.nonEmpty) { - sys.error( - s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}. - |You are likely trying to use an unsupported Hive feature."""".stripMargin) - } - clauses - } - - protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode = - getClauseOption(clauseName, nodeList).getOrElse(sys.error( - s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}")) - - protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = { - nodeList.filter { case ast: ASTNode => ast.text == clauseName } match { - case Seq(oneMatch) => Some(oneMatch) - case Seq() => None - case _ => sys.error(s"Found multiple instances of clause $clauseName") - } - } - - protected def nodeToAttribute(node: ASTNode): Attribute = node match { - case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) => - AttributeReference(colName, nodeToDataType(dataType), nullable = true)() - case _ => - noParseRule("Attribute", node) - } - - protected def nodeToDataType(node: ASTNode): DataType = node match { - case Token("TOK_DECIMAL", precision :: scale :: Nil) => - DecimalType(precision.text.toInt, scale.text.toInt) - case Token("TOK_DECIMAL", precision :: Nil) => - DecimalType(precision.text.toInt, 0) - case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT - case Token("TOK_BIGINT", Nil) => LongType - case Token("TOK_INT", Nil) => IntegerType - case Token("TOK_TINYINT", Nil) => ByteType - case Token("TOK_SMALLINT", Nil) => ShortType - case Token("TOK_BOOLEAN", Nil) => BooleanType - case Token("TOK_STRING", Nil) => StringType - case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType - case Token("TOK_CHAR", Token(_, Nil) :: Nil) => StringType - case Token("TOK_FLOAT", Nil) => FloatType - case Token("TOK_DOUBLE", Nil) => DoubleType - case Token("TOK_DATE", Nil) => DateType - case Token("TOK_TIMESTAMP", Nil) => TimestampType - case Token("TOK_BINARY", Nil) => BinaryType - case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) - case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) => - StructType(fields.map(nodeToStructField)) - case Token("TOK_MAP", keyType :: valueType :: Nil) => - MapType(nodeToDataType(keyType), nodeToDataType(valueType)) - case _ => - noParseRule("DataType", node) - } - - protected def nodeToStructField(node: ASTNode): StructField = node match { - case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) => - StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true) - case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: comment :: Nil) => - val meta = new MetadataBuilder().putString("comment", unquoteString(comment.text)).build() - StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true, meta) - case _ => - noParseRule("StructField", node) - } - - protected def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = { - tableNameParts.children.map { - case Token(part, Nil) => cleanIdentifier(part) - } match { - case Seq(tableOnly) => TableIdentifier(tableOnly) - case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName)) - case other => sys.error("Hive only supports tables names like 'tableName' " + - s"or 'databaseName.tableName', found '$other'") - } - } - /** * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) * is equivalent to @@ -625,42 +523,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C noParseRule("Select", node) } - protected val escapedIdentifier = "`(.+)`".r - protected val doubleQuotedString = "\"([^\"]+)\"".r - protected val singleQuotedString = "'([^']+)'".r - - protected def unquoteString(str: String) = str match { - case singleQuotedString(s) => s - case doubleQuotedString(s) => s - case other => other - } - - /** Strips backticks from ident if present */ - protected def cleanIdentifier(ident: String): String = ident match { - case escapedIdentifier(i) => i - case plainIdent => plainIdent - } - - /* Case insensitive matches */ - val COUNT = "(?i)COUNT".r - val SUM = "(?i)SUM".r - val AND = "(?i)AND".r - val OR = "(?i)OR".r - val NOT = "(?i)NOT".r - val TRUE = "(?i)TRUE".r - val FALSE = "(?i)FALSE".r - val LIKE = "(?i)LIKE".r - val RLIKE = "(?i)RLIKE".r - val REGEXP = "(?i)REGEXP".r - val IN = "(?i)IN".r - val DIV = "(?i)DIV".r - val BETWEEN = "(?i)BETWEEN".r - val WHEN = "(?i)WHEN".r - val CASE = "(?i)CASE".r - - val INTEGRAL = "[+-]?\\d+".r - val DECIMAL = "[+-]?((\\d+(\\.\\d*)?)|(\\.\\d+))".r - protected def nodeToExpr(node: ASTNode): Expression = node match { /* Attribute References */ case Token("TOK_TABLE_OR_COL", Token(name, Nil) :: Nil) => @@ -1007,6 +869,4 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node) - protected def noParseRule(msg: String, node: ASTNode): Nothing = throw new NotImplementedError( - s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala new file mode 100644 index 0000000000000..c105b53f1fc62 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.parser + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.types._ + + +/** + * A collection of utility methods and patterns for parsing query texts. + */ +// TODO: merge with ParseUtils +object ParserUtils { + + object Token { + def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { + CurrentOrigin.setPosition(node.line, node.positionInLine) + node.pattern + } + } + + private val escapedIdentifier = "`(.+)`".r + private val doubleQuotedString = "\"([^\"]+)\"".r + private val singleQuotedString = "'([^']+)'".r + + // Token patterns + val COUNT = "(?i)COUNT".r + val SUM = "(?i)SUM".r + val AND = "(?i)AND".r + val OR = "(?i)OR".r + val NOT = "(?i)NOT".r + val TRUE = "(?i)TRUE".r + val FALSE = "(?i)FALSE".r + val LIKE = "(?i)LIKE".r + val RLIKE = "(?i)RLIKE".r + val REGEXP = "(?i)REGEXP".r + val IN = "(?i)IN".r + val DIV = "(?i)DIV".r + val BETWEEN = "(?i)BETWEEN".r + val WHEN = "(?i)WHEN".r + val CASE = "(?i)CASE".r + val INTEGRAL = "[+-]?\\d+".r + val DECIMAL = "[+-]?((\\d+(\\.\\d*)?)|(\\.\\d+))".r + + /** + * Strip quotes, if any, from the string. + */ + def unquoteString(str: String): String = { + str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + } + + /** + * Strip backticks, if any, from the string. + */ + def cleanIdentifier(ident: String): String = { + ident match { + case escapedIdentifier(i) => i + case plainIdent => plainIdent + } + } + + def getClauses( + clauseNames: Seq[String], + nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { + var remainingNodes = nodeList + val clauses = clauseNames.map { clauseName => + val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName) + remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) + matches.headOption + } + + if (remainingNodes.nonEmpty) { + sys.error( + s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}. + |You are likely trying to use an unsupported Hive feature."""".stripMargin) + } + clauses + } + + def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode = { + getClauseOption(clauseName, nodeList).getOrElse(sys.error( + s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}")) + } + + def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = { + nodeList.filter { case ast: ASTNode => ast.text == clauseName } match { + case Seq(oneMatch) => Some(oneMatch) + case Seq() => None + case _ => sys.error(s"Found multiple instances of clause $clauseName") + } + } + + def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = { + tableNameParts.children.map { + case Token(part, Nil) => cleanIdentifier(part) + } match { + case Seq(tableOnly) => TableIdentifier(tableOnly) + case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName)) + case other => sys.error("Hive only supports tables names like 'tableName' " + + s"or 'databaseName.tableName', found '$other'") + } + } + + def nodeToDataType(node: ASTNode): DataType = node match { + case Token("TOK_DECIMAL", precision :: scale :: Nil) => + DecimalType(precision.text.toInt, scale.text.toInt) + case Token("TOK_DECIMAL", precision :: Nil) => + DecimalType(precision.text.toInt, 0) + case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT + case Token("TOK_BIGINT", Nil) => LongType + case Token("TOK_INT", Nil) => IntegerType + case Token("TOK_TINYINT", Nil) => ByteType + case Token("TOK_SMALLINT", Nil) => ShortType + case Token("TOK_BOOLEAN", Nil) => BooleanType + case Token("TOK_STRING", Nil) => StringType + case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType + case Token("TOK_CHAR", Token(_, Nil) :: Nil) => StringType + case Token("TOK_FLOAT", Nil) => FloatType + case Token("TOK_DOUBLE", Nil) => DoubleType + case Token("TOK_DATE", Nil) => DateType + case Token("TOK_TIMESTAMP", Nil) => TimestampType + case Token("TOK_BINARY", Nil) => BinaryType + case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) + case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) => + StructType(fields.map(nodeToStructField)) + case Token("TOK_MAP", keyType :: valueType :: Nil) => + MapType(nodeToDataType(keyType), nodeToDataType(valueType)) + case _ => + noParseRule("DataType", node) + } + + def nodeToStructField(node: ASTNode): StructField = node match { + case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) => + StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true) + case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: comment :: Nil) => + val meta = new MetadataBuilder().putString("comment", unquoteString(comment.text)).build() + StructField(cleanIdentifier(fieldName), nodeToDataType(dataType), nullable = true, meta) + case _ => + noParseRule("StructField", node) + } + + /** + * Throw an exception because we cannot parse the given node. + */ + def noParseRule(msg: String, node: ASTNode): Nothing = { + throw new NotImplementedError( + s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}") + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 9143258abbc5b..471a5e436c8ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, SimpleParserConf} +import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) { + import ParserUtils._ + /** Check if a command should not be explained. */ protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 812e1fe51a03e..0bdebdc5fd741 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser._ -import org.apache.spark.sql.catalyst.parser.ParseUtils._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkQl @@ -81,6 +80,9 @@ private[hive] case class CreateViewAsSelect( /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging { + import ParseUtils._ + import ParserUtils._ + protected val nativeCommands = Seq( "TOK_ALTERDATABASE_OWNER", "TOK_ALTERDATABASE_PROPERTIES", From 010afddf40d776b3009cee98057e74d499c45012 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Mar 2016 15:56:09 -0800 Subject: [PATCH 02/12] Port over viirya's changes in #11048 --- .../org/apache/spark/sql/SQLContext.scala | 9 + .../apache/spark/sql/execution/SparkQl.scala | 78 ++- .../command/AlterTableCommandParser.scala | 395 +++++++++++ .../spark/sql/execution/command/ddl.scala | 186 +++++ .../sql/execution/datasources/bucket.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 652 ++++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 9 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 18 +- .../spark/sql/sources/BucketedReadSuite.scala | 2 +- 10 files changed, 1325 insertions(+), 30 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c742bf2f8923f..3cdb4a8ffa84a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -784,6 +784,15 @@ class SQLContext private[sql]( DataFrame(this, parseSql(sqlText)) } + /** + * Executes a SQL query without parsing it, but instead passing it directly to an underlying + * system to process. This is currently only used for Hive DDLs and will be removed as soon + * as Spark can parse all supported Hive DDLs itself. + */ + private[sql] def runNativeSql(sqlText: String): Seq[Row] = { + throw new UnsupportedOperationException + } + /** * Returns the specified table as a [[DataFrame]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 471a5e436c8ee..7d782a4d8fb6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -29,7 +29,21 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly import ParserUtils._ /** Check if a command should not be explained. */ - protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command + protected def isNoExplainCommand(command: String): Boolean = + "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command + + protected def extractProps( + node: ASTNode, + firstLevelNodeStr: String, + secondLevelNodeStr: String): Seq[(String, String)] = node match { + case Token(firstLevelNodeStr, options) => + options.map { + case Token(secondLevelNodeStr, keysAndValue) => + val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + } + } protected override def nodeToPlan(node: ASTNode): LogicalPlan = { node match { @@ -64,6 +78,56 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly val tableIdent = extractTableIdent(nameParts) RefreshTable(tableIdent) + case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) => + val Seq( + allowExisting, + dbLocation, + databaseComment, + dbprops) = getClauses(Seq( + "TOK_IFNOTEXISTS", + "TOK_DATABASELOCATION", + "TOK_DATABASECOMMENT", + "TOK_DATABASEPROPERTIES"), createDatabaseArgs) + + val location = dbLocation.map { + case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc) + } + val comment = databaseComment.map { + case Token("TOK_DATABASECOMMENT", Token(comment, Nil) :: Nil) => unquoteString(comment) + } + val props: Map[String, String] = dbprops.toSeq.flatMap { + case Token("TOK_DATABASEPROPERTIES", propList) => + propList.flatMap(extractProps(_, "TOK_DBPROPLIST", "TOK_TABLEPROPERTY")) + }.toMap + + CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) + + case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) => + val funcName = func.map(x => unquoteString(x.text)).mkString(".") + val asName = unquoteString(as.text) + val Seq( + rList, + temp) = getClauses(Seq( + "TOK_RESOURCE_LIST", + "TOK_TEMPORARY"), createFuncArgs) + + val resourcesMap: Map[String, String] = rList.toSeq.flatMap { + case Token("TOK_RESOURCE_LIST", resources) => + resources.map { + case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) => + val resourceType = rType match { + case Token("TOK_JAR", Nil) => "jar" + case Token("TOK_FILE", Nil) => "file" + case Token("TOK_ARCHIVE", Nil) => "archive" + } + (resourceType, unquoteString(rPath)) + } + }.toMap + CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source) + + case Token("TOK_ALTERTABLE", alterTableArgs) => + AlterTableCommandParser.parse(node) + case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( temp, @@ -90,16 +154,8 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case Token(name, Nil) => name }.mkString(".") - val options: Map[String, String] = tableOpts.toSeq.flatMap { - case Token("TOK_TABLEOPTIONS", options) => - options.map { - case Token("TOK_TABLEOPTION", keysAndValue) => - val key = keysAndValue.init.map(_.text).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - } - }.toMap - + val options: Map[String, String] = + tableOpts.toSeq.flatMap(extractProps(_, "TOK_TABLEOPTIONS", "TOK_TABLEOPTION")).toMap val asClause = tableAs.map(nodeToPlan(_)) if (temp.isDefined && allowExisting.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala new file mode 100644 index 0000000000000..b99753b4ddabd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -0,0 +1,395 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} +import org.apache.spark.sql.catalyst.parser._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.types.StructType + +object AlterTableCommandParser { + import ParserUtils._ + + private def cleanAndUnquoteString(s: String): String = { + cleanIdentifier(unquoteString(s)) + } + + private def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = { + node match { + case Token("TOK_PARTSPEC", partitions) => + val spec = partitions.map { + case Token("TOK_PARTVAL", ident :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), Some(cleanAndUnquoteString(constant.text))) + case Token("TOK_PARTVAL", ident :: Nil) => + (cleanAndUnquoteString(ident.text), None) + }.toMap + Some(spec) + case _ => None + } + } + + private def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match { + case Token("TOK_TABLEPROPERTIES", propsList) => + propsList.flatMap { + case Token("TOK_TABLEPROPLIST", props) => + props.map { + case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) => + val k = cleanAndUnquoteString(key.text) + (k, None) + case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = cleanAndUnquoteString(key.text) + val v = cleanAndUnquoteString(value.text) + (k, Some(v)) + } + }.toMap + } + + def parse(v1: ASTNode): LogicalPlan = v1.children match { + case (tabName @ Token("TOK_TABNAME", _)) :: restNodes => + val tableIdent: TableIdentifier = extractTableIdent(tabName) + val partitionSpec = getClauseOption("TOK_PARTSPEC", v1.children) + val partition = partitionSpec.flatMap(parsePartitionSpec) + matchAlterTableCommands(v1, restNodes, tableIdent, partition) + case _ => + throw new NotImplementedError(v1.text) + } + + private def matchAlterTableCommands( + node: ASTNode, + nodes: Seq[ASTNode], + tableIdent: TableIdentifier, + partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match { + case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => + val renamedTable = getClause("TOK_TABNAME", renameArgs) + val renamedTableIdent: TableIdentifier = extractTableIdent(renamedTable) + AlterTableRename(tableIdent, renamedTableIdent)(node.source) + + case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => + val setTableProperties = extractTableProps(args.head) + AlterTableSetProperties( + tableIdent, + setTableProperties)(node.source) + + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => + val dropTableProperties = extractTableProps(args.head) + val allowExisting = getClauseOption("TOK_IFEXISTS", args) + AlterTableDropProperties( + tableIdent, + dropTableProperties, allowExisting.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ => + // When SET SERDE serde_classname WITH SERDEPROPERTIES, this is None + val serdeProperties: Option[Map[String, Option[String]]] = + serdeArgs.headOption.map(extractTableProps) + + AlterTableSerDeProperties( + tableIdent, + Some(cleanAndUnquoteString(serdeClassName)), + serdeProperties, + partition)(node.source) + + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => + val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) + + AlterTableSerDeProperties( + tableIdent, + None, + Some(serdeProperties), + partition)(node.source) + + case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) :: _ => + val (buckets, noClustered, noSorted) = clusterAndSoryByArgs match { + case Token("TOK_ALTERTABLE_BUCKETS", bucketArgsHead :: bucketArgs) => + val bucketCols = bucketArgsHead.children.map(_.text) + + val (sortCols, sortDirections, numBuckets) = { + if (bucketArgs.head.text == "TOK_TABCOLNAME") { + val (cols, directions) = bucketArgs.head.children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => + (colName, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => + (colName, Descending) + }.unzip + (cols, directions, bucketArgs.last.text.toInt) + } else { + (Nil, Nil, bucketArgs.head.text.toInt) + } + } + + (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)), + false, false) + case Token("TOK_NOT_CLUSTERED", Nil) => + (None, true, false) + case Token("TOK_NOT_SORTED", Nil) => + (None, false, true) + } + + AlterTableStoreProperties( + tableIdent, + buckets, + noClustered, + noSorted)(node.source) + + case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => + val num = bucketNum.toInt + val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) + AlterTableStoreProperties( + tableIdent, + buckets, + false, + false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => + // ALTER TABLE table_name NOT SKEWED + AlterTableSkewed(tableIdent, Nil, Nil, false, true, false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => + // ALTER TABLE table_name NOT STORED AS DIRECTORIES + AlterTableSkewed(tableIdent, Nil, Nil, false, false, true)(node.source) + + case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: _ => + val skewedArgs = tableSkewed match { + case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) => + skewedArgs match { + case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => + val cols = skewedCols.children.map(n => cleanAndUnquoteString(n.text)) + val values = skewedValues match { + case Token("TOK_TABCOLVALUE", values) => + Seq(values.map(n => cleanAndUnquoteString(n.text))) + case Token("TOK_TABCOLVALUE_PAIR", pairs) => + pairs.map { + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => + vals.map(n => cleanAndUnquoteString(n.text)) + } + } + } + + val storedAsDirs = stored match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false + } + + (cols, values, storedAsDirs) + } + } + + val (cols, values, storedAsDirs) = skewedArgs + + AlterTableSkewed( + tableIdent, + cols, + values, + storedAsDirs, + notSkewed = false, + notStoredAsDirs = false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED_LOCATION", + Token("TOK_SKEWED_LOCATIONS", + Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => + val skewedMaps = locationMaps.map { + case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => + val k = key match { + case Token(const, Nil) => Seq(cleanAndUnquoteString(const)) + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => + vals.map(n => cleanAndUnquoteString(n.text)) + } + } + (k, cleanAndUnquoteString(value.text)) + }.toMap + AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) + + case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: _ => + val (allowExisting, parts) = addPartsArgs match { + case Token("TOK_IFNOTEXISTS", Nil) :: others => (true, others) + case _ => (false, addPartsArgs) + } + + val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] = + new ArrayBuffer() + var currentPart: Map[String, Option[String]] = null + parts.map { + case t @ Token("TOK_PARTSPEC", partArgs) => + if (currentPart != null) { + partitions += ((currentPart, None)) + } + currentPart = parsePartitionSpec(t).get + case Token("TOK_PARTITIONLOCATION", loc :: Nil) => + val location = unquoteString(loc.text) + if (currentPart != null) { + partitions += ((currentPart, Some(location))) + currentPart = null + } else { + // We should not reach here + throw new AnalysisException("Partition location must follow a partition spec.") + } + } + + if (currentPart != null) { + partitions += ((currentPart, None)) + } + AlterTableAddPartition(tableIdent, partitions, allowExisting)(node.source) + + case Token("TOK_ALTERTABLE_RENAMEPART", partArg :: Nil) :: _ => + val Some(newPartition) = parsePartitionSpec(partArg) + AlterTableRenamePartition(tableIdent, partition.get, newPartition)(node.source) + + case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", + (p @ Token("TOK_PARTSPEC", _)) :: (t @ Token("TOK_TABNAME", _)) :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(p) + val fromTableIdent = extractTableIdent(t) + AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ => + val parts = args.collect { + case Token("TOK_PARTSPEC", partitions) => + partitions.map { + case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), + op.text, cleanAndUnquoteString(constant.text)) + } + } + + val allowExisting = getClauseOption("TOK_IFEXISTS", args).isDefined + + val purge = getClauseOption("PURGE", args) + + val replication = getClauseOption("TOK_REPLICATION", args).map { + case Token("TOK_REPLICATION", replId :: metadata) => + (cleanAndUnquoteString(replId.text), metadata.nonEmpty) + } + + AlterTableDropPartition( + tableIdent, + parts, + allowExisting, + purge.isDefined, + replication)(node.source) + + case Token("TOK_ALTERTABLE_ARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) + AlterTableArchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_UNARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) + AlterTableUnarchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => + val Seq(fileFormat, genericFormat) = + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), + args) + val fFormat = fileFormat.map(_.children.map(n => cleanAndUnquoteString(n.text))) + val gFormat = genericFormat.map(f => cleanAndUnquoteString(f.children(0).text)) + AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + + case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => + AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) + + case Token("TOK_ALTERTABLE_TOUCH", args) :: _ => + val part = getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) + AlterTableTouch(tableIdent, part)(node.source) + + case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => + AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) + + case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => + AlterTableMerge(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_RENAMECOL", args) :: _ => + val oldName = args(0).text + val newName = args(1).text + val dataType = nodeToDataType(args(2)) + val afterPos = + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) + val afterPosCol = afterPos.map { ap => + ap.children match { + case Token(col, Nil) :: Nil => col + case _ => null + } + } + + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + + val comment = if (args.size > 3) { + args(3) match { + case Token(commentStr, Nil) + if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && + commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => + Some(cleanAndUnquoteString(commentStr)) + case _ => + None + } + } else { + None + } + + AlterTableChangeCol( + tableIdent, + partition, + oldName, + newName, + dataType, + comment, + afterPos.isDefined, + afterPosCol, + restrict.isDefined, + cascade.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + } + + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + + AlterTableAddCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + } + + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + + AlterTableReplaceCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + } +} + 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 new file mode 100644 index 0000000000000..ec31c7f058cfb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.Logging +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + + +/** + * A DDL command expected to be run in the underlying system without Spark parsing the + * query text. + */ +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.runNativeSql(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class CreateDataBase( + databaseName: String, + allowExisting: Boolean, + path: Option[String], + comment: Option[String], + props: Map[String, String])(sql: String) extends NativeDDLCommands(sql) with Logging + +case class CreateFunction( + functionName: String, + asName: String, + resourcesMap: Map[String, String], + isTemp: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableRename( + tableName: TableIdentifier, + renameTableName: TableIdentifier)(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableSetProperties( + tableName: TableIdentifier, + setProperties: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableDropProperties( + tableName: TableIdentifier, + dropProperties: Map[String, Option[String]], + allowExisting: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableSerDeProperties( + tableName: TableIdentifier, + serdeClassName: Option[String], + serdeProperties: Option[Map[String, Option[String]]], + partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableStoreProperties( + tableName: TableIdentifier, + buckets: Option[BucketSpec], + noClustered: Boolean, + noSorted: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableSkewed( + tableName: TableIdentifier, + skewedCols: Seq[String], + skewedValues: Seq[Seq[String]], + storedAsDirs: Boolean, + notSkewed: Boolean, + // TODO: wtf? + notStoredAsDirs: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableSkewedLocation( + tableName: TableIdentifier, + skewedMap: Map[Seq[String], String])(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableAddPartition( + tableName: TableIdentifier, + partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], + allowExisting: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + +case class AlterTableRenamePartition( + tableName: TableIdentifier, + oldPartition: Map[String, Option[String]], + newPartition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableExchangePartition( + tableName: TableIdentifier, + fromTableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableDropPartition( + tableName: TableIdentifier, + partitions: Seq[Seq[(String, String, String)]], + allowExisting: Boolean, + purge: Boolean, + replication: Option[(String, Boolean)])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableArchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableUnarchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableSetFileFormat( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + fileFormat: Option[Seq[String]], + genericFormat: Option[String])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableSetLocation( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + location: String)(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableTouch( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableCompact( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + compactType: String)(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableMerge( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableChangeCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + oldColName: String, + newColName: String, + dataType: DataType, + comment: Option[String], + afterPos: Boolean, + afterPosCol: Option[String], + restrict: Boolean, + cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableAddCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) + with Logging + +case class AlterTableReplaceCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) + with Logging diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 3e0d484b74cfe..d73647c8efaf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions.SortDirection import org.apache.spark.sql.sources.{HadoopFsRelation, HadoopFsRelationProvider, OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.StructType @@ -35,7 +36,8 @@ import org.apache.spark.sql.types.StructType private[sql] case class BucketSpec( numBuckets: Int, bucketColumnNames: Seq[String], - sortColumnNames: Seq[String]) + sortColumnNames: Seq[String], + sortDirections: Seq[SortDirection] = Nil) private[sql] trait BucketedHadoopFsRelationProvider extends HadoopFsRelationProvider { final override def createRelation( 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 new file mode 100644 index 0000000000000..049060d88f191 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -0,0 +1,652 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.execution.SparkQl +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + +class DDLSuite extends PlanTest { + val parser = new SparkQl() + + test("create database") { + val sql = + """ + |CREATE DATABASE IF NOT EXISTS database_name + |COMMENT 'database_comment' LOCATION '/home/user/db' + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + + val parsed = parser.parsePlan(sql) + + val expected = CreateDataBase( + "database_name", + true, + Some("/home/user/db"), + Some("database_comment"), + Map("a" -> "a", "b" -> "b", "c" -> "c"))(sql) + + comparePlans(parsed, expected) + } + + test("create function") { + val sql = + """ + |CREATE TEMPORARY FUNCTION helloworld as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', + |FILE 'path/to/file' + """.stripMargin + + val parsed = parser.parsePlan(sql) + + val expected = CreateFunction( + "helloworld", + "com.matthewrathbone.example.SimpleUDFExample", + Map("jar" -> "/path/to/jar", "file" -> "path/to/file"), + true)(sql) + + comparePlans(parsed, expected) + } + + test("alter table: rename table") { + val sql = "ALTER TABLE table_name RENAME TO new_table_name" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRename( + TableIdentifier("table_name", None), + TableIdentifier("new_table_name", None))(sql) + comparePlans(parsed, expected) + } + + test("alter table: alter table properties") { + val sql1 = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2 = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3 = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + + val expected1 = AlterTableSetProperties( + TableIdentifier("table_name", None), + Map("test" -> Some("test"), "comment" -> Some("new_comment")))(sql1) + + val expected2 = AlterTableDropProperties( + TableIdentifier("table_name", None), + Map("comment" -> None, "test" -> None), + false)(sql2) + + val expected3 = AlterTableDropProperties( + TableIdentifier("table_name", None), + Map("comment" -> None, "test" -> None), + true)(sql3) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: SerDe properties") { + val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" + val sql2 = + """ + |ALTER TABLE table_name SET SERDE 'org.apache.class' + |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + + val sql3 = + """ + |ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + + val sql4 = + """ + |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', + |country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + + val sql5 = + """ + |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', + |country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + + val expected1 = AlterTableSerDeProperties( + TableIdentifier("table_name", None), + Some("org.apache.class"), + None, + None)(sql1) + + val expected2 = AlterTableSerDeProperties( + TableIdentifier("table_name", None), + Some("org.apache.class"), + Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), + None)(sql2) + + val expected3 = AlterTableSerDeProperties( + TableIdentifier("table_name", None), + None, + Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), + None)(sql3) + + val expected4 = AlterTableSerDeProperties( + TableIdentifier("table_name", None), + Some("org.apache.class"), + Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), + Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql4) + + val expected5 = AlterTableSerDeProperties( + TableIdentifier("table_name", None), + None, + Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), + Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql5) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("alter table: storage properties") { + val sql1 = "ALTER TABLE table_name CLUSTERED BY (dt, country) INTO 10 BUCKETS" + + val sql2 = "ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + + "(dt, country DESC) INTO 10 BUCKETS" + + val sql3 = "ALTER TABLE table_name INTO 20 BUCKETS" + val sql4 = "ALTER TABLE table_name NOT CLUSTERED" + val sql5 = "ALTER TABLE table_name NOT SORTED" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + + val expected1 = AlterTableStoreProperties( + TableIdentifier("table_name", None), + Some(BucketSpec(10, List("dt", "country"), List(), List())), + false, + false)(sql1) + + val expected2 = AlterTableStoreProperties( + TableIdentifier("table_name", None), + Some(BucketSpec(10, List("dt", "country"), List("dt", "country"), + List(Ascending, Descending))), + false, + false)(sql2) + + val expected3 = AlterTableStoreProperties( + TableIdentifier("table_name", None), + Some(BucketSpec(20, List(), List(), List())), + false, + false)(sql3) + + val expected4 = AlterTableStoreProperties( + TableIdentifier("table_name", None), + None, + true, + false)(sql4) + + val expected5 = AlterTableStoreProperties( + TableIdentifier("table_name", None), + None, + false, + true)(sql5) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("alter table: skewed") { + val sql1 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES + """.stripMargin + + val sql2 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |('2008-08-08', 'us') STORED AS DIRECTORIES + """.stripMargin + + val sql3 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + + val expected1 = AlterTableSkewed( + TableIdentifier("table_name", None), + Seq("dt", "country"), + Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), + true, + false, + false)(sql1) + + val expected2 = AlterTableSkewed( + TableIdentifier("table_name", None), + Seq("dt", "country"), + Seq(List("2008-08-08", "us")), + true, + false, + false)(sql2) + + val expected3 = AlterTableSkewed( + TableIdentifier("table_name", None), + Seq("dt", "country"), + Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), + false, + false, + false)(sql3) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: skewed location") { + val sql1 = + """ + |ALTER TABLE table_name SET SKEWED LOCATION + |('123'='location1', 'test'='location2') + """.stripMargin + + val sql2 = + """ + |ALTER TABLE table_name SET SKEWED LOCATION + |(('2008-08-08', 'us')='location1', 'test'='location2') + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableSkewedLocation( + TableIdentifier("table_name", None), + Map(List("123") -> "location1", List("test") -> "location2"))(sql1) + + val expected2 = AlterTableSkewedLocation( + TableIdentifier("table_name", None), + Map(List("2008-08-08", "us") -> "location1", List("test") -> "location2"))(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: add partition") { + val sql = + """ + |ALTER TABLE table_name ADD IF NOT EXISTS PARTITION + |(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION + |(dt='2009-09-09', country='uk') + """.stripMargin + + val parsed = parser.parsePlan(sql) + + val expected = AlterTableAddPartition( + TableIdentifier("table_name", None), + Seq( + (Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), Some("location1")), + (Map("dt" -> Some("2009-09-09"), "country" -> Some("uk")), None)), + true)(sql) + + comparePlans(parsed, expected) + } + + test("alter table: rename partition") { + val sql = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |RENAME TO PARTITION (dt='2008-09-09', country='uk') + """.stripMargin + + val parsed = parser.parsePlan(sql) + + val expected = AlterTableRenamePartition( + TableIdentifier("table_name", None), + Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), + Map("dt" -> Some("2008-09-09"), "country" -> Some("uk")))(sql) + + comparePlans(parsed, expected) + } + + test("alter table: exchange partition") { + val sql = + """ + |ALTER TABLE table_name_1 EXCHANGE PARTITION + |(dt='2008-08-08', country='us') WITH TABLE table_name_2 + """.stripMargin + + val parsed = parser.parsePlan(sql) + + val expected = AlterTableExchangePartition( + TableIdentifier("table_name_1", None), + TableIdentifier("table_name_2", None), + Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + + comparePlans(parsed, expected) + } + + test("alter table: drop partitions") { + val sql1 = + """ + |ALTER TABLE table_name DROP IF EXISTS PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + + val sql2 = + """ + |ALTER TABLE table_name DROP IF EXISTS PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + |PURGE FOR METADATA REPLICATION ('test') + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableDropPartition( + TableIdentifier("table_name", None), + Seq( + List(("dt", "=", "2008-08-08"), ("country", "=", "us")), + List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), + true, + false, + None)(sql1) + + val expected2 = AlterTableDropPartition( + TableIdentifier("table_name", None), + Seq( + List(("dt", "=", "2008-08-08"), ("country", "=", "us")), + List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), + true, + true, + Some(("test", true)))(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: archive partition") { + val sql = "ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')" + val parsed = parser.parsePlan(sql) + + val expected = AlterTableArchivePartition( + TableIdentifier("table_name", None), + Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + + comparePlans(parsed, expected) + } + + test("alter table: unarchive partition") { + val sql = "ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')" + val parsed = parser.parsePlan(sql) + + val expected = AlterTableUnarchivePartition( + TableIdentifier("table_name", None), + Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + + comparePlans(parsed, expected) + } + + test("alter table: set file format") { + val sql1 = + """ + |ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' + |OUTPUTFORMAT 'test' SERDE 'test' INPUTDRIVER 'test' OUTPUTDRIVER 'test' + """.stripMargin + + val sql2 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + + "OUTPUTFORMAT 'test' SERDE 'test'" + + val sql3 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET FILEFORMAT PARQUET" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + + val expected1 = AlterTableSetFileFormat( + TableIdentifier("table_name", None), + None, + Some(List("test", "test", "test", "test", "test")), + None)(sql1) + + val expected2 = AlterTableSetFileFormat( + TableIdentifier("table_name", None), + None, + Some(List("test", "test", "test")), + None)(sql2) + + val expected3 = AlterTableSetFileFormat( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + None, + Some("PARQUET"))(sql3) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: set location") { + val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET LOCATION 'new location'" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableSetLocation( + TableIdentifier("table_name", None), + None, + "new location")(sql1) + + val expected2 = AlterTableSetLocation( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + "new location")(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: touch") { + val sql1 = "ALTER TABLE table_name TOUCH" + val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableTouch( + TableIdentifier("table_name", None), + None)(sql1) + + val expected2 = AlterTableTouch( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: compact") { + val sql1 = "ALTER TABLE table_name COMPACT 'compaction_type'" + val sql2 = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |COMPACT 'MAJOR' + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableCompact( + TableIdentifier("table_name", None), + None, + "compaction_type")(sql1) + + val expected2 = AlterTableCompact( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + "MAJOR")(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: concatenate") { + val sql1 = "ALTER TABLE table_name CONCATENATE" + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableMerge( + TableIdentifier("table_name", None), + None)(sql1) + + val expected2 = AlterTableMerge( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: change column name/type/position/comment") { + val sql1 = "ALTER TABLE table_name CHANGE col_old_name col_new_name INT" + + val sql2 = + """ + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' FIRST CASCADE + """.stripMargin + + val sql3 = + """ + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' AFTER column_name RESTRICT + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + + val expected1 = AlterTableChangeCol( + TableIdentifier("table_name", None), + None, + "col_old_name", + "col_new_name", + IntegerType, + None, + false, + None, + false, + false)(sql1) + + val expected2 = AlterTableChangeCol( + TableIdentifier("table_name", None), + None, + "col_old_name", + "col_new_name", + IntegerType, + Some("col_comment"), + false, + None, + false, + true)(sql2) + + val expected3 = AlterTableChangeCol( + TableIdentifier("table_name", None), + None, + "col_old_name", + "col_new_name", + IntegerType, + Some("col_comment"), + true, + Some("column_name"), + true, + false)(sql3) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: add/replace columns") { + val sql1 = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG + |COMMENT 'test_comment2') CASCADE + """.stripMargin + + val sql2 = + """ + |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT + |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT + """.stripMargin + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() + val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() + + val expected1 = AlterTableAddCol( + TableIdentifier("table_name", None), + Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + StructType(Seq( + StructField("new_col1", IntegerType, true, meta1), + StructField("new_col2", LongType, true, meta2))), + false, + true)(sql1) + + val expected2 = AlterTableReplaceCol( + TableIdentifier("table_name", None), + None, + StructType(Seq( + StructField("new_col1", IntegerType, true, meta1), + StructField("new_col2", LongType, true, meta2))), + true, + false)(sql2) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 69669d79be2d8..081d849a88886 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -535,6 +535,15 @@ class HiveContext private[hive]( } } + /** + * Executes a SQL query without parsing it, but instead passing it directly to Hive. + * This is currently only used for DDLs and will be removed as soon as Spark can parse + * all supported Hive DDLs itself. + */ + protected[sql] override def runNativeSql(sqlText: String): Seq[Row] = { + runSqlHive(sqlText).map { s => Row(s) } + } + /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends org.apache.spark.sql.execution.QueryExecution(this, logicalPlan) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index a053108b7d7f5..d8620a89fe685 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -248,7 +248,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { - val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get + val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames, _) = bucketSpec.get tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) tableProperties.put("spark.sql.sources.schema.numBucketCols", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 0bdebdc5fd741..56acb87c800d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -88,29 +88,14 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", - "TOK_ALTERTABLE", - "TOK_ALTERTABLE_ADDCOLS", - "TOK_ALTERTABLE_ADDPARTS", "TOK_ALTERTABLE_ALTERPARTS", - "TOK_ALTERTABLE_ARCHIVE", - "TOK_ALTERTABLE_CLUSTER_SORT", - "TOK_ALTERTABLE_DROPPARTS", "TOK_ALTERTABLE_PARTITION", - "TOK_ALTERTABLE_PROPERTIES", - "TOK_ALTERTABLE_RENAME", - "TOK_ALTERTABLE_RENAMECOL", - "TOK_ALTERTABLE_REPLACECOLS", - "TOK_ALTERTABLE_SKEWED", - "TOK_ALTERTABLE_TOUCH", - "TOK_ALTERTABLE_UNARCHIVE", "TOK_ALTERVIEW_ADDPARTS", "TOK_ALTERVIEW_AS", "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEMACRO", "TOK_CREATEROLE", @@ -164,7 +149,8 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging protected val noExplainCommands = Seq( "TOK_DESCTABLE", "TOK_SHOWTABLES", - "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. + "TOK_TRUNCATETABLE", // truncate table" is a NativeCommand, does not need to explain. + "TOK_ALTERTABLE" ) ++ nativeCommands /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 9a52276fcdc6a..d6166358607e4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -78,7 +78,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // This test verifies parts of the plan. Disable whole stage codegen. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k") - val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec + val BucketSpec(numBuckets, bucketColumnNames, _, _) = bucketSpec // Limit: bucket pruning only works when the bucket column has one and only one column assert(bucketColumnNames.length == 1) val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head) From 007907460d72a3aa82b222567c08589c62efb614 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Mar 2016 20:43:31 -0800 Subject: [PATCH 03/12] Address comments from #11408 + fix style --- .../apache/spark/sql/execution/SparkQl.scala | 43 +- .../command/AlterTableCommandParser.scala | 677 +++++++++--------- .../spark/sql/execution/command/ddl.scala | 96 +-- .../{DDLSuite.scala => DDLCommandSuite.scala} | 193 ++--- 4 files changed, 473 insertions(+), 536 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/command/{DDLSuite.scala => DDLCommandSuite.scala} (92%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 7d782a4d8fb6d..6a456b67340f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -29,20 +29,27 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly import ParserUtils._ /** Check if a command should not be explained. */ - protected def isNoExplainCommand(command: String): Boolean = + protected def isNoExplainCommand(command: String): Boolean = { "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command + } protected def extractProps( node: ASTNode, - firstLevelNodeStr: String, - secondLevelNodeStr: String): Seq[(String, String)] = node match { - case Token(firstLevelNodeStr, options) => - options.map { - case Token(secondLevelNodeStr, keysAndValue) => - val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - } + firstLevelProp: String, + secondLevelProp: String): Seq[(String, String)] = { + node match { + case Token(x, options) if x == firstLevelProp => + options.map { + case Token(y, keysAndValue) if y == secondLevelProp => + val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + case _ => + throw new AnalysisException(s"Expected property '$secondLevelProp' in '${node.text}'") + } + case _ => + throw new AnalysisException(s"Expected property '$firstLevelProp' in '${node.text}'") + } } protected override def nodeToPlan(node: ASTNode): LogicalPlan = { @@ -100,7 +107,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly propList.flatMap(extractProps(_, "TOK_DBPROPLIST", "TOK_TABLEPROPERTY")) }.toMap - CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) + CreateDatabase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) => val funcName = func.map(x => unquoteString(x.text)).mkString(".") @@ -137,12 +144,12 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly Some(Token("TOK_TABLEPROVIDER", providerNameParts)), tableOpts, tableAs) = getClauses(Seq( - "TEMPORARY", - "TOK_IFNOTEXISTS", - "TOK_TABNAME", "TOK_TABCOLLIST", - "TOK_TABLEPROVIDER", - "TOK_TABLEOPTIONS", - "TOK_QUERY"), createTableArgs) + "TEMPORARY", + "TOK_IFNOTEXISTS", + "TOK_TABNAME", "TOK_TABCOLLIST", + "TOK_TABLEPROVIDER", + "TOK_TABLEOPTIONS", + "TOK_QUERY"), createTableArgs) val tableIdent: TableIdentifier = extractTableIdent(tabName) @@ -156,7 +163,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly val options: Map[String, String] = tableOpts.toSeq.flatMap(extractProps(_, "TOK_TABLEOPTIONS", "TOK_TABLEOPTION")).toMap - val asClause = tableAs.map(nodeToPlan(_)) + val asClause = tableAs.map(nodeToPlan) if (temp.isDefined && allowExisting.isDefined) { throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index b99753b4ddabd..58ff8de6c52d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -27,9 +27,28 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType + +/** + * Helper object to parse alter table commands. + */ object AlterTableCommandParser { import ParserUtils._ + /** + * Parse the given node assuming it is an alter table command. + */ + def parse(v1: ASTNode): LogicalPlan = { + v1.children match { + case (tabName @ Token("TOK_TABNAME", _)) :: restNodes => + val tableIdent: TableIdentifier = extractTableIdent(tabName) + val partitionSpec = getClauseOption("TOK_PARTSPEC", v1.children) + val partition = partitionSpec.flatMap(parsePartitionSpec) + matchAlterTableCommands(v1, restNodes, tableIdent, partition) + case _ => + throw new AnalysisException(s"Could not parse alter table command: '${v1.text}'") + } + } + private def cleanAndUnquoteString(s: String): String = { cleanIdentifier(unquoteString(s)) } @@ -48,348 +67,344 @@ object AlterTableCommandParser { } } - private def extractTableProps(node: ASTNode): Map[String, Option[String]] = node match { - case Token("TOK_TABLEPROPERTIES", propsList) => - propsList.flatMap { - case Token("TOK_TABLEPROPLIST", props) => - props.map { - case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) => - val k = cleanAndUnquoteString(key.text) - (k, None) - case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => - val k = cleanAndUnquoteString(key.text) - val v = cleanAndUnquoteString(value.text) - (k, Some(v)) - } - }.toMap - } - - def parse(v1: ASTNode): LogicalPlan = v1.children match { - case (tabName @ Token("TOK_TABNAME", _)) :: restNodes => - val tableIdent: TableIdentifier = extractTableIdent(tabName) - val partitionSpec = getClauseOption("TOK_PARTSPEC", v1.children) - val partition = partitionSpec.flatMap(parsePartitionSpec) - matchAlterTableCommands(v1, restNodes, tableIdent, partition) - case _ => - throw new NotImplementedError(v1.text) + private def extractTableProps(node: ASTNode): Map[String, Option[String]] = { + node match { + case Token("TOK_TABLEPROPERTIES", propsList) => + propsList.flatMap { + case Token("TOK_TABLEPROPLIST", props) => + props.map { + case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) => + val k = cleanAndUnquoteString(key.text) + (k, None) + case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = cleanAndUnquoteString(key.text) + val v = cleanAndUnquoteString(value.text) + (k, Some(v)) + } + }.toMap + case _ => + throw new AnalysisException( + s"Expected table properties in alter table command: '${node.text}'") + } } + // TODO: This method is massive. Break it down. Also, add some comments... private def matchAlterTableCommands( node: ASTNode, nodes: Seq[ASTNode], tableIdent: TableIdentifier, - partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match { - case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => - val renamedTable = getClause("TOK_TABNAME", renameArgs) - val renamedTableIdent: TableIdentifier = extractTableIdent(renamedTable) - AlterTableRename(tableIdent, renamedTableIdent)(node.source) - - case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => - val setTableProperties = extractTableProps(args.head) - AlterTableSetProperties( - tableIdent, - setTableProperties)(node.source) - - case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => - val dropTableProperties = extractTableProps(args.head) - val allowExisting = getClauseOption("TOK_IFEXISTS", args) - AlterTableDropProperties( - tableIdent, - dropTableProperties, allowExisting.isDefined)(node.source) - - case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ => - // When SET SERDE serde_classname WITH SERDEPROPERTIES, this is None - val serdeProperties: Option[Map[String, Option[String]]] = - serdeArgs.headOption.map(extractTableProps) - - AlterTableSerDeProperties( - tableIdent, - Some(cleanAndUnquoteString(serdeClassName)), - serdeProperties, - partition)(node.source) - - case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => - val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) - - AlterTableSerDeProperties( - tableIdent, - None, - Some(serdeProperties), - partition)(node.source) - - case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) :: _ => - val (buckets, noClustered, noSorted) = clusterAndSoryByArgs match { - case Token("TOK_ALTERTABLE_BUCKETS", bucketArgsHead :: bucketArgs) => - val bucketCols = bucketArgsHead.children.map(_.text) - - val (sortCols, sortDirections, numBuckets) = { - if (bucketArgs.head.text == "TOK_TABCOLNAME") { - val (cols, directions) = bucketArgs.head.children.map { - case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => - (colName, Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => - (colName, Descending) - }.unzip - (cols, directions, bucketArgs.last.text.toInt) - } else { - (Nil, Nil, bucketArgs.head.text.toInt) - } - } - - (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)), - false, false) - case Token("TOK_NOT_CLUSTERED", Nil) => - (None, true, false) - case Token("TOK_NOT_SORTED", Nil) => - (None, false, true) - } - - AlterTableStoreProperties( - tableIdent, - buckets, - noClustered, - noSorted)(node.source) - - case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => - val num = bucketNum.toInt - val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) - AlterTableStoreProperties( - tableIdent, - buckets, - false, - false)(node.source) - - case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => - // ALTER TABLE table_name NOT SKEWED - AlterTableSkewed(tableIdent, Nil, Nil, false, true, false)(node.source) - - case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => - // ALTER TABLE table_name NOT STORED AS DIRECTORIES - AlterTableSkewed(tableIdent, Nil, Nil, false, false, true)(node.source) - - case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: _ => - val skewedArgs = tableSkewed match { - case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) => - skewedArgs match { - case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => - val cols = skewedCols.children.map(n => cleanAndUnquoteString(n.text)) - val values = skewedValues match { - case Token("TOK_TABCOLVALUE", values) => - Seq(values.map(n => cleanAndUnquoteString(n.text))) - case Token("TOK_TABCOLVALUE_PAIR", pairs) => - pairs.map { - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => - vals.map(n => cleanAndUnquoteString(n.text)) - } - } - } - - val storedAsDirs = stored match { - case Token("TOK_STOREDASDIRS", Nil) :: Nil => true - case _ => false + partition: Option[Map[String, Option[String]]]): LogicalPlan = { + nodes match { + case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => + val renamedTable = getClause("TOK_TABNAME", renameArgs) + val renamedTableIdent: TableIdentifier = extractTableIdent(renamedTable) + AlterTableRename(tableIdent, renamedTableIdent)(node.source) + + case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => + val setTableProperties = extractTableProps(args.head) + AlterTableSetProperties( + tableIdent, + setTableProperties)(node.source) + + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => + val dropTableProperties = extractTableProps(args.head) + val allowExisting = getClauseOption("TOK_IFEXISTS", args) + AlterTableDropProperties( + tableIdent, + dropTableProperties, allowExisting.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ => + // When SET SERDE serde_classname WITH SERDEPROPERTIES, this is None + val serdeProperties: Option[Map[String, Option[String]]] = + serdeArgs.headOption.map(extractTableProps) + + AlterTableSerDeProperties( + tableIdent, + Some(cleanAndUnquoteString(serdeClassName)), + serdeProperties, + partition)(node.source) + + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => + val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) + + AlterTableSerDeProperties( + tableIdent, + None, + Some(serdeProperties), + partition)(node.source) + + case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) :: _ => + val (buckets, noClustered, noSorted) = clusterAndSoryByArgs match { + case Token("TOK_ALTERTABLE_BUCKETS", bucketArgsHead :: bucketArgs) => + val bucketCols = bucketArgsHead.children.map(_.text) + val (sortCols, sortDirections, numBuckets) = { + if (bucketArgs.head.text == "TOK_TABCOLNAME") { + val (cols, directions) = bucketArgs.head.children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => + (colName, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => + (colName, Descending) + }.unzip + (cols, directions, bucketArgs.last.text.toInt) + } else { + (Nil, Nil, bucketArgs.head.text.toInt) } + } + val bucketSpec = BucketSpec(numBuckets, bucketCols, sortCols, sortDirections) + (Some(bucketSpec), false, false) + case Token("TOK_NOT_CLUSTERED", Nil) => + (None, true, false) + case Token("TOK_NOT_SORTED", Nil) => + (None, false, true) + } + AlterTableStoreProperties( + tableIdent, + buckets, + noClustered, + noSorted)(node.source) + + case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => + val num = bucketNum.toInt + val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) + AlterTableStoreProperties( + tableIdent, + buckets, + noClustered = false, + noSorted = false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => + // ALTER TABLE table_name NOT SKEWED + AlterTableSkewed( + tableIdent, + Nil, + Nil, + storedAsDirs = false, + notSkewed = true, + notStoredAsDirs = false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => + // ALTER TABLE table_name NOT STORED AS DIRECTORIES + AlterTableSkewed( + tableIdent, + Nil, + Nil, + storedAsDirs = false, + notSkewed = false, + notStoredAsDirs = true)(node.source) + + case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: _ => + val skewedArgs = tableSkewed match { + case Token("TOK_ALTERTABLE_SKEWED", args :: Nil) => + args match { + case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => + val cols = skewedCols.children.map(n => cleanAndUnquoteString(n.text)) + val values = skewedValues match { + case Token("TOK_TABCOLVALUE", colVal) => + Seq(colVal.map(n => cleanAndUnquoteString(n.text))) + case Token("TOK_TABCOLVALUE_PAIR", pairs) => + pairs.map { + case Token("TOK_TABCOLVALUES", colVals :: Nil) => + colVals match { + case Token("TOK_TABCOLVALUE", vals) => + vals.map(n => cleanAndUnquoteString(n.text)) + } + } + } + + val storedAsDirs = stored match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false + } + + (cols, values, storedAsDirs) + } + } + val (cols, values, storedAsDirs) = skewedArgs + AlterTableSkewed( + tableIdent, + cols, + values, + storedAsDirs, + notSkewed = false, + notStoredAsDirs = false)(node.source) + + case Token("TOK_ALTERTABLE_SKEWED_LOCATION", + Token("TOK_SKEWED_LOCATIONS", + Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => + val skewedMaps = locationMaps.map { + case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => + val k = key match { + case Token(const, Nil) => Seq(cleanAndUnquoteString(const)) + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => + vals.map(n => cleanAndUnquoteString(n.text)) + } + } + (k, cleanAndUnquoteString(value.text)) + }.toMap + AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) - (cols, values, storedAsDirs) - } - } - - val (cols, values, storedAsDirs) = skewedArgs - - AlterTableSkewed( - tableIdent, - cols, - values, - storedAsDirs, - notSkewed = false, - notStoredAsDirs = false)(node.source) - - case Token("TOK_ALTERTABLE_SKEWED_LOCATION", - Token("TOK_SKEWED_LOCATIONS", - Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => - val skewedMaps = locationMaps.map { - case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => - val k = key match { - case Token(const, Nil) => Seq(cleanAndUnquoteString(const)) - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => - vals.map(n => cleanAndUnquoteString(n.text)) - } - } - (k, cleanAndUnquoteString(value.text)) - }.toMap - AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) - - case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: _ => - val (allowExisting, parts) = addPartsArgs match { - case Token("TOK_IFNOTEXISTS", Nil) :: others => (true, others) - case _ => (false, addPartsArgs) - } - - val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] = - new ArrayBuffer() - var currentPart: Map[String, Option[String]] = null - parts.map { - case t @ Token("TOK_PARTSPEC", partArgs) => - if (currentPart != null) { - partitions += ((currentPart, None)) - } - currentPart = parsePartitionSpec(t).get - case Token("TOK_PARTITIONLOCATION", loc :: Nil) => - val location = unquoteString(loc.text) - if (currentPart != null) { - partitions += ((currentPart, Some(location))) - currentPart = null - } else { - // We should not reach here - throw new AnalysisException("Partition location must follow a partition spec.") - } - } - - if (currentPart != null) { - partitions += ((currentPart, None)) - } - AlterTableAddPartition(tableIdent, partitions, allowExisting)(node.source) - - case Token("TOK_ALTERTABLE_RENAMEPART", partArg :: Nil) :: _ => - val Some(newPartition) = parsePartitionSpec(partArg) - AlterTableRenamePartition(tableIdent, partition.get, newPartition)(node.source) - - case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", - (p @ Token("TOK_PARTSPEC", _)) :: (t @ Token("TOK_TABNAME", _)) :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(p) - val fromTableIdent = extractTableIdent(t) - AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) - - case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ => - val parts = args.collect { - case Token("TOK_PARTSPEC", partitions) => - partitions.map { - case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => - (cleanAndUnquoteString(ident.text), - op.text, cleanAndUnquoteString(constant.text)) + case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: _ => + val (allowExisting, parts) = addPartsArgs match { + case Token("TOK_IFNOTEXISTS", Nil) :: others => (true, others) + case _ => (false, addPartsArgs) + } + val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] = + new ArrayBuffer() + var currentPart: Map[String, Option[String]] = null + parts.map { + case t @ Token("TOK_PARTSPEC", partArgs) => + if (currentPart != null) { + partitions += ((currentPart, None)) + } + currentPart = parsePartitionSpec(t).get + case Token("TOK_PARTITIONLOCATION", loc :: Nil) => + val location = unquoteString(loc.text) + if (currentPart != null) { + partitions += ((currentPart, Some(location))) + currentPart = null + } else { + // We should not reach here + throw new AnalysisException("Partition location must follow a partition spec.") + } + } + if (currentPart != null) { + partitions += ((currentPart, None)) + } + AlterTableAddPartition(tableIdent, partitions, allowExisting)(node.source) + + case Token("TOK_ALTERTABLE_RENAMEPART", partArg :: Nil) :: _ => + val Some(newPartition) = parsePartitionSpec(partArg) + AlterTableRenamePartition(tableIdent, partition.get, newPartition)(node.source) + + case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", + (p @ Token("TOK_PARTSPEC", _)) :: (t @ Token("TOK_TABNAME", _)) :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(p) + val fromTableIdent = extractTableIdent(t) + AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ => + val parts = args.collect { + case Token("TOK_PARTSPEC", partitions) => + partitions.map { + case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), + op.text, cleanAndUnquoteString(constant.text)) + } + } + val allowExisting = getClauseOption("TOK_IFEXISTS", args).isDefined + val purge = getClauseOption("PURGE", args) + val replication = getClauseOption("TOK_REPLICATION", args).map { + case Token("TOK_REPLICATION", replId :: metadata) => + (cleanAndUnquoteString(replId.text), metadata.nonEmpty) + } + AlterTableDropPartition( + tableIdent, + parts, + allowExisting, + purge.isDefined, + replication)(node.source) + + case Token("TOK_ALTERTABLE_ARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) + AlterTableArchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_UNARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) + AlterTableUnarchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => + val Seq(fileFormat, genericFormat) = + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), + args) + val fFormat = fileFormat.map(_.children.map(n => cleanAndUnquoteString(n.text))) + val gFormat = genericFormat.map(f => cleanAndUnquoteString(f.children(0).text)) + AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + + case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => + AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) + + case Token("TOK_ALTERTABLE_TOUCH", args) :: _ => + val part = getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) + AlterTableTouch(tableIdent, part)(node.source) + + case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => + AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) + + case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => + AlterTableMerge(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_RENAMECOL", args) :: _ => + val oldName = args(0).text + val newName = args(1).text + val dataType = nodeToDataType(args(2)) + val afterPos = + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) + val afterPosCol = afterPos.map { ap => + ap.children match { + case Token(col, Nil) :: Nil => col + case _ => null } - } - - val allowExisting = getClauseOption("TOK_IFEXISTS", args).isDefined - - val purge = getClauseOption("PURGE", args) - - val replication = getClauseOption("TOK_REPLICATION", args).map { - case Token("TOK_REPLICATION", replId :: metadata) => - (cleanAndUnquoteString(replId.text), metadata.nonEmpty) - } - - AlterTableDropPartition( - tableIdent, - parts, - allowExisting, - purge.isDefined, - replication)(node.source) - - case Token("TOK_ALTERTABLE_ARCHIVE", partArg :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(partArg) - AlterTableArchivePartition(tableIdent, partition)(node.source) - - case Token("TOK_ALTERTABLE_UNARCHIVE", partArg :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(partArg) - AlterTableUnarchivePartition(tableIdent, partition)(node.source) - - case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => - val Seq(fileFormat, genericFormat) = - getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), - args) - val fFormat = fileFormat.map(_.children.map(n => cleanAndUnquoteString(n.text))) - val gFormat = genericFormat.map(f => cleanAndUnquoteString(f.children(0).text)) - AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) - - case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => - AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) - - case Token("TOK_ALTERTABLE_TOUCH", args) :: _ => - val part = getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) - AlterTableTouch(tableIdent, part)(node.source) - - case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => - AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) - - case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => - AlterTableMerge(tableIdent, partition)(node.source) - - case Token("TOK_ALTERTABLE_RENAMECOL", args) :: _ => - val oldName = args(0).text - val newName = args(1).text - val dataType = nodeToDataType(args(2)) - val afterPos = - getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) - val afterPosCol = afterPos.map { ap => - ap.children match { - case Token(col, Nil) :: Nil => col - case _ => null } - } - - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - - val comment = if (args.size > 3) { - args(3) match { - case Token(commentStr, Nil) - if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && - commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + val comment = if (args.size > 3) { + args(3) match { + case Token(commentStr, Nil) + if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && + commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => Some(cleanAndUnquoteString(commentStr)) - case _ => - None + case _ => + None + } + } else { + None } - } else { - None - } - - AlterTableChangeCol( - tableIdent, - partition, - oldName, - newName, - dataType, - comment, - afterPos.isDefined, - afterPosCol, - restrict.isDefined, - cascade.isDefined)(node.source) - - case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => - val tableCols = getClause("TOK_TABCOLLIST", args) - val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) - } - - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - - AlterTableAddCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) - - case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => - val tableCols = getClause("TOK_TABCOLLIST", args) - val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) - } - - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - - AlterTableReplaceCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) + AlterTableChangeCol( + tableIdent, + partition, + oldName, + newName, + dataType, + comment, + afterPos.isDefined, + afterPosCol, + restrict.isDefined, + cascade.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + } + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + AlterTableAddCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + } + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) + AlterTableReplaceCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + + case _ => + throw new AnalysisException( + s"Unexpected children nodes in alter table command: '${node.text}'") + } } -} +} 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 ec31c7f058cfb..4ccb7c340ed5c 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 @@ -30,54 +30,61 @@ import org.apache.spark.sql.types._ * query text. */ abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { + override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.runNativeSql(sql) } - override val output: Seq[Attribute] = + override val output: Seq[Attribute] = { Seq(AttributeReference("result", StringType, nullable = false)()) + } + } -case class CreateDataBase( +case class CreateDatabase( databaseName: String, allowExisting: Boolean, path: Option[String], comment: Option[String], - props: Map[String, String])(sql: String) extends NativeDDLCommands(sql) with Logging + props: Map[String, String])(sql: String) + extends NativeDDLCommands(sql) with Logging case class CreateFunction( functionName: String, - asName: String, + alias: String, resourcesMap: Map[String, String], - isTemp: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + isTemp: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableRename( tableName: TableIdentifier, - renameTableName: TableIdentifier)(sql: String) extends NativeDDLCommands(sql) - with Logging + renameTableName: TableIdentifier)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSetProperties( tableName: TableIdentifier, - setProperties: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) - with Logging + setProperties: Map[String, Option[String]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableDropProperties( tableName: TableIdentifier, dropProperties: Map[String, Option[String]], - allowExisting: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + allowExisting: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSerDeProperties( tableName: TableIdentifier, serdeClassName: Option[String], serdeProperties: Option[Map[String, Option[String]]], - partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Option[Map[String, Option[String]]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableStoreProperties( tableName: TableIdentifier, buckets: Option[BucketSpec], noClustered: Boolean, - noSorted: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + noSorted: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSkewed( tableName: TableIdentifier, @@ -85,76 +92,79 @@ case class AlterTableSkewed( skewedValues: Seq[Seq[String]], storedAsDirs: Boolean, notSkewed: Boolean, - // TODO: wtf? - notStoredAsDirs: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + // TODO: what?? + notStoredAsDirs: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSkewedLocation( tableName: TableIdentifier, - skewedMap: Map[Seq[String], String])(sql: String) extends NativeDDLCommands(sql) with Logging + skewedMap: Map[Seq[String], String])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableAddPartition( tableName: TableIdentifier, partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], - allowExisting: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging + allowExisting: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableRenamePartition( tableName: TableIdentifier, oldPartition: Map[String, Option[String]], - newPartition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) - with Logging + newPartition: Map[String, Option[String]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableExchangePartition( tableName: TableIdentifier, fromTableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Map[String, Option[String]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableDropPartition( tableName: TableIdentifier, partitions: Seq[Seq[(String, String, String)]], allowExisting: Boolean, purge: Boolean, - replication: Option[(String, Boolean)])(sql: String) extends NativeDDLCommands(sql) - with Logging + replication: Option[(String, Boolean)])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableArchivePartition( tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Map[String, Option[String]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableUnarchivePartition( tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Map[String, Option[String]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSetFileFormat( tableName: TableIdentifier, partition: Option[Map[String, Option[String]]], fileFormat: Option[Seq[String]], - genericFormat: Option[String])(sql: String) extends NativeDDLCommands(sql) - with Logging + genericFormat: Option[String])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableSetLocation( tableName: TableIdentifier, partition: Option[Map[String, Option[String]]], - location: String)(sql: String) extends NativeDDLCommands(sql) - with Logging + location: String)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableTouch( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Option[Map[String, Option[String]]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableCompact( tableName: TableIdentifier, partition: Option[Map[String, Option[String]]], - compactType: String)(sql: String) extends NativeDDLCommands(sql) - with Logging + compactType: String)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableMerge( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) - with Logging + partition: Option[Map[String, Option[String]]])(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableChangeCol( tableName: TableIdentifier, @@ -166,21 +176,21 @@ case class AlterTableChangeCol( afterPos: Boolean, afterPosCol: Option[String], restrict: Boolean, - cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) - with Logging + cascade: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableAddCol( tableName: TableIdentifier, partition: Option[Map[String, Option[String]]], columns: StructType, restrict: Boolean, - cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) - with Logging + cascade: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging case class AlterTableReplaceCol( tableName: TableIdentifier, partition: Option[Map[String, Option[String]]], columns: StructType, restrict: Boolean, - cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) - with Logging + cascade: Boolean)(sql: String) + extends NativeDDLCommands(sql) with Logging 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/DDLCommandSuite.scala similarity index 92% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 049060d88f191..27ff583ca5de3 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/DDLCommandSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.execution.SparkQl import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ -class DDLSuite extends PlanTest { - val parser = new SparkQl() +class DDLCommandSuite extends PlanTest { + private val parser = new SparkQl test("create database") { val sql = @@ -34,16 +34,13 @@ class DDLSuite extends PlanTest { |COMMENT 'database_comment' LOCATION '/home/user/db' |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') """.stripMargin - val parsed = parser.parsePlan(sql) - - val expected = CreateDataBase( + val expected = CreateDatabase( "database_name", - true, + allowExisting = true, Some("/home/user/db"), Some("database_comment"), Map("a" -> "a", "b" -> "b", "c" -> "c"))(sql) - comparePlans(parsed, expected) } @@ -54,15 +51,12 @@ class DDLSuite extends PlanTest { |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', |FILE 'path/to/file' """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateFunction( "helloworld", "com.matthewrathbone.example.SimpleUDFExample", Map("jar" -> "/path/to/jar", "file" -> "path/to/file"), - true)(sql) - + isTemp = true)(sql) comparePlans(parsed, expected) } @@ -80,25 +74,20 @@ class DDLSuite extends PlanTest { "'comment' = 'new_comment')" val sql2 = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" val sql3 = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) - val expected1 = AlterTableSetProperties( TableIdentifier("table_name", None), Map("test" -> Some("test"), "comment" -> Some("new_comment")))(sql1) - val expected2 = AlterTableDropProperties( TableIdentifier("table_name", None), Map("comment" -> None, "test" -> None), - false)(sql2) - + allowExisting = false)(sql2) val expected3 = AlterTableDropProperties( TableIdentifier("table_name", None), Map("comment" -> None, "test" -> None), - true)(sql3) - + allowExisting = true)(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -111,62 +100,52 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name SET SERDE 'org.apache.class' |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin - val sql3 = """ |ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', |'field.delim' = ',') """.stripMargin - val sql4 = """ |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', |country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', |'field.delim' = ',') """.stripMargin - val sql5 = """ |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', |country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) - val expected1 = AlterTableSerDeProperties( TableIdentifier("table_name", None), Some("org.apache.class"), None, None)(sql1) - val expected2 = AlterTableSerDeProperties( TableIdentifier("table_name", None), Some("org.apache.class"), Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), None)(sql2) - val expected3 = AlterTableSerDeProperties( TableIdentifier("table_name", None), None, Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), None)(sql3) - val expected4 = AlterTableSerDeProperties( TableIdentifier("table_name", None), Some("org.apache.class"), Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql4) - val expected5 = AlterTableSerDeProperties( TableIdentifier("table_name", None), None, Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql5) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -176,51 +155,42 @@ class DDLSuite extends PlanTest { test("alter table: storage properties") { val sql1 = "ALTER TABLE table_name CLUSTERED BY (dt, country) INTO 10 BUCKETS" - val sql2 = "ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + "(dt, country DESC) INTO 10 BUCKETS" - val sql3 = "ALTER TABLE table_name INTO 20 BUCKETS" val sql4 = "ALTER TABLE table_name NOT CLUSTERED" val sql5 = "ALTER TABLE table_name NOT SORTED" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) - val expected1 = AlterTableStoreProperties( TableIdentifier("table_name", None), Some(BucketSpec(10, List("dt", "country"), List(), List())), - false, - false)(sql1) - + noClustered = false, + noSorted = false)(sql1) val expected2 = AlterTableStoreProperties( TableIdentifier("table_name", None), Some(BucketSpec(10, List("dt", "country"), List("dt", "country"), List(Ascending, Descending))), - false, - false)(sql2) - + noClustered = false, + noSorted = false)(sql2) val expected3 = AlterTableStoreProperties( TableIdentifier("table_name", None), Some(BucketSpec(20, List(), List(), List())), - false, - false)(sql3) - + noClustered = false, + noSorted = false)(sql3) val expected4 = AlterTableStoreProperties( TableIdentifier("table_name", None), None, - true, - false)(sql4) - + noClustered = true, + noSorted = false)(sql4) val expected5 = AlterTableStoreProperties( TableIdentifier("table_name", None), None, - false, - true)(sql5) - + noClustered = false, + noSorted = true)(sql5) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -234,47 +204,40 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name SKEWED BY (dt, country) ON |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES """.stripMargin - val sql2 = """ |ALTER TABLE table_name SKEWED BY (dt, country) ON |('2008-08-08', 'us') STORED AS DIRECTORIES """.stripMargin - val sql3 = """ |ALTER TABLE table_name SKEWED BY (dt, country) ON |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) - val expected1 = AlterTableSkewed( TableIdentifier("table_name", None), Seq("dt", "country"), Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), - true, - false, - false)(sql1) - + storedAsDirs = true, + notSkewed = false, + notStoredAsDirs = false)(sql1) val expected2 = AlterTableSkewed( TableIdentifier("table_name", None), Seq("dt", "country"), Seq(List("2008-08-08", "us")), - true, - false, - false)(sql2) - + storedAsDirs = true, + notSkewed = false, + notStoredAsDirs = false)(sql2) val expected3 = AlterTableSkewed( TableIdentifier("table_name", None), Seq("dt", "country"), Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), - false, - false, - false)(sql3) - + storedAsDirs = false, + notSkewed = false, + notStoredAsDirs = false)(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -286,24 +249,19 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name SET SKEWED LOCATION |('123'='location1', 'test'='location2') """.stripMargin - val sql2 = """ |ALTER TABLE table_name SET SKEWED LOCATION |(('2008-08-08', 'us')='location1', 'test'='location2') """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableSkewedLocation( TableIdentifier("table_name", None), Map(List("123") -> "location1", List("test") -> "location2"))(sql1) - val expected2 = AlterTableSkewedLocation( TableIdentifier("table_name", None), Map(List("2008-08-08", "us") -> "location1", List("test") -> "location2"))(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -315,16 +273,13 @@ class DDLSuite extends PlanTest { |(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION |(dt='2009-09-09', country='uk') """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = AlterTableAddPartition( TableIdentifier("table_name", None), Seq( (Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), Some("location1")), (Map("dt" -> Some("2009-09-09"), "country" -> Some("uk")), None)), - true)(sql) - + allowExisting = true)(sql) comparePlans(parsed, expected) } @@ -334,14 +289,11 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') |RENAME TO PARTITION (dt='2008-09-09', country='uk') """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = AlterTableRenamePartition( TableIdentifier("table_name", None), Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), Map("dt" -> Some("2008-09-09"), "country" -> Some("uk")))(sql) - comparePlans(parsed, expected) } @@ -351,14 +303,11 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name_1 EXCHANGE PARTITION |(dt='2008-08-08', country='us') WITH TABLE table_name_2 """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = AlterTableExchangePartition( TableIdentifier("table_name_1", None), TableIdentifier("table_name_2", None), Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) - comparePlans(parsed, expected) } @@ -368,35 +317,30 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name DROP IF EXISTS PARTITION |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') """.stripMargin - val sql2 = """ |ALTER TABLE table_name DROP IF EXISTS PARTITION |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') |PURGE FOR METADATA REPLICATION ('test') """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableDropPartition( TableIdentifier("table_name", None), Seq( List(("dt", "=", "2008-08-08"), ("country", "=", "us")), List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), - true, - false, + allowExisting = true, + purge = false, None)(sql1) - val expected2 = AlterTableDropPartition( TableIdentifier("table_name", None), Seq( List(("dt", "=", "2008-08-08"), ("country", "=", "us")), List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), - true, - true, + allowExisting = true, + purge = true, Some(("test", true)))(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -404,22 +348,18 @@ class DDLSuite extends PlanTest { test("alter table: archive partition") { val sql = "ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')" val parsed = parser.parsePlan(sql) - val expected = AlterTableArchivePartition( TableIdentifier("table_name", None), Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) - comparePlans(parsed, expected) } test("alter table: unarchive partition") { val sql = "ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')" val parsed = parser.parsePlan(sql) - val expected = AlterTableUnarchivePartition( TableIdentifier("table_name", None), Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) - comparePlans(parsed, expected) } @@ -429,35 +369,28 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' |OUTPUTFORMAT 'test' SERDE 'test' INPUTDRIVER 'test' OUTPUTDRIVER 'test' """.stripMargin - val sql2 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + "OUTPUTFORMAT 'test' SERDE 'test'" - val sql3 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + "SET FILEFORMAT PARQUET" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) - val expected1 = AlterTableSetFileFormat( TableIdentifier("table_name", None), None, Some(List("test", "test", "test", "test", "test")), None)(sql1) - val expected2 = AlterTableSetFileFormat( TableIdentifier("table_name", None), None, Some(List("test", "test", "test")), None)(sql2) - val expected3 = AlterTableSetFileFormat( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), None, Some("PARQUET"))(sql3) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -467,20 +400,16 @@ class DDLSuite extends PlanTest { val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + "SET LOCATION 'new location'" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableSetLocation( TableIdentifier("table_name", None), None, "new location")(sql1) - val expected2 = AlterTableSetLocation( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), "new location")(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -488,18 +417,14 @@ class DDLSuite extends PlanTest { test("alter table: touch") { val sql1 = "ALTER TABLE table_name TOUCH" val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableTouch( TableIdentifier("table_name", None), None)(sql1) - val expected2 = AlterTableTouch( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -511,20 +436,16 @@ class DDLSuite extends PlanTest { |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') |COMPACT 'MAJOR' """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableCompact( TableIdentifier("table_name", None), None, "compaction_type")(sql1) - val expected2 = AlterTableCompact( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), "MAJOR")(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -532,41 +453,33 @@ class DDLSuite extends PlanTest { test("alter table: concatenate") { val sql1 = "ALTER TABLE table_name CONCATENATE" val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableMerge( TableIdentifier("table_name", None), None)(sql1) - val expected2 = AlterTableMerge( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } test("alter table: change column name/type/position/comment") { val sql1 = "ALTER TABLE table_name CHANGE col_old_name col_new_name INT" - val sql2 = """ |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT |COMMENT 'col_comment' FIRST CASCADE """.stripMargin - val sql3 = """ |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT |COMMENT 'col_comment' AFTER column_name RESTRICT """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) - val expected1 = AlterTableChangeCol( TableIdentifier("table_name", None), None, @@ -574,11 +487,10 @@ class DDLSuite extends PlanTest { "col_new_name", IntegerType, None, - false, + afterPos = false, None, - false, - false)(sql1) - + restrict = false, + cascade = false)(sql1) val expected2 = AlterTableChangeCol( TableIdentifier("table_name", None), None, @@ -586,11 +498,10 @@ class DDLSuite extends PlanTest { "col_new_name", IntegerType, Some("col_comment"), - false, + afterPos = false, None, - false, - true)(sql2) - + restrict = false, + cascade = true)(sql2) val expected3 = AlterTableChangeCol( TableIdentifier("table_name", None), None, @@ -598,11 +509,10 @@ class DDLSuite extends PlanTest { "col_new_name", IntegerType, Some("col_comment"), - true, + afterPos = true, Some("column_name"), - true, - false)(sql3) - + restrict = true, + cascade = false)(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -615,38 +525,33 @@ class DDLSuite extends PlanTest { |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG |COMMENT 'test_comment2') CASCADE """.stripMargin - val sql2 = """ |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT """.stripMargin - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() - val expected1 = AlterTableAddCol( TableIdentifier("table_name", None), Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), StructType(Seq( - StructField("new_col1", IntegerType, true, meta1), - StructField("new_col2", LongType, true, meta2))), - false, - true)(sql1) - + StructField("new_col1", IntegerType, nullable = true, meta1), + StructField("new_col2", LongType, nullable = true, meta2))), + restrict = false, + cascade = true)(sql1) val expected2 = AlterTableReplaceCol( TableIdentifier("table_name", None), None, StructType(Seq( - StructField("new_col1", IntegerType, true, meta1), - StructField("new_col2", LongType, true, meta2))), - true, - false)(sql2) - + StructField("new_col1", IntegerType, nullable = true, meta1), + StructField("new_col2", LongType, nullable = true, meta2))), + restrict = true, + cascade = false)(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } + } From a663b5ccca7718b41df8b8ede462ac6c9e0b8e8f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Mar 2016 20:49:22 -0800 Subject: [PATCH 04/12] Minor fixes --- .../org/apache/spark/sql/execution/SparkQl.scala | 15 ++++++--------- .../apache/spark/sql/execution/command/ddl.scala | 1 + 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 6a456b67340f1..66087711bb9eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -95,7 +95,6 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "TOK_DATABASELOCATION", "TOK_DATABASECOMMENT", "TOK_DATABASEPROPERTIES"), createDatabaseArgs) - val location = dbLocation.map { case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc) } @@ -106,7 +105,6 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case Token("TOK_DATABASEPROPERTIES", propList) => propList.flatMap(extractProps(_, "TOK_DBPROPLIST", "TOK_TABLEPROPERTY")) }.toMap - CreateDatabase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) => @@ -117,7 +115,6 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly temp) = getClauses(Seq( "TOK_RESOURCE_LIST", "TOK_TEMPORARY"), createFuncArgs) - val resourcesMap: Map[String, String] = rList.toSeq.flatMap { case Token("TOK_RESOURCE_LIST", resources) => resources.map { @@ -144,12 +141,12 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly Some(Token("TOK_TABLEPROVIDER", providerNameParts)), tableOpts, tableAs) = getClauses(Seq( - "TEMPORARY", - "TOK_IFNOTEXISTS", - "TOK_TABNAME", "TOK_TABCOLLIST", - "TOK_TABLEPROVIDER", - "TOK_TABLEOPTIONS", - "TOK_QUERY"), createTableArgs) + "TEMPORARY", + "TOK_IFNOTEXISTS", + "TOK_TABNAME", "TOK_TABCOLLIST", + "TOK_TABLEPROVIDER", + "TOK_TABLEOPTIONS", + "TOK_QUERY"), createTableArgs) val tableIdent: TableIdentifier = extractTableIdent(tabName) 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 4ccb7c340ed5c..b0b73946d5bb3 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 @@ -82,6 +82,7 @@ case class AlterTableSerDeProperties( case class AlterTableStoreProperties( tableName: TableIdentifier, buckets: Option[BucketSpec], + // TODO: use `clustered` and `sorted` instead for simplicity noClustered: Boolean, noSorted: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging From 042222d393c118bcae7a68209875bff5035b3ed2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Mar 2016 13:28:06 -0800 Subject: [PATCH 05/12] Rename `notClustered` and `notSorted` --- .../command/AlterTableCommandParser.scala | 28 ++++++++-------- .../spark/sql/execution/command/ddl.scala | 5 ++- .../execution/command/DDLCommandSuite.scala | 33 ++++++++----------- 3 files changed, 29 insertions(+), 37 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 58ff8de6c52d4..6d53259e9e918 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -117,7 +117,6 @@ object AlterTableCommandParser { // When SET SERDE serde_classname WITH SERDEPROPERTIES, this is None val serdeProperties: Option[Map[String, Option[String]]] = serdeArgs.headOption.map(extractTableProps) - AlterTableSerDeProperties( tableIdent, Some(cleanAndUnquoteString(serdeClassName)), @@ -126,15 +125,14 @@ object AlterTableCommandParser { case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) - AlterTableSerDeProperties( tableIdent, None, Some(serdeProperties), partition)(node.source) - case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) :: _ => - val (buckets, noClustered, noSorted) = clusterAndSoryByArgs match { + case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterSortArgs :: Nil) :: _ => + clusterSortArgs match { case Token("TOK_ALTERTABLE_BUCKETS", bucketArgsHead :: bucketArgs) => val bucketCols = bucketArgsHead.children.map(_.text) val (sortCols, sortDirections, numBuckets) = { @@ -150,18 +148,18 @@ object AlterTableCommandParser { (Nil, Nil, bucketArgs.head.text.toInt) } } - val bucketSpec = BucketSpec(numBuckets, bucketCols, sortCols, sortDirections) - (Some(bucketSpec), false, false) + AlterTableStoreProperties( + tableIdent, + Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)), + clustered = true, + sorted = true)(node.source) case Token("TOK_NOT_CLUSTERED", Nil) => - (None, true, false) + AlterTableStoreProperties( + tableIdent, None, clustered = false, sorted = true)(node.source) case Token("TOK_NOT_SORTED", Nil) => - (None, false, true) + AlterTableStoreProperties( + tableIdent, None, clustered = true, sorted = false)(node.source) } - AlterTableStoreProperties( - tableIdent, - buckets, - noClustered, - noSorted)(node.source) case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => val num = bucketNum.toInt @@ -169,8 +167,8 @@ object AlterTableCommandParser { AlterTableStoreProperties( tableIdent, buckets, - noClustered = false, - noSorted = false)(node.source) + clustered = true, + sorted = true)(node.source) case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => // ALTER TABLE table_name NOT SKEWED 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 b0b73946d5bb3..bf05dd4074135 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 @@ -82,9 +82,8 @@ case class AlterTableSerDeProperties( case class AlterTableStoreProperties( tableName: TableIdentifier, buckets: Option[BucketSpec], - // TODO: use `clustered` and `sorted` instead for simplicity - noClustered: Boolean, - noSorted: Boolean)(sql: String) + clustered: Boolean, + sorted: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging case class AlterTableSkewed( 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 27ff583ca5de3..c786806aee300 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 @@ -165,32 +165,27 @@ class DDLCommandSuite extends PlanTest { val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableStoreProperties( - TableIdentifier("table_name", None), + tableIdent, Some(BucketSpec(10, List("dt", "country"), List(), List())), - noClustered = false, - noSorted = false)(sql1) + clustered = true, + sorted = true)(sql1) val expected2 = AlterTableStoreProperties( - TableIdentifier("table_name", None), - Some(BucketSpec(10, List("dt", "country"), List("dt", "country"), - List(Ascending, Descending))), - noClustered = false, - noSorted = false)(sql2) + tableIdent, + Some(BucketSpec( + 10, List("dt", "country"), List("dt", "country"), List(Ascending, Descending))), + clustered = true, + sorted = true)(sql2) val expected3 = AlterTableStoreProperties( - TableIdentifier("table_name", None), + tableIdent, Some(BucketSpec(20, List(), List(), List())), - noClustered = false, - noSorted = false)(sql3) + clustered = true, + sorted = true)(sql3) val expected4 = AlterTableStoreProperties( - TableIdentifier("table_name", None), - None, - noClustered = true, - noSorted = false)(sql4) + tableIdent, None, clustered = false, sorted = true)(sql4) val expected5 = AlterTableStoreProperties( - TableIdentifier("table_name", None), - None, - noClustered = false, - noSorted = true)(sql5) + tableIdent, None, clustered = true, sorted = false)(sql5) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) From 6c3f9946c5244bc61776183f412acb7be068795a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Mar 2016 00:42:32 -0800 Subject: [PATCH 06/12] Significant cleanup + documentation in ALTER TABLE This commit cleans up the case class signatures representing ALTER TABLE commands along with the code that parses them. This commit also adds a lot of missing documentation that was sorely needed for AlterTableCommandParser to be readable. The cleanup in this commit concerns only ALTER TABLE commands, but is expected to spread to other parts of the PR in subsequent commits. --- .../sql/catalyst/parser/ParserUtils.scala | 1 + .../command/AlterTableCommandParser.scala | 565 +++++++++--------- .../spark/sql/execution/command/ddl.scala | 148 ++--- .../execution/command/DDLCommandSuite.scala | 217 +++---- 4 files changed, 472 insertions(+), 459 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index c105b53f1fc62..57b2a8c06dcfc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.types._ object ParserUtils { object Token { + // Match on (text, children) def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { CurrentOrigin.setPosition(node.line, node.positionInLine) node.pattern diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 6d53259e9e918..8a18e896453af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -21,7 +21,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} +import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending, SortDirection} import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources._ @@ -37,15 +38,14 @@ object AlterTableCommandParser { /** * Parse the given node assuming it is an alter table command. */ - def parse(v1: ASTNode): LogicalPlan = { - v1.children match { - case (tabName @ Token("TOK_TABNAME", _)) :: restNodes => - val tableIdent: TableIdentifier = extractTableIdent(tabName) - val partitionSpec = getClauseOption("TOK_PARTSPEC", v1.children) - val partition = partitionSpec.flatMap(parsePartitionSpec) - matchAlterTableCommands(v1, restNodes, tableIdent, partition) + def parse(node: ASTNode): LogicalPlan = { + node.children match { + case (tabName @ Token("TOK_TABNAME", _)) :: otherNodes => + val tableIdent = extractTableIdent(tabName) + val partSpec = getClauseOption("TOK_PARTSPEC", node.children).map(parsePartitionSpec) + matchAlterTableCommands(node, otherNodes, tableIdent, partSpec) case _ => - throw new AnalysisException(s"Could not parse alter table command: '${v1.text}'") + parseFailed("Could not parse ALTER TABLE command", node) } } @@ -53,355 +53,380 @@ object AlterTableCommandParser { cleanIdentifier(unquoteString(s)) } - private def parsePartitionSpec(node: ASTNode): Option[Map[String, Option[String]]] = { + private def parseFailed(msg: String, node: ASTNode): Nothing = { + throw new AnalysisException(s"$msg: '${node.source}") + } + + /** + * Extract partition spec from the given [[ASTNode]] as a map, assuming it exists. + * + * Expected format: + * +- TOK_PARTSPEC + * :- TOK_PARTVAL + * : :- dt + * : +- '2008-08-08' + * +- TOK_PARTVAL + * :- country + * +- 'us' + */ + private def parsePartitionSpec(node: ASTNode): Map[String, String] = { node match { case Token("TOK_PARTSPEC", partitions) => - val spec = partitions.map { + partitions.map { + // Note: sometimes there's an equal sign between the key and the value + // (e.g. drop partitions). We should figure out why... + case Token("TOK_PARTVAL", ident :: Token("=", Nil) :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) case Token("TOK_PARTVAL", ident :: constant :: Nil) => - (cleanAndUnquoteString(ident.text), Some(cleanAndUnquoteString(constant.text))) + (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) case Token("TOK_PARTVAL", ident :: Nil) => - (cleanAndUnquoteString(ident.text), None) + (cleanAndUnquoteString(ident.text), null) + case _ => + parseFailed("Invalid ALTER TABLE command", node) }.toMap - Some(spec) - case _ => None + case _ => + parseFailed("Expected partition spec in ALTER TABLE command", node) } } - private def extractTableProps(node: ASTNode): Map[String, Option[String]] = { + /** + * Extract table properties from the given [[ASTNode]] as a map, assuming it exists. + * + * Expected format: + * +- TOK_TABLEPROPERTIES + * +- TOK_TABLEPROPLIST + * :- TOK_TABLEPROPERTY + * : :- 'test' + * : +- 'value' + * +- TOK_TABLEPROPERTY + * :- 'comment' + * +- 'new_comment' + */ + private def extractTableProps(node: ASTNode): Map[String, String] = { node match { case Token("TOK_TABLEPROPERTIES", propsList) => propsList.flatMap { case Token("TOK_TABLEPROPLIST", props) => - props.map { - case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) => - val k = cleanAndUnquoteString(key.text) - (k, None) - case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => - val k = cleanAndUnquoteString(key.text) - val v = cleanAndUnquoteString(value.text) - (k, Some(v)) + props.map { case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = cleanAndUnquoteString(key.text) + val v = value match { + case Token("TOK_NULL", Nil) => null + case _ => cleanAndUnquoteString(value.text) + } + (k, v) } + case _ => + parseFailed("Invalid ALTER TABLE command", node) }.toMap case _ => - throw new AnalysisException( - s"Expected table properties in alter table command: '${node.text}'") + parseFailed("Expected table properties in ALTER TABLE command", node) } } - // TODO: This method is massive. Break it down. Also, add some comments... + /** + * Parse an alter table command from a [[ASTNode]] into a [[LogicalPlan]]. + * This follows https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL. + * + * @param node the original [[ASTNode]] to parse. + * @param otherNodes the other [[ASTNode]]s after the first one containing the table name. + * @param tableIdent identifier of the table, parsed from the first [[ASTNode]]. + * @param partition spec identifying the partition this command is concerned with, if any. + */ + // TODO: This method is massive. Break it down. private def matchAlterTableCommands( node: ASTNode, - nodes: Seq[ASTNode], + otherNodes: Seq[ASTNode], tableIdent: TableIdentifier, - partition: Option[Map[String, Option[String]]]): LogicalPlan = { - nodes match { - case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => - val renamedTable = getClause("TOK_TABNAME", renameArgs) - val renamedTableIdent: TableIdentifier = extractTableIdent(renamedTable) - AlterTableRename(tableIdent, renamedTableIdent)(node.source) - + partition: Option[TablePartitionSpec]): LogicalPlan = { + otherNodes match { + // ALTER TABLE table_name RENAME TO new_table_name; + case Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => + val tableNameClause = getClause("TOK_TABNAME", renameArgs) + val newTableIdent = extractTableIdent(tableNameClause) + AlterTableRename(tableIdent, newTableIdent)(node.source) + + // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => - val setTableProperties = extractTableProps(args.head) - AlterTableSetProperties( - tableIdent, - setTableProperties)(node.source) + val properties = extractTableProps(args.head) + AlterTableSetProperties(tableIdent, properties)(node.source) + // ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'key'); case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => - val dropTableProperties = extractTableProps(args.head) - val allowExisting = getClauseOption("TOK_IFEXISTS", args) - AlterTableDropProperties( - tableIdent, - dropTableProperties, allowExisting.isDefined)(node.source) + val properties = extractTableProps(args.head) + val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined + AlterTableUnsetProperties(tableIdent, properties, ifExists)(node.source) + // ALTER TABLE table_name [PARTITION spec] SET SERDE serde_name [WITH SERDEPROPERTIES props]; case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ => - // When SET SERDE serde_classname WITH SERDEPROPERTIES, this is None - val serdeProperties: Option[Map[String, Option[String]]] = - serdeArgs.headOption.map(extractTableProps) AlterTableSerDeProperties( tableIdent, Some(cleanAndUnquoteString(serdeClassName)), - serdeProperties, + serdeArgs.headOption.map(extractTableProps), partition)(node.source) + // ALTER TABLE table_name [PARTITION spec] SET SERDEPROPERTIES serde_properties; case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => - val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) AlterTableSerDeProperties( tableIdent, None, - Some(serdeProperties), + Some(extractTableProps(args.head)), partition)(node.source) - case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterSortArgs :: Nil) :: _ => - clusterSortArgs match { - case Token("TOK_ALTERTABLE_BUCKETS", bucketArgsHead :: bucketArgs) => - val bucketCols = bucketArgsHead.children.map(_.text) - val (sortCols, sortDirections, numBuckets) = { - if (bucketArgs.head.text == "TOK_TABCOLNAME") { - val (cols, directions) = bucketArgs.head.children.map { - case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => - (colName, Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => - (colName, Descending) - }.unzip - (cols, directions, bucketArgs.last.text.toInt) - } else { - (Nil, Nil, bucketArgs.head.text.toInt) - } - } - AlterTableStoreProperties( - tableIdent, - Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)), - clustered = true, - sorted = true)(node.source) - case Token("TOK_NOT_CLUSTERED", Nil) => - AlterTableStoreProperties( - tableIdent, None, clustered = false, sorted = true)(node.source) - case Token("TOK_NOT_SORTED", Nil) => - AlterTableStoreProperties( - tableIdent, None, clustered = true, sorted = false)(node.source) + // ALTER TABLE table_name CLUSTERED BY (col, ...) [SORTED BY (col, ...)] INTO n BUCKETS; + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_ALTERTABLE_BUCKETS", b) :: Nil) :: _ => + val clusterCols: Seq[String] = b.head match { + case Token("TOK_TABCOLNAME", children) => children.map(_.text) + case _ => parseFailed("Invalid ALTER TABLE command", node) } - - case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => - val num = bucketNum.toInt - val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) - AlterTableStoreProperties( - tableIdent, - buckets, - clustered = true, - sorted = true)(node.source) - - case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => - // ALTER TABLE table_name NOT SKEWED - AlterTableSkewed( - tableIdent, - Nil, - Nil, - storedAsDirs = false, - notSkewed = true, - notStoredAsDirs = false)(node.source) - - case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => - // ALTER TABLE table_name NOT STORED AS DIRECTORIES - AlterTableSkewed( + // If sort columns are specified, num buckets should be the third arg. + // If sort columns are not specified, num buckets should be the second arg. + val (sortCols: Seq[String], sortDirections: Seq[SortDirection], numBuckets: Int) = { + b.tail match { + case Token("TOK_TABCOLNAME", children) :: numBucketsNode :: Nil => + val (cols, directions) = children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(col, Nil) :: Nil) => (col, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(col, Nil) :: Nil) => (col, Descending) + }.unzip + (cols, directions, numBucketsNode.text.toInt) + case numBucketsNode :: Nil => + (Nil, Nil, numBucketsNode.text.toInt) + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + } + AlterTableStorageProperties( tableIdent, - Nil, - Nil, - storedAsDirs = false, - notSkewed = false, - notStoredAsDirs = true)(node.source) - - case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: _ => - val skewedArgs = tableSkewed match { - case Token("TOK_ALTERTABLE_SKEWED", args :: Nil) => - args match { - case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => - val cols = skewedCols.children.map(n => cleanAndUnquoteString(n.text)) - val values = skewedValues match { - case Token("TOK_TABCOLVALUE", colVal) => - Seq(colVal.map(n => cleanAndUnquoteString(n.text))) - case Token("TOK_TABCOLVALUE_PAIR", pairs) => - pairs.map { - case Token("TOK_TABCOLVALUES", colVals :: Nil) => - colVals match { - case Token("TOK_TABCOLVALUE", vals) => - vals.map(n => cleanAndUnquoteString(n.text)) - } - } - } - - val storedAsDirs = stored match { - case Token("TOK_STOREDASDIRS", Nil) :: Nil => true - case _ => false - } - - (cols, values, storedAsDirs) + BucketSpec(numBuckets, clusterCols, sortCols, sortDirections))(node.source) + + // ALTER TABLE table_name NOT CLUSTERED + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ => + AlterTableNotClustered(tableIdent)(node.source) + + // ALTER TABLE table_name NOT SORTED + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_SORTED", Nil) :: Nil) :: _ => + AlterTableNotSorted(tableIdent)(node.source) + + // ALTER TABLE table_name SKEWED BY (col1, col2) + // ON ((col1_value, col2_value) [, (col1_value, col2_value), ...]) + // [STORED AS DIRECTORIES]; + case Token("TOK_ALTERTABLE_SKEWED", + Token("TOK_TABLESKEWED", + Token("TOK_TABCOLNAME", colNames) :: colValues :: rest) :: Nil) :: _ => + // Example format: + // + // +- TOK_ALTERTABLE_SKEWED + // :- TOK_TABLESKEWED + // : :- TOK_TABCOLNAME + // : : :- dt + // : : +- country + // :- TOK_TABCOLVALUE_PAIR + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2008-08-08' + // : : : +- 'us' + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2009-09-09' + // : : : +- 'uk' + // +- TOK_STOREASDIR + val names = colNames.map { n => cleanAndUnquoteString(n.text) } + val values = colValues match { + case Token("TOK_TABCOLVALUE", vals) => + Seq(vals.map { n => cleanAndUnquoteString(n.text) }) + case Token("TOK_TABCOLVALUE_PAIR", pairs) => + pairs.map { + case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", vals) :: Nil) => + vals.map { n => cleanAndUnquoteString(n.text) } + case _ => + parseFailed("Invalid ALTER TABLE command", node) } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + val storedAsDirs = rest match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false } - val (cols, values, storedAsDirs) = skewedArgs AlterTableSkewed( tableIdent, - cols, + names, values, - storedAsDirs, - notSkewed = false, - notStoredAsDirs = false)(node.source) + storedAsDirs)(node.source) + + // ALTER TABLE table_name NOT SKEWED + case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => + AlterTableNotSkewed(tableIdent)(node.source) + + // ALTER TABLE table_name NOT STORED AS DIRECTORIES + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) :: _ => + AlterTableNotStoredAsDirs(tableIdent)(node.source) + // ALTER TABLE table_name SET SKEWED LOCATION (col1="loc1" [, (col2, col3)="loc2", ...] ); case Token("TOK_ALTERTABLE_SKEWED_LOCATION", - Token("TOK_SKEWED_LOCATIONS", - Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => - val skewedMaps = locationMaps.map { - case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => - val k = key match { - case Token(const, Nil) => Seq(cleanAndUnquoteString(const)) - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => - vals.map(n => cleanAndUnquoteString(n.text)) - } + Token("TOK_SKEWED_LOCATIONS", + Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => + // Expected format: + // + // +- TOK_ALTERTABLE_SKEWED_LOCATION + // +- TOK_SKEWED_LOCATIONS + // +- TOK_SKEWED_LOCATION_LIST + // :- TOK_SKEWED_LOCATION_MAP + // : :- 'col1' + // : +- 'loc1' + // +- TOK_SKEWED_LOCATION_MAP + // :- TOK_TABCOLVALUES + // : +- TOK_TABCOLVALUE + // : :- 'col2' + // : +- 'col3' + // +- 'loc2' + val skewedMaps = locationMaps.flatMap { + case Token("TOK_SKEWED_LOCATION_MAP", col :: loc :: Nil) => + col match { + case Token(const, Nil) => + Seq((cleanAndUnquoteString(const), cleanAndUnquoteString(loc.text))) + case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", keys) :: Nil) => + keys.map { k => (cleanAndUnquoteString(k.text), cleanAndUnquoteString(loc.text)) } } - (k, cleanAndUnquoteString(value.text)) + case _ => + parseFailed("Invalid ALTER TABLE command", node) }.toMap AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) - case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: _ => - val (allowExisting, parts) = addPartsArgs match { - case Token("TOK_IFNOTEXISTS", Nil) :: others => (true, others) - case _ => (false, addPartsArgs) + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION spec [LOCATION 'loc1'] + // spec [LOCATION 'loc2'] ...; + case Token("TOK_ALTERTABLE_ADDPARTS", args) :: _ => + val (ifNotExists, parts) = args.head match { + case Token("TOK_IFNOTEXISTS", Nil) => (true, args.tail) + case _ => (false, args) } - val partitions: ArrayBuffer[(Map[String, Option[String]], Option[String])] = - new ArrayBuffer() - var currentPart: Map[String, Option[String]] = null - parts.map { - case t @ Token("TOK_PARTSPEC", partArgs) => - if (currentPart != null) { - partitions += ((currentPart, None)) - } - currentPart = parsePartitionSpec(t).get + // List of (spec, location) to describe partitions to add + // Each partition spec may or may not be followed by a location + val parsedParts = new ArrayBuffer[(TablePartitionSpec, Option[String])] + parts.foreach { + case t @ Token("TOK_PARTSPEC", _) => + parsedParts += ((parsePartitionSpec(t), None)) case Token("TOK_PARTITIONLOCATION", loc :: Nil) => - val location = unquoteString(loc.text) - if (currentPart != null) { - partitions += ((currentPart, Some(location))) - currentPart = null - } else { - // We should not reach here - throw new AnalysisException("Partition location must follow a partition spec.") + // Update the location of the last partition we just added + if (parsedParts.nonEmpty) { + val (spec, _) = parsedParts.remove(parsedParts.length - 1) + parsedParts += ((spec, Some(unquoteString(loc.text)))) } + case _ => + parseFailed("Invalid ALTER TABLE command", node) } - if (currentPart != null) { - partitions += ((currentPart, None)) - } - AlterTableAddPartition(tableIdent, partitions, allowExisting)(node.source) + AlterTableAddPartition(tableIdent, parsedParts, ifNotExists)(node.source) - case Token("TOK_ALTERTABLE_RENAMEPART", partArg :: Nil) :: _ => - val Some(newPartition) = parsePartitionSpec(partArg) - AlterTableRenamePartition(tableIdent, partition.get, newPartition)(node.source) + // ALTER TABLE table_name PARTITION spec1 RENAME TO PARTITION spec2; + case Token("TOK_ALTERTABLE_RENAMEPART", spec :: Nil) :: _ => + val newPartition = parsePartitionSpec(spec) + val oldPartition = partition.getOrElse { + parseFailed("Expected old partition spec in ALTER TABLE rename partition command", node) + } + AlterTableRenamePartition(tableIdent, oldPartition, newPartition)(node.source) - case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", - (p @ Token("TOK_PARTSPEC", _)) :: (t @ Token("TOK_TABNAME", _)) :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(p) - val fromTableIdent = extractTableIdent(t) - AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) + // ALTER TABLE table_name_1 EXCHANGE PARTITION spec WITH TABLE table_name_2; + case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", spec :: newTable :: Nil) :: _ => + val parsedSpec = parsePartitionSpec(spec) + val newTableIdent = extractTableIdent(newTable) + AlterTableExchangePartition(tableIdent, newTableIdent, parsedSpec)(node.source) + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ => - val parts = args.collect { - case Token("TOK_PARTSPEC", partitions) => - partitions.map { - case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => - (cleanAndUnquoteString(ident.text), - op.text, cleanAndUnquoteString(constant.text)) - } - } - val allowExisting = getClauseOption("TOK_IFEXISTS", args).isDefined - val purge = getClauseOption("PURGE", args) - val replication = getClauseOption("TOK_REPLICATION", args).map { - case Token("TOK_REPLICATION", replId :: metadata) => - (cleanAndUnquoteString(replId.text), metadata.nonEmpty) - } - AlterTableDropPartition( - tableIdent, - parts, - allowExisting, - purge.isDefined, - replication)(node.source) + val parts = args.collect { case p @ Token("TOK_PARTSPEC", _) => parsePartitionSpec(p) } + val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined + val purge = getClauseOption("PURGE", args).isDefined + AlterTableDropPartition(tableIdent, parts, ifExists, purge)(node.source) - case Token("TOK_ALTERTABLE_ARCHIVE", partArg :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(partArg) - AlterTableArchivePartition(tableIdent, partition)(node.source) + // ALTER TABLE table_name ARCHIVE PARTITION spec; + case Token("TOK_ALTERTABLE_ARCHIVE", spec :: Nil) :: _ => + AlterTableArchivePartition(tableIdent, parsePartitionSpec(spec))(node.source) - case Token("TOK_ALTERTABLE_UNARCHIVE", partArg :: Nil) :: _ => - val Some(partition) = parsePartitionSpec(partArg) - AlterTableUnarchivePartition(tableIdent, partition)(node.source) + // ALTER TABLE table_name UNARCHIVE PARTITION spec; + case Token("TOK_ALTERTABLE_UNARCHIVE", spec :: Nil) :: _ => + AlterTableUnarchivePartition(tableIdent, parsePartitionSpec(spec))(node.source) + // ALTER TABLE table_name [PARTITION spec] SET FILEFORMAT file_format; case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => val Seq(fileFormat, genericFormat) = - getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), - args) - val fFormat = fileFormat.map(_.children.map(n => cleanAndUnquoteString(n.text))) - val gFormat = genericFormat.map(f => cleanAndUnquoteString(f.children(0).text)) + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), args) + // Note: the AST doesn't contain information about which file format is being set here. + // E.g. we can't differentiate between INPUTFORMAT and OUTPUTFORMAT if either is set. + // Right now this just stores the values, but we should figure out how to get the keys. + val fFormat = fileFormat + .map { _.children.map { n => cleanAndUnquoteString(n.text) }} + .getOrElse(Seq()) + val gFormat = genericFormat.map { f => cleanAndUnquoteString(f.children(0).text) } AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + // ALTER TABLE table_name [PARTITION spec] SET LOCATION "loc"; case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) + // ALTER TABLE table_name TOUCH [PARTITION spec]; case Token("TOK_ALTERTABLE_TOUCH", args) :: _ => - val part = getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) + // Note: the partition spec, if it exists, comes after TOUCH, so `partition` should + // always be None here. Instead, we need to parse it from the TOUCH node's children. + val part = getClauseOption("TOK_PARTSPEC", args).map(parsePartitionSpec) AlterTableTouch(tableIdent, part)(node.source) + // ALTER TABLE table_name [PARTITION spec] COMPACT 'compaction_type'; case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) + // ALTER TABLE table_name [PARTITION spec] CONCATENATE; case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => AlterTableMerge(tableIdent, partition)(node.source) - case Token("TOK_ALTERTABLE_RENAMECOL", args) :: _ => - val oldName = args(0).text - val newName = args(1).text - val dataType = nodeToDataType(args(2)) - val afterPos = - getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) - val afterPosCol = afterPos.map { ap => - ap.children match { - case Token(col, Nil) :: Nil => col - case _ => null - } - } - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - val comment = if (args.size > 3) { - args(3) match { - case Token(commentStr, Nil) - if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && - commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => - Some(cleanAndUnquoteString(commentStr)) - case _ => - None + // ALTER TABLE table_name [PARTITION spec] CHANGE [COLUMN] col_old_name col_new_name + // column_type [COMMENT col_comment] [FIRST|AFTER column_name] [CASCADE|RESTRICT]; + case Token("TOK_ALTERTABLE_RENAMECOL", oldName :: newName :: dataType :: args) :: _ => + val afterColName: Option[String] = + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args).map { ap => + ap.children match { + case Token(col, Nil) :: Nil => col + case _ => parseFailed("Invalid ALTER TABLE command", node) + } } - } else { - None + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + val comment = args.headOption.map { + case Token("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", _) => null + case Token("TOK_RESTRICT", _) => null + case Token("TOK_CASCADE", _) => null + case Token(commentStr, Nil) => cleanAndUnquoteString(commentStr) + case _ => parseFailed("Invalid ALTER TABLE command", node) } AlterTableChangeCol( tableIdent, partition, - oldName, - newName, - dataType, + oldName.text, + newName.text, + nodeToDataType(dataType), comment, - afterPos.isDefined, - afterPosCol, - restrict.isDefined, - cascade.isDefined)(node.source) + afterColName, + restrict, + cascade)(node.source) + // ALTER TABLE table_name [PARTITION spec] ADD COLUMNS (name type [COMMENT comment], ...) + // [CASCADE|RESTRICT] case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => - val tableCols = getClause("TOK_TABCOLLIST", args) - val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) - } - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - AlterTableAddCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) - + val columnNodes = getClause("TOK_TABCOLLIST", args).children + val columns = StructType(columnNodes.map(nodeToStructField)) + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + AlterTableAddCol(tableIdent, partition, columns, restrict, cascade)(node.source) + + // ALTER TABLE table_name [PARTITION spec] REPLACE COLUMNS (name type [COMMENT comment], ...) + // [CASCADE|RESTRICT] case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => - val tableCols = getClause("TOK_TABCOLLIST", args) - val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) - } - val restrict = getClauseOption("TOK_RESTRICT", args) - val cascade = getClauseOption("TOK_CASCADE", args) - AlterTableReplaceCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) + val columnNodes = getClause("TOK_TABCOLLIST", args).children + val columns = StructType(columnNodes.map(nodeToStructField)) + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + AlterTableReplaceCol(tableIdent, partition, columns, restrict, cascade)(node.source) case _ => - throw new AnalysisException( - s"Unexpected children nodes in alter table command: '${node.text}'") + parseFailed("Unsupported ALTER TABLE command", node) } } 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 bf05dd4074135..9df58d214a504 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,16 +20,19 @@ package org.apache.spark.sql.execution.command import org.apache.spark.Logging import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ +// Note: The definition of these commands are based on the ones described in +// https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL + /** - * A DDL command expected to be run in the underlying system without Spark parsing the - * query text. + * A DDL command expected to be parsed and run in an underlying system instead of in Spark. */ -abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { +abstract class NativeDDLCommand(val sql: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.runNativeSql(sql) @@ -43,154 +46,165 @@ abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { case class CreateDatabase( databaseName: String, - allowExisting: Boolean, + ifNotExists: Boolean, path: Option[String], comment: Option[String], props: Map[String, String])(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class CreateFunction( functionName: String, alias: String, resourcesMap: Map[String, String], isTemp: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableRename( - tableName: TableIdentifier, - renameTableName: TableIdentifier)(sql: String) - extends NativeDDLCommands(sql) with Logging + oldName: TableIdentifier, + newName: TableIdentifier)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableSetProperties( tableName: TableIdentifier, - setProperties: Map[String, Option[String]])(sql: String) - extends NativeDDLCommands(sql) with Logging + properties: Map[String, String])(sql: String) + extends NativeDDLCommand(sql) with Logging -case class AlterTableDropProperties( +case class AlterTableUnsetProperties( tableName: TableIdentifier, - dropProperties: Map[String, Option[String]], - allowExisting: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + properties: Map[String, String], + ifExists: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableSerDeProperties( tableName: TableIdentifier, serdeClassName: Option[String], - serdeProperties: Option[Map[String, Option[String]]], - partition: Option[Map[String, Option[String]]])(sql: String) - extends NativeDDLCommands(sql) with Logging + serdeProperties: Option[Map[String, String]], + partition: Option[Map[String, String]])(sql: String) + extends NativeDDLCommand(sql) with Logging -case class AlterTableStoreProperties( +case class AlterTableStorageProperties( tableName: TableIdentifier, - buckets: Option[BucketSpec], - clustered: Boolean, - sorted: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + buckets: BucketSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotClustered( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotSorted( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging case class AlterTableSkewed( tableName: TableIdentifier, + // e.g. (dt, country) skewedCols: Seq[String], + // e.g. ('2008-08-08', 'us), ('2009-09-09', 'uk') skewedValues: Seq[Seq[String]], - storedAsDirs: Boolean, - notSkewed: Boolean, - // TODO: what?? - notStoredAsDirs: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + storedAsDirs: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging { + + require(skewedValues.forall(_.size == skewedCols.size), + "number of columns in skewed values do not match number of skewed columns provided") +} + +case class AlterTableNotSkewed( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotStoredAsDirs( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging case class AlterTableSkewedLocation( tableName: TableIdentifier, - skewedMap: Map[Seq[String], String])(sql: String) - extends NativeDDLCommands(sql) with Logging + skewedMap: Map[String, String])(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableAddPartition( tableName: TableIdentifier, - partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], - allowExisting: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + partitionSpecsAndLocs: Seq[(TablePartitionSpec, Option[String])], + ifNotExists: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableRenamePartition( tableName: TableIdentifier, - oldPartition: Map[String, Option[String]], - newPartition: Map[String, Option[String]])(sql: String) - extends NativeDDLCommands(sql) with Logging + oldPartition: TablePartitionSpec, + newPartition: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableExchangePartition( - tableName: TableIdentifier, fromTableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) - extends NativeDDLCommands(sql) with Logging + toTableName: TableIdentifier, + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableDropPartition( tableName: TableIdentifier, - partitions: Seq[Seq[(String, String, String)]], - allowExisting: Boolean, - purge: Boolean, - replication: Option[(String, Boolean)])(sql: String) - extends NativeDDLCommands(sql) with Logging + specs: Seq[TablePartitionSpec], + ifExists: Boolean, + purge: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableArchivePartition( tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) - extends NativeDDLCommands(sql) with Logging + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableUnarchivePartition( tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) - extends NativeDDLCommands(sql) with Logging + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableSetFileFormat( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - fileFormat: Option[Seq[String]], + partitionSpec: Option[TablePartitionSpec], + fileFormat: Seq[String], genericFormat: Option[String])(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableSetLocation( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], + partitionSpec: Option[TablePartitionSpec], location: String)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableTouch( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) - extends NativeDDLCommands(sql) with Logging + partitionSpec: Option[TablePartitionSpec])(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableCompact( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], + partitionSpec: Option[TablePartitionSpec], compactType: String)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableMerge( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) - extends NativeDDLCommands(sql) with Logging + partitionSpec: Option[TablePartitionSpec])(sql: String) + extends NativeDDLCommand(sql) with Logging case class AlterTableChangeCol( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], + partitionSpec: Option[TablePartitionSpec], oldColName: String, newColName: String, dataType: DataType, comment: Option[String], - afterPos: Boolean, - afterPosCol: Option[String], + afterColName: Option[String], restrict: Boolean, cascade: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableAddCol( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], + partitionSpec: Option[TablePartitionSpec], columns: StructType, restrict: Boolean, cascade: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging case class AlterTableReplaceCol( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], + partitionSpec: Option[TablePartitionSpec], columns: StructType, restrict: Boolean, cascade: Boolean)(sql: String) - extends NativeDDLCommands(sql) with Logging + extends NativeDDLCommand(sql) with Logging 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 c786806aee300..c4862907c18db 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 @@ -37,7 +37,7 @@ class DDLCommandSuite extends PlanTest { val parsed = parser.parsePlan(sql) val expected = CreateDatabase( "database_name", - allowExisting = true, + ifNotExists = true, Some("/home/user/db"), Some("database_comment"), Map("a" -> "a", "b" -> "b", "c" -> "c"))(sql) @@ -77,17 +77,13 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSetProperties( - TableIdentifier("table_name", None), - Map("test" -> Some("test"), "comment" -> Some("new_comment")))(sql1) - val expected2 = AlterTableDropProperties( - TableIdentifier("table_name", None), - Map("comment" -> None, "test" -> None), - allowExisting = false)(sql2) - val expected3 = AlterTableDropProperties( - TableIdentifier("table_name", None), - Map("comment" -> None, "test" -> None), - allowExisting = true)(sql3) + tableIdent, Map("test" -> "test", "comment" -> "new_comment"))(sql1) + val expected2 = AlterTableUnsetProperties( + tableIdent, Map("comment" -> null, "test" -> null), ifExists = false)(sql2) + val expected3 = AlterTableUnsetProperties( + tableIdent, Map("comment" -> null, "test" -> null), ifExists = true)(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -121,31 +117,26 @@ class DDLCommandSuite extends PlanTest { val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSerDeProperties( - TableIdentifier("table_name", None), - Some("org.apache.class"), - None, - None)(sql1) + tableIdent, Some("org.apache.class"), None, None)(sql1) val expected2 = AlterTableSerDeProperties( - TableIdentifier("table_name", None), + tableIdent, Some("org.apache.class"), - Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None)(sql2) val expected3 = AlterTableSerDeProperties( - TableIdentifier("table_name", None), - None, - Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), - None)(sql3) + tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None)(sql3) val expected4 = AlterTableSerDeProperties( - TableIdentifier("table_name", None), + tableIdent, Some("org.apache.class"), - Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), - Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql4) + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us")))(sql4) val expected5 = AlterTableSerDeProperties( - TableIdentifier("table_name", None), + tableIdent, None, - Some(Map("columns" -> Some("foo,bar"), "field.delim" -> Some(","))), - Some(Map("test" -> None, "dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql5) + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us")))(sql5) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) @@ -157,47 +148,31 @@ class DDLCommandSuite extends PlanTest { val sql1 = "ALTER TABLE table_name CLUSTERED BY (dt, country) INTO 10 BUCKETS" val sql2 = "ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + "(dt, country DESC) INTO 10 BUCKETS" - val sql3 = "ALTER TABLE table_name INTO 20 BUCKETS" - val sql4 = "ALTER TABLE table_name NOT CLUSTERED" - val sql5 = "ALTER TABLE table_name NOT SORTED" + val sql3 = "ALTER TABLE table_name NOT CLUSTERED" + val sql4 = "ALTER TABLE table_name NOT SORTED" val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) - val parsed5 = parser.parsePlan(sql5) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableStoreProperties( - tableIdent, - Some(BucketSpec(10, List("dt", "country"), List(), List())), - clustered = true, - sorted = true)(sql1) - val expected2 = AlterTableStoreProperties( - tableIdent, - Some(BucketSpec( - 10, List("dt", "country"), List("dt", "country"), List(Ascending, Descending))), - clustered = true, - sorted = true)(sql2) - val expected3 = AlterTableStoreProperties( - tableIdent, - Some(BucketSpec(20, List(), List(), List())), - clustered = true, - sorted = true)(sql3) - val expected4 = AlterTableStoreProperties( - tableIdent, None, clustered = false, sorted = true)(sql4) - val expected5 = AlterTableStoreProperties( - tableIdent, None, clustered = true, sorted = false)(sql5) + val cols = List("dt", "country") + val expected1 = AlterTableStorageProperties( + tableIdent, BucketSpec(10, cols, Nil, Nil))(sql1) + val expected2 = AlterTableStorageProperties( + tableIdent, BucketSpec(10, cols, cols, List(Ascending, Descending)))(sql2) + val expected3 = AlterTableNotClustered(tableIdent)(sql3) + val expected4 = AlterTableNotSorted(tableIdent)(sql4) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) comparePlans(parsed4, expected4) - comparePlans(parsed5, expected5) } test("alter table: skewed") { val sql1 = """ |ALTER TABLE table_name SKEWED BY (dt, country) ON - |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES + |(('2008-08-08', 'us'), ('2009-09-09', 'uk'), ('2010-10-10', 'cn')) STORED AS DIRECTORIES """.stripMargin val sql2 = """ @@ -209,33 +184,36 @@ class DDLCommandSuite extends PlanTest { |ALTER TABLE table_name SKEWED BY (dt, country) ON |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) """.stripMargin + val sql4 = "ALTER TABLE table_name NOT SKEWED" + val sql5 = "ALTER TABLE table_name NOT STORED AS DIRECTORIES" val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSkewed( - TableIdentifier("table_name", None), + tableIdent, Seq("dt", "country"), - Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), - storedAsDirs = true, - notSkewed = false, - notStoredAsDirs = false)(sql1) + Seq(List("2008-08-08", "us"), List("2009-09-09", "uk"), List("2010-10-10", "cn")), + storedAsDirs = true)(sql1) val expected2 = AlterTableSkewed( - TableIdentifier("table_name", None), + tableIdent, Seq("dt", "country"), Seq(List("2008-08-08", "us")), - storedAsDirs = true, - notSkewed = false, - notStoredAsDirs = false)(sql2) + storedAsDirs = true)(sql2) val expected3 = AlterTableSkewed( - TableIdentifier("table_name", None), + tableIdent, Seq("dt", "country"), Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), - storedAsDirs = false, - notSkewed = false, - notStoredAsDirs = false)(sql3) + storedAsDirs = false)(sql3) + val expected4 = AlterTableNotSkewed(tableIdent)(sql4) + val expected5 = AlterTableNotStoredAsDirs(tableIdent)(sql5) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) } test("alter table: skewed location") { @@ -253,10 +231,10 @@ class DDLCommandSuite extends PlanTest { val parsed2 = parser.parsePlan(sql2) val expected1 = AlterTableSkewedLocation( TableIdentifier("table_name", None), - Map(List("123") -> "location1", List("test") -> "location2"))(sql1) + Map("123" -> "location1", "test" -> "location2"))(sql1) val expected2 = AlterTableSkewedLocation( TableIdentifier("table_name", None), - Map(List("2008-08-08", "us") -> "location1", List("test") -> "location2"))(sql2) + Map("2008-08-08" -> "location1", "us" -> "location1", "test" -> "location2"))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -272,9 +250,9 @@ class DDLCommandSuite extends PlanTest { val expected = AlterTableAddPartition( TableIdentifier("table_name", None), Seq( - (Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), Some("location1")), - (Map("dt" -> Some("2009-09-09"), "country" -> Some("uk")), None)), - allowExisting = true)(sql) + (Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), + (Map("dt" -> "2009-09-09", "country" -> "uk"), None)), + ifNotExists = true)(sql) comparePlans(parsed, expected) } @@ -287,8 +265,8 @@ class DDLCommandSuite extends PlanTest { val parsed = parser.parsePlan(sql) val expected = AlterTableRenamePartition( TableIdentifier("table_name", None), - Map("dt" -> Some("2008-08-08"), "country" -> Some("us")), - Map("dt" -> Some("2008-09-09"), "country" -> Some("uk")))(sql) + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2008-09-09", "country" -> "uk"))(sql) comparePlans(parsed, expected) } @@ -302,7 +280,7 @@ class DDLCommandSuite extends PlanTest { val expected = AlterTableExchangePartition( TableIdentifier("table_name_1", None), TableIdentifier("table_name_2", None), - Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) comparePlans(parsed, expected) } @@ -314,28 +292,25 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val sql2 = """ - |ALTER TABLE table_name DROP IF EXISTS PARTITION - |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') - |PURGE FOR METADATA REPLICATION ('test') + |ALTER TABLE table_name DROP PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') PURGE """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val expected1 = AlterTableDropPartition( TableIdentifier("table_name", None), Seq( - List(("dt", "=", "2008-08-08"), ("country", "=", "us")), - List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), - allowExisting = true, - purge = false, - None)(sql1) + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = true, + purge = false)(sql1) val expected2 = AlterTableDropPartition( TableIdentifier("table_name", None), Seq( - List(("dt", "=", "2008-08-08"), ("country", "=", "us")), - List(("dt", "=", "2009-09-09"), ("country", "=", "uk"))), - allowExisting = true, - purge = true, - Some(("test", true)))(sql2) + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = false, + purge = true)(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -345,7 +320,7 @@ class DDLCommandSuite extends PlanTest { val parsed = parser.parsePlan(sql) val expected = AlterTableArchivePartition( TableIdentifier("table_name", None), - Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) comparePlans(parsed, expected) } @@ -354,7 +329,7 @@ class DDLCommandSuite extends PlanTest { val parsed = parser.parsePlan(sql) val expected = AlterTableUnarchivePartition( TableIdentifier("table_name", None), - Map("dt" -> Some("2008-08-08"), "country" -> Some("us")))(sql) + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) comparePlans(parsed, expected) } @@ -374,17 +349,17 @@ class DDLCommandSuite extends PlanTest { val expected1 = AlterTableSetFileFormat( TableIdentifier("table_name", None), None, - Some(List("test", "test", "test", "test", "test")), + List("test", "test", "test", "test", "test"), None)(sql1) val expected2 = AlterTableSetFileFormat( TableIdentifier("table_name", None), None, - Some(List("test", "test", "test")), + List("test", "test", "test"), None)(sql2) val expected3 = AlterTableSetFileFormat( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), - None, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + Seq(), Some("PARQUET"))(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -403,7 +378,7 @@ class DDLCommandSuite extends PlanTest { "new location")(sql1) val expected2 = AlterTableSetLocation( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + Some(Map("dt" -> "2008-08-08", "country" -> "us")), "new location")(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -419,7 +394,7 @@ class DDLCommandSuite extends PlanTest { None)(sql1) val expected2 = AlterTableTouch( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) + Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -439,7 +414,7 @@ class DDLCommandSuite extends PlanTest { "compaction_type")(sql1) val expected2 = AlterTableCompact( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + Some(Map("dt" -> "2008-08-08", "country" -> "us")), "MAJOR")(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -455,7 +430,7 @@ class DDLCommandSuite extends PlanTest { None)(sql1) val expected2 = AlterTableMerge( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))))(sql2) + Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -475,37 +450,35 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableChangeCol( - TableIdentifier("table_name", None), - None, - "col_old_name", - "col_new_name", - IntegerType, - None, - afterPos = false, - None, + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = None, + afterColName = None, restrict = false, cascade = false)(sql1) val expected2 = AlterTableChangeCol( - TableIdentifier("table_name", None), - None, - "col_old_name", - "col_new_name", - IntegerType, - Some("col_comment"), - afterPos = false, - None, + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = Some("col_comment"), + afterColName = None, restrict = false, cascade = true)(sql2) val expected3 = AlterTableChangeCol( - TableIdentifier("table_name", None), - None, - "col_old_name", - "col_new_name", - IntegerType, - Some("col_comment"), - afterPos = true, - Some("column_name"), + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = Some("col_comment"), + afterColName = Some("column_name"), restrict = true, cascade = false)(sql3) comparePlans(parsed1, expected1) @@ -531,7 +504,7 @@ class DDLCommandSuite extends PlanTest { val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() val expected1 = AlterTableAddCol( TableIdentifier("table_name", None), - Some(Map("dt" -> Some("2008-08-08"), "country" -> Some("us"))), + Some(Map("dt" -> "2008-08-08", "country" -> "us")), StructType(Seq( StructField("new_col1", IntegerType, nullable = true, meta1), StructField("new_col2", LongType, nullable = true, meta2))), From 794ae7854291abb3b7558b1465c9385fb29cc22c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Mar 2016 11:09:48 -0800 Subject: [PATCH 07/12] Fix tests --- .../sql/execution/command/AlterTableCommandParser.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 8a18e896453af..9ea9bed99ae5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -73,9 +73,8 @@ object AlterTableCommandParser { node match { case Token("TOK_PARTSPEC", partitions) => partitions.map { - // Note: sometimes there's an equal sign between the key and the value - // (e.g. drop partitions). We should figure out why... - case Token("TOK_PARTVAL", ident :: Token("=", Nil) :: constant :: Nil) => + // Note: sometimes there's a "=", "<" or ">" between the key and the value + case Token("TOK_PARTVAL", ident :: conj :: constant :: Nil) => (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) case Token("TOK_PARTVAL", ident :: constant :: Nil) => (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) From 63e99c3d3d51d2dddc276504a26e3bab3ec86478 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Mar 2016 14:17:41 -0800 Subject: [PATCH 08/12] Fix CREATE FUNCTION with dot + clean up SparkQl This used to fail: CREATE FUNCTION dbname.fname AS alias. Now it doesn't! --- .../sql/catalyst/parser/ParserUtils.scala | 5 + .../apache/spark/sql/execution/SparkQl.scala | 121 +++++++++++------- .../command/AlterTableCommandParser.scala | 5 - .../execution/command/DDLCommandSuite.scala | 23 +++- 4 files changed, 97 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 57b2a8c06dcfc..3eac54262614e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.parser +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types._ @@ -80,6 +81,10 @@ object ParserUtils { } } + def parseFailed(msg: String, node: ASTNode): Nothing = { + throw new AnalysisException(s"$msg: '${node.source}") + } + def getClauses( clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 66087711bb9eb..a4fd58b379475 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -33,22 +33,20 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command } + /** + * For each node, extract properties in the form of a list ['key1', 'key2', 'key3', 'value'] + * into a pair (key1.key2.key3, value). + */ protected def extractProps( - node: ASTNode, - firstLevelProp: String, - secondLevelProp: String): Seq[(String, String)] = { - node match { - case Token(x, options) if x == firstLevelProp => - options.map { - case Token(y, keysAndValue) if y == secondLevelProp => - val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - case _ => - throw new AnalysisException(s"Expected property '$secondLevelProp' in '${node.text}'") - } - case _ => - throw new AnalysisException(s"Expected property '$firstLevelProp' in '${node.text}'") + props: Seq[ASTNode], + expectedNodeText: String): Seq[(String, String)] = { + props.map { + case Token(x, keysAndValue) if x == expectedNodeText => + val key = keysAndValue.init.map { x => unquoteString(x.text) }.mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + case p => + parseFailed(s"Expected property '$expectedNodeText' in command", p) } } @@ -85,37 +83,63 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly val tableIdent = extractTableIdent(nameParts) RefreshTable(tableIdent) - case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: createDatabaseArgs) => - val Seq( - allowExisting, - dbLocation, - databaseComment, - dbprops) = getClauses(Seq( + // CREATE DATABASE [IF NOT EXISTS] database_name [COMMENT database_comment] + // [LOCATION path] [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]; + case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: args) => + val Seq(ifNotExists, dbLocation, databaseComment, dbprops) = getClauses(Seq( "TOK_IFNOTEXISTS", "TOK_DATABASELOCATION", "TOK_DATABASECOMMENT", - "TOK_DATABASEPROPERTIES"), createDatabaseArgs) + "TOK_DATABASEPROPERTIES"), args) val location = dbLocation.map { case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc) + case _ => parseFailed("Invalid CREATE DATABASE command", node) } val comment = databaseComment.map { - case Token("TOK_DATABASECOMMENT", Token(comment, Nil) :: Nil) => unquoteString(comment) + case Token("TOK_DATABASECOMMENT", Token(com, Nil) :: Nil) => unquoteString(com) + case _ => parseFailed("Invalid CREATE DATABASE command", node) } - val props: Map[String, String] = dbprops.toSeq.flatMap { - case Token("TOK_DATABASEPROPERTIES", propList) => - propList.flatMap(extractProps(_, "TOK_DBPROPLIST", "TOK_TABLEPROPERTY")) + val props = dbprops.toSeq.flatMap { + case Token("TOK_DATABASEPROPERTIES", Token("TOK_DBPROPLIST", propList) :: Nil) => + extractProps(propList, "TOK_TABLEPROPERTY") + case _ => parseFailed("Invalid CREATE DATABASE command", node) }.toMap - CreateDatabase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) + CreateDatabase(databaseName, ifNotExists.isDefined, location, comment, props)(node.source) - case Token("TOK_CREATEFUNCTION", func :: as :: createFuncArgs) => - val funcName = func.map(x => unquoteString(x.text)).mkString(".") - val asName = unquoteString(as.text) - val Seq( - rList, - temp) = getClauses(Seq( - "TOK_RESOURCE_LIST", - "TOK_TEMPORARY"), createFuncArgs) - val resourcesMap: Map[String, String] = rList.toSeq.flatMap { + // CREATE [TEMPORARY] FUNCTION [db_name.]function_name AS class_name + // [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] ]; + case Token("TOK_CREATEFUNCTION", args) => + // Example format: + // + // TOK_CREATEFUNCTION + // :- db_name + // :- func_name + // :- alias + // +- TOK_RESOURCE_LIST + // :- TOK_RESOURCE_URI + // : :- TOK_JAR + // : +- '/path/to/jar' + // +- TOK_RESOURCE_URI + // :- TOK_FILE + // +- 'path/to/file' + val (funcNameArgs, otherArgs) = args.partition { + case Token("TOK_RESOURCE_LIST", _) => false + case Token("TOK_TEMPORARY", _) => false + case Token(_, Nil) => true + case _ => parseFailed("Invalid CREATE FUNCTION command", node) + } + // If database name is specified, there are 3 tokens, otherwise 2. + val (funcName, alias) = funcNameArgs match { + case Token(dbName, Nil) :: Token(fname, Nil) :: Token(aname, Nil) :: Nil => + (unquoteString(dbName) + "." + unquoteString(fname), unquoteString(aname)) + case Token(fname, Nil) :: Token(aname, Nil) :: Nil => + (unquoteString(fname), unquoteString(aname)) + case _ => + parseFailed("Invalid CREATE FUNCTION command", node) + } + // Extract other keywords, if they exist + val Seq(rList, temp) = getClauses(Seq("TOK_RESOURCE_LIST", "TOK_TEMPORARY"), otherArgs) + val resourcesMap = rList.toSeq.flatMap { case Token("TOK_RESOURCE_LIST", resources) => resources.map { case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) => @@ -123,11 +147,14 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case Token("TOK_JAR", Nil) => "jar" case Token("TOK_FILE", Nil) => "file" case Token("TOK_ARCHIVE", Nil) => "archive" + case Token(f, _) => parseFailed(s"Unexpected resource format '$f'", node) } (resourceType, unquoteString(rPath)) + case _ => parseFailed("Invalid CREATE FUNCTION command", node) } + case _ => parseFailed("Invalid CREATE FUNCTION command", node) }.toMap - CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source) + CreateFunction(funcName, alias, resourcesMap, temp.isDefined)(node.source) case Token("TOK_ALTERTABLE", alterTableArgs) => AlterTableCommandParser.parse(node) @@ -135,7 +162,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( temp, - allowExisting, + ifNotExists, Some(tabName), tableCols, Some(Token("TOK_TABLEPROVIDER", providerNameParts)), @@ -147,22 +174,22 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "TOK_TABLEPROVIDER", "TOK_TABLEOPTIONS", "TOK_QUERY"), createTableArgs) - val tableIdent: TableIdentifier = extractTableIdent(tabName) - val columns = tableCols.map { case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + case _ => parseFailed("Invalid CREATE TABLE command", node) } - val provider = providerNameParts.map { case Token(name, Nil) => name + case _ => parseFailed("Invalid CREATE TABLE command", node) }.mkString(".") - - val options: Map[String, String] = - tableOpts.toSeq.flatMap(extractProps(_, "TOK_TABLEOPTIONS", "TOK_TABLEOPTION")).toMap + val options = tableOpts.toSeq.flatMap { + case Token("TOK_TABLEOPTIONS", opts) => extractProps(opts, "TOK_TABLEOPTION") + case _ => parseFailed("Invalid CREATE TABLE command", node) + }.toMap val asClause = tableAs.map(nodeToPlan) - if (temp.isDefined && allowExisting.isDefined) { + if (temp.isDefined && ifNotExists.isDefined) { throw new AnalysisException( "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.") } @@ -173,7 +200,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "a CREATE TABLE AS SELECT statement does not allow column definitions.") } - val mode = if (allowExisting.isDefined) { + val mode = if (ifNotExists.isDefined) { SaveMode.Ignore } else if (temp.isDefined) { SaveMode.Overwrite @@ -196,7 +223,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly provider, temp.isDefined, options, - allowExisting.isDefined, + ifNotExists.isDefined, managedIfNoPath = false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 9ea9bed99ae5e..56daa83d3fb41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.command import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending, SortDirection} @@ -53,10 +52,6 @@ object AlterTableCommandParser { cleanIdentifier(unquoteString(s)) } - private def parseFailed(msg: String, node: ASTNode): Nothing = { - throw new AnalysisException(s"$msg: '${node.source}") - } - /** * Extract partition spec from the given [[ASTNode]] as a map, assuming it exists. * 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 c4862907c18db..d073c4799a7fb 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 @@ -45,19 +45,32 @@ class DDLCommandSuite extends PlanTest { } test("create function") { - val sql = + val sql1 = """ |CREATE TEMPORARY FUNCTION helloworld as |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', |FILE 'path/to/file' """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateFunction( + val sql2 = + """ + |CREATE FUNCTION hello.world as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE 'path/to/file' + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val expected1 = CreateFunction( "helloworld", "com.matthewrathbone.example.SimpleUDFExample", Map("jar" -> "/path/to/jar", "file" -> "path/to/file"), - isTemp = true)(sql) - comparePlans(parsed, expected) + isTemp = true)(sql1) + val expected2 = CreateFunction( + "hello.world", + "com.matthewrathbone.example.SimpleUDFExample", + Map("archive" -> "/path/to/archive", "file" -> "path/to/file"), + isTemp = false)(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) } test("alter table: rename table") { From 307a588009dcd8479fe219157b25a8c4b420c126 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Mar 2016 14:40:34 -0800 Subject: [PATCH 09/12] Group (col_name, sort_dir) in BucketSpec --- .../main/scala/org/apache/spark/sql/DataFrameWriter.scala | 4 +++- .../sql/execution/command/AlterTableCommandParser.scala | 2 +- .../apache/spark/sql/execution/datasources/bucket.scala | 8 +++++--- .../spark/sql/execution/command/DDLCommandSuite.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++++++-- .../org/apache/spark/sql/sources/BucketedReadSuite.scala | 2 +- 6 files changed, 18 insertions(+), 10 deletions(-) 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 6d8c8f6b4f979..85aa3b88cb4ff 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 @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.Ascending import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, ResolvedDataSource} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils @@ -311,7 +312,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { s"partitionBy columns '${partitioningColumns.get.mkString(", ")}'") } - BucketSpec(n, normalizedBucketColNames.get, normalizedSortColNames.getOrElse(Nil)) + val sortColumns = normalizedSortColNames.getOrElse(Nil).map { n => (n, Ascending) } + BucketSpec(n, normalizedBucketColNames.get, sortColumns) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 56daa83d3fb41..3c51a79cdc555 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -190,7 +190,7 @@ object AlterTableCommandParser { } AlterTableStorageProperties( tableIdent, - BucketSpec(numBuckets, clusterCols, sortCols, sortDirections))(node.source) + BucketSpec(numBuckets, clusterCols, sortCols.zip(sortDirections)))(node.source) // ALTER TABLE table_name NOT CLUSTERED case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index e6ea7bf1ceeb9..1a1332b40da30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -27,13 +27,15 @@ import org.apache.spark.sql.catalyst.expressions.SortDirection * * @param numBuckets number of buckets. * @param bucketColumnNames the names of the columns that used to generate the bucket id. - * @param sortColumnNames the names of the columns that used to sort data in each bucket. + * @param sortColumns (name, sort direction) of columns that used to sort data in each bucket. */ private[sql] case class BucketSpec( numBuckets: Int, bucketColumnNames: Seq[String], - sortColumnNames: Seq[String], - sortDirections: Seq[SortDirection] = Nil) + sortColumns: Seq[(String, SortDirection)]) { + + def sortColumnNames: Seq[String] = sortColumns.map { case (name, _) => name } +} private[sql] object BucketingUtils { // The file name of bucketed data should have 3 parts: 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 d073c4799a7fb..e77db464e5852 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 @@ -170,9 +170,9 @@ class DDLCommandSuite extends PlanTest { val tableIdent = TableIdentifier("table_name", None) val cols = List("dt", "country") val expected1 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, Nil, Nil))(sql1) + tableIdent, BucketSpec(10, cols, Nil))(sql1) val expected2 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, cols, List(Ascending, Descending)))(sql2) + tableIdent, BucketSpec(10, cols, List(("dt", Ascending), ("country", Descending))))(sql2) val expected3 = AlterTableNotClustered(tableIdent)(sql3) val expected4 = AlterTableNotSorted(tableIdent)(sql4) comparePlans(parsed1, expected1) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index a0f4436dc8c35..4170009d02f25 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -172,7 +172,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val partitionColumns = getColumnNames("part") val bucketSpec = table.properties.get("spark.sql.sources.schema.numBuckets").map { n => - BucketSpec(n.toInt, getColumnNames("bucket"), getColumnNames("sort")) + BucketSpec( + n.toInt, + getColumnNames("bucket"), + getColumnNames("sort").map { n => (n, Ascending) }) } val options = table.storage.serdeProperties @@ -245,7 +248,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { - val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames, _) = bucketSpec.get + val BucketSpec(numBuckets, bucketColumnNames, sortColumns) = bucketSpec.get tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) tableProperties.put("spark.sql.sources.schema.numBucketCols", @@ -254,6 +257,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte tableProperties.put(s"spark.sql.sources.schema.bucketCol.$index", bucketCol) } + val sortColumnNames = sortColumns.map { case (name, _) => name } if (sortColumnNames.nonEmpty) { tableProperties.put("spark.sql.sources.schema.numSortCols", sortColumnNames.length.toString) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 04d11be18acf4..35573f62dc633 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -81,7 +81,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // This test verifies parts of the plan. Disable whole stage codegen. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k") - val BucketSpec(numBuckets, bucketColumnNames, _, _) = bucketSpec + val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec // Limit: bucket pruning only works when the bucket column has one and only one column assert(bucketColumnNames.length == 1) val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head) From 6ad8dd5f5294fb7c9424651920706645d1dba8f9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Mar 2016 14:51:57 -0800 Subject: [PATCH 10/12] Minor changes --- .../sql/catalyst/parser/ParserUtils.scala | 13 +++--- .../apache/spark/sql/execution/SparkQl.scala | 2 +- .../execution/command/DDLCommandSuite.scala | 45 ++++++++++--------- 3 files changed, 34 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 3eac54262614e..0c2e481954a5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -81,10 +81,6 @@ object ParserUtils { } } - def parseFailed(msg: String, node: ASTNode): Nothing = { - throw new AnalysisException(s"$msg: '${node.source}") - } - def getClauses( clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { @@ -166,7 +162,14 @@ object ParserUtils { } /** - * Throw an exception because we cannot parse the given node. + * Throw an exception because we cannot parse the given node for some unexpected reason. + */ + def parseFailed(msg: String, node: ASTNode): Nothing = { + throw new AnalysisException(s"$msg: '${node.source}") + } + + /** + * Throw an exception because there are no rules to parse the node. */ def noParseRule(msg: String, node: ASTNode): Nothing = { throw new NotImplementedError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index a4fd58b379475..d12dab567b00a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -37,7 +37,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly * For each node, extract properties in the form of a list ['key1', 'key2', 'key3', 'value'] * into a pair (key1.key2.key3, value). */ - protected def extractProps( + private def extractProps( props: Seq[ASTNode], expectedNodeText: String): Seq[(String, String)] = { props.map { 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 e77db464e5852..1b19423755d7d 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 @@ -242,11 +242,12 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSkewedLocation( - TableIdentifier("table_name", None), + tableIdent, Map("123" -> "location1", "test" -> "location2"))(sql1) val expected2 = AlterTableSkewedLocation( - TableIdentifier("table_name", None), + tableIdent, Map("2008-08-08" -> "location1", "us" -> "location1", "test" -> "location2"))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -310,15 +311,16 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableDropPartition( - TableIdentifier("table_name", None), + tableIdent, Seq( Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2009-09-09", "country" -> "uk")), ifExists = true, purge = false)(sql1) val expected2 = AlterTableDropPartition( - TableIdentifier("table_name", None), + tableIdent, Seq( Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2009-09-09", "country" -> "uk")), @@ -359,18 +361,19 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSetFileFormat( - TableIdentifier("table_name", None), + tableIdent, None, List("test", "test", "test", "test", "test"), None)(sql1) val expected2 = AlterTableSetFileFormat( - TableIdentifier("table_name", None), + tableIdent, None, List("test", "test", "test"), None)(sql2) val expected3 = AlterTableSetFileFormat( - TableIdentifier("table_name", None), + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), Seq(), Some("PARQUET"))(sql3) @@ -385,12 +388,13 @@ class DDLCommandSuite extends PlanTest { "SET LOCATION 'new location'" val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableSetLocation( - TableIdentifier("table_name", None), + tableIdent, None, "new location")(sql1) val expected2 = AlterTableSetLocation( - TableIdentifier("table_name", None), + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), "new location")(sql2) comparePlans(parsed1, expected1) @@ -402,11 +406,12 @@ class DDLCommandSuite extends PlanTest { val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableTouch( - TableIdentifier("table_name", None), + tableIdent, None)(sql1) val expected2 = AlterTableTouch( - TableIdentifier("table_name", None), + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -421,12 +426,13 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableCompact( - TableIdentifier("table_name", None), + tableIdent, None, "compaction_type")(sql1) val expected2 = AlterTableCompact( - TableIdentifier("table_name", None), + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), "MAJOR")(sql2) comparePlans(parsed1, expected1) @@ -438,12 +444,10 @@ class DDLCommandSuite extends PlanTest { val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableMerge( - TableIdentifier("table_name", None), - None)(sql1) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableMerge(tableIdent, None)(sql1) val expected2 = AlterTableMerge( - TableIdentifier("table_name", None), - Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -515,8 +519,9 @@ class DDLCommandSuite extends PlanTest { val parsed2 = parser.parsePlan(sql2) val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() + val tableIdent = TableIdentifier("table_name", None) val expected1 = AlterTableAddCol( - TableIdentifier("table_name", None), + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), StructType(Seq( StructField("new_col1", IntegerType, nullable = true, meta1), @@ -524,7 +529,7 @@ class DDLCommandSuite extends PlanTest { restrict = false, cascade = true)(sql1) val expected2 = AlterTableReplaceCol( - TableIdentifier("table_name", None), + tableIdent, None, StructType(Seq( StructField("new_col1", IntegerType, nullable = true, meta1), From 7f3281eb5408054d0fdbf505177ca8253bfef0a2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 11 Mar 2016 11:43:33 -0800 Subject: [PATCH 11/12] Revert "Group (col_name, sort_dir) in BucketSpec" This reverts commit 307a588009dcd8479fe219157b25a8c4b420c126. --- .../main/scala/org/apache/spark/sql/DataFrameWriter.scala | 4 +--- .../sql/execution/command/AlterTableCommandParser.scala | 2 +- .../apache/spark/sql/execution/datasources/bucket.scala | 8 +++----- .../spark/sql/execution/command/DDLCommandSuite.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++------ .../org/apache/spark/sql/sources/BucketedReadSuite.scala | 2 +- 6 files changed, 10 insertions(+), 18 deletions(-) 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 c8907b085f3e4..3349b8421b3e8 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 @@ -24,7 +24,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.expressions.Ascending import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, DataSource} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils @@ -313,8 +312,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { s"partitionBy columns '${partitioningColumns.get.mkString(", ")}'") } - val sortColumns = normalizedSortColNames.getOrElse(Nil).map { n => (n, Ascending) } - BucketSpec(n, normalizedBucketColNames.get, sortColumns) + BucketSpec(n, normalizedBucketColNames.get, normalizedSortColNames.getOrElse(Nil)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 3c51a79cdc555..56daa83d3fb41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -190,7 +190,7 @@ object AlterTableCommandParser { } AlterTableStorageProperties( tableIdent, - BucketSpec(numBuckets, clusterCols, sortCols.zip(sortDirections)))(node.source) + BucketSpec(numBuckets, clusterCols, sortCols, sortDirections))(node.source) // ALTER TABLE table_name NOT CLUSTERED case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 1a1332b40da30..e6ea7bf1ceeb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -27,15 +27,13 @@ import org.apache.spark.sql.catalyst.expressions.SortDirection * * @param numBuckets number of buckets. * @param bucketColumnNames the names of the columns that used to generate the bucket id. - * @param sortColumns (name, sort direction) of columns that used to sort data in each bucket. + * @param sortColumnNames the names of the columns that used to sort data in each bucket. */ private[sql] case class BucketSpec( numBuckets: Int, bucketColumnNames: Seq[String], - sortColumns: Seq[(String, SortDirection)]) { - - def sortColumnNames: Seq[String] = sortColumns.map { case (name, _) => name } -} + sortColumnNames: Seq[String], + sortDirections: Seq[SortDirection] = Nil) private[sql] object BucketingUtils { // The file name of bucketed data should have 3 parts: 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 1b19423755d7d..b819d394ddb10 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 @@ -170,9 +170,9 @@ class DDLCommandSuite extends PlanTest { val tableIdent = TableIdentifier("table_name", None) val cols = List("dt", "country") val expected1 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, Nil))(sql1) + tableIdent, BucketSpec(10, cols, Nil, Nil))(sql1) val expected2 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, List(("dt", Ascending), ("country", Descending))))(sql2) + tableIdent, BucketSpec(10, cols, cols, List(Ascending, Descending)))(sql2) val expected3 = AlterTableNotClustered(tableIdent)(sql3) val expected4 = AlterTableNotSorted(tableIdent)(sql4) comparePlans(parsed1, expected1) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index a3c1f74813bef..bbbdf8a5c57de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -172,10 +172,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val partitionColumns = getColumnNames("part") val bucketSpec = table.properties.get("spark.sql.sources.schema.numBuckets").map { n => - BucketSpec( - n.toInt, - getColumnNames("bucket"), - getColumnNames("sort").map { n => (n, Ascending) }) + BucketSpec(n.toInt, getColumnNames("bucket"), getColumnNames("sort")) } val options = table.storage.serdeProperties @@ -248,7 +245,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { - val BucketSpec(numBuckets, bucketColumnNames, sortColumns) = bucketSpec.get + val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames, _) = bucketSpec.get tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) tableProperties.put("spark.sql.sources.schema.numBucketCols", @@ -257,7 +254,6 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte tableProperties.put(s"spark.sql.sources.schema.bucketCol.$index", bucketCol) } - val sortColumnNames = sortColumns.map { case (name, _) => name } if (sortColumnNames.nonEmpty) { tableProperties.put("spark.sql.sources.schema.numSortCols", sortColumnNames.length.toString) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 35573f62dc633..04d11be18acf4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -81,7 +81,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // This test verifies parts of the plan. Disable whole stage codegen. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k") - val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec + val BucketSpec(numBuckets, bucketColumnNames, _, _) = bucketSpec // Limit: bucket pruning only works when the bucket column has one and only one column assert(bucketColumnNames.length == 1) val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head) From bd91b0f3c850da796b22c7e575658625d2486057 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 11 Mar 2016 11:47:08 -0800 Subject: [PATCH 12/12] Revert sort directions in bucket spec for now --- .../sql/execution/command/AlterTableCommandParser.scala | 3 ++- .../org/apache/spark/sql/execution/datasources/bucket.scala | 6 +----- .../spark/sql/execution/command/DDLCommandSuite.scala | 5 +++-- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/sources/BucketedReadSuite.scala | 2 +- 5 files changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 56daa83d3fb41..58639275c111b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -174,6 +174,7 @@ object AlterTableCommandParser { } // If sort columns are specified, num buckets should be the third arg. // If sort columns are not specified, num buckets should be the second arg. + // TODO: actually use `sortDirections` once we actually store that in the metastore val (sortCols: Seq[String], sortDirections: Seq[SortDirection], numBuckets: Int) = { b.tail match { case Token("TOK_TABCOLNAME", children) :: numBucketsNode :: Nil => @@ -190,7 +191,7 @@ object AlterTableCommandParser { } AlterTableStorageProperties( tableIdent, - BucketSpec(numBuckets, clusterCols, sortCols, sortDirections))(node.source) + BucketSpec(numBuckets, clusterCols, sortCols))(node.source) // ALTER TABLE table_name NOT CLUSTERED case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index e6ea7bf1ceeb9..6008d73717f77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.execution.datasources -import org.apache.spark.sql.catalyst.expressions.SortDirection - - /** * A container for bucketing information. * Bucketing is a technology for decomposing data sets into more manageable parts, and the number @@ -32,8 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.SortDirection private[sql] case class BucketSpec( numBuckets: Int, bucketColumnNames: Seq[String], - sortColumnNames: Seq[String], - sortDirections: Seq[SortDirection] = Nil) + sortColumnNames: Seq[String]) private[sql] object BucketingUtils { // The file name of bucketed data should have 3 parts: 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 b819d394ddb10..0d632a8a130ed 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 @@ -169,10 +169,11 @@ class DDLCommandSuite extends PlanTest { val parsed4 = parser.parsePlan(sql4) val tableIdent = TableIdentifier("table_name", None) val cols = List("dt", "country") + // TODO: also test the sort directions once we keep track of that val expected1 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, Nil, Nil))(sql1) + tableIdent, BucketSpec(10, cols, Nil))(sql1) val expected2 = AlterTableStorageProperties( - tableIdent, BucketSpec(10, cols, cols, List(Ascending, Descending)))(sql2) + tableIdent, BucketSpec(10, cols, cols))(sql2) val expected3 = AlterTableNotClustered(tableIdent)(sql3) val expected4 = AlterTableNotSorted(tableIdent)(sql4) comparePlans(parsed1, expected1) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bbbdf8a5c57de..8f6cd66f1f681 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -245,7 +245,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { - val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames, _) = bucketSpec.get + val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) tableProperties.put("spark.sql.sources.schema.numBucketCols", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 04d11be18acf4..35573f62dc633 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -81,7 +81,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // This test verifies parts of the plan. Disable whole stage codegen. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k") - val BucketSpec(numBuckets, bucketColumnNames, _, _) = bucketSpec + val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec // Limit: bucket pruning only works when the bucket column has one and only one column assert(bucketColumnNames.length == 1) val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head)