From 77252af5b83d026030c5dd1270ecd4d0788af779 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 3 Feb 2016 10:00:02 +0000 Subject: [PATCH 01/13] init import: create database and create function. --- .../spark/sql/catalyst/analysis/Catalog.scala | 6 +- .../apache/spark/sql/execution/SparkQl.scala | 55 +++++++++++++++++++ .../apache/spark/sql/execution/commands.scala | 29 ++++++++++ .../spark/sql/execution/SparkQlSuite.scala | 36 ++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 +- .../org/apache/spark/sql/hive/HiveQl.scala | 2 - 6 files changed, 130 insertions(+), 4 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index f2f9ec59417ef..620e0808b4043 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} @@ -50,6 +50,10 @@ trait Catalog { throw new UnsupportedOperationException } + def runNativeCommand(sql: String): Seq[Row] = { + throw new UnsupportedOperationException + } + /** * Returns tuples of (tableName, isTemporary) for all tables in the given database. * isTemporary is a Boolean value indicates if a table is a temporary or not. 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 4174e27e9c8b7..ecb6aec653288 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 @@ -62,6 +62,61 @@ 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 { + case Token("TOK_DBPROPLIST", props) => + props.map { + case Token("TOK_TABLEPROPERTY", keysAndValue) => + val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + } + } + }.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_CREATETABLEUSING", createTableArgs) => val Seq( temp, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index c6adb583f931b..956aa527eaf93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -418,3 +418,32 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { override val output: Seq[Attribute] = Seq.empty } + +case class CreateDataBase( + databaseName: String, + allowExisting: Boolean, + path: Option[String], + comment: Option[String], + props: Map[String, String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class CreateFunction( + functionName: String, + asName: String, + resourcesMap: Map[String, String], + isTemp: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala new file mode 100644 index 0000000000000..067a184c0addd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala @@ -0,0 +1,36 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.plans.PlanTest + +class SparkQlSuite extends PlanTest { + val parser = new SparkQl() + + test("create database") { + parser.parsePlan("CREATE DATABASE IF NOT EXISTS database_name " + + "COMMENT 'database_comment' LOCATION '/home/user/db' " + + "WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + } + + test("create function") { + parser.parsePlan("CREATE TEMPORARY FUNCTION helloworld as " + + "'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', " + + "FILE 'path/to/file'") + } +} 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 61d0d6759ff72..4fc2e0307cf09 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 @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} +import org.apache.spark.sql.{AnalysisException, SaveMode, Row, SQLContext} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ @@ -715,6 +715,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte override def setCurrentDatabase(databaseName: String): Unit = { client.setCurrentDatabase(databaseName) } + + override def runNativeCommand(sql: String): Seq[Row] = { + hive.runSqlHive(sql).map(Row(_)) + } } /** 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 752c037a842a8..f9df49425b4f7 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 @@ -107,8 +107,6 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEMACRO", "TOK_CREATEROLE", From 2b38d1197bfaeef2f56691543ece01051e9cbb3e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 3 Feb 2016 13:43:25 +0000 Subject: [PATCH 02/13] Fix import order. --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4fc2e0307cf09..8e06f6cf981d8 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 @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, Row, SQLContext} +import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ From 127a68416ffdfda83a743b00274e780983481b5d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 5 Feb 2016 10:51:33 +0800 Subject: [PATCH 03/13] alter table command. --- .../apache/spark/sql/execution/SparkQl.scala | 400 ++++++++++++++++++ .../apache/spark/sql/execution/commands.scala | 283 ++++++++++++- .../sql/execution/datasources/bucket.scala | 4 +- .../spark/sql/execution/SparkQlSuite.scala | 113 +++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 13 - 6 files changed, 799 insertions(+), 16 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 ecb6aec653288..1835bb78cb799 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 @@ -16,9 +16,12 @@ */ package org.apache.spark.sql.execution +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.plans.logical @@ -117,6 +120,403 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly }.toMap CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source) + case Token("TOK_ALTERTABLE", alterTableArgs) => + val tabName = getClause("TOK_TABNAME", alterTableArgs) + val rename = getClauseOption("TOK_ALTERTABLE_RENAME", alterTableArgs) + val setTableProps = getClauseOption("TOK_ALTERTABLE_PROPERTIES", alterTableArgs) + val dropTableProps = getClauseOption("TOK_ALTERTABLE_DROPPROPERTIES", alterTableArgs) + val serde = getClauseOption("TOK_ALTERTABLE_SERIALIZER", alterTableArgs) + val serdeProps = getClauseOption("TOK_ALTERTABLE_SERDEPROPERTIES", alterTableArgs) + val partitionSpec = getClauseOption("TOK_PARTSPEC", alterTableArgs) + val bucketSpec = getClauseOption("TOK_ALTERTABLE_CLUSTER_SORT", alterTableArgs) + val bucketNum = getClauseOption("TOK_ALTERTABLE_BUCKETS", alterTableArgs) + val tableSkewed = getClauseOption("TOK_ALTERTABLE_SKEWED", alterTableArgs) + val tableSkewedLocation = getClauseOption("TOK_ALTERTABLE_SKEWED_LOCATION", alterTableArgs) + val addParts = getClauseOption("TOK_ALTERTABLE_ADDPARTS", alterTableArgs) + val renamePart = getClauseOption("TOK_ALTERTABLE_RENAMEPART", alterTableArgs) + val exchangePart = getClauseOption("TOK_ALTERTABLE_EXCHANGEPARTITION", alterTableArgs) + val dropParts = getClauseOption("TOK_ALTERTABLE_DROPPARTS", alterTableArgs) + val archivePart = getClauseOption("TOK_ALTERTABLE_ARCHIVE", alterTableArgs) + val unarchivePart = getClauseOption("TOK_ALTERTABLE_UNARCHIVE", alterTableArgs) + val setFileFormat = getClauseOption("TOK_ALTERTABLE_FILEFORMAT", alterTableArgs) + val setLocation = getClauseOption("TOK_ALTERTABLE_LOCATION", alterTableArgs) + val touch = getClauseOption("TOK_ALTERTABLE_TOUCH", alterTableArgs) + val compact = getClauseOption("TOK_ALTERTABLE_COMPACT", alterTableArgs) + val merge = getClauseOption("TOK_ALTERTABLE_MERGEFILES", alterTableArgs) + val renameCol = getClauseOption("TOK_ALTERTABLE_RENAMECOL", alterTableArgs) + val addCol = getClauseOption("TOK_ALTERTABLE_ADDCOLS", alterTableArgs) + val replaceCol = getClauseOption("TOK_ALTERTABLE_REPLACECOLS", alterTableArgs) + + val tableIdent: TableIdentifier = extractTableIdent(tabName) + + 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) => + (cleanIdentifier(ident.text), Some(cleanIdentifier(constant.text))) + case Token("TOK_PARTVAL", ident :: Nil) => + (cleanIdentifier(ident.text), None) + }.toMap + Some(spec) + case _ => None + } + } + + // Partition Spec + val partition: Option[Map[String, Option[String]]] = + partitionSpec.flatMap(parsePartitionSpec) + + 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 = unquoteString(key.text) + (k, None) + case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = unquoteString(key.text) + val v = unquoteString(value.text) + (k, Some(v)) + } + }.toMap + } + + if (rename.isDefined) { + // Rename table + val renamedTable = rename.map { + case Token("TOK_ALTERTABLE_RENAME", renameArgs) => + getClause("TOK_TABNAME", renameArgs) + } + val renamedTableIdent: Option[TableIdentifier] = renamedTable.map(extractTableIdent) + + AlterTableRename(tableIdent, renamedTableIdent)(node.source) + + } else if (setTableProps.isDefined || dropTableProps.isDefined) { + // Alter table properties + val setTableProperties = setTableProps.map { + case Token("TOK_ALTERTABLE_PROPERTIES", args :: Nil) => extractTableProps(args) + } + val dropTableProperties = dropTableProps.map { + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) => + extractTableProps(args.head) + } + val allowExisting = dropTableProps.flatMap { + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) => + getClauseOption("TOK_IFEXISTS", args) + } + + AlterTableProperties( + tableIdent, + setTableProperties, + dropTableProperties, + allowExisting.isDefined)(node.source) + } else if (serde.isDefined || serdeProps.isDefined) { + val serdeArgs: Option[Seq[ASTNode]] = serde.map { + case Token("TOK_ALTERTABLE_SERIALIZER", args) => + args + } + + val serdeClassName = serdeArgs.map(_.head).map { + case Token(className, Nil) => className + } + + val serdeProperties: Option[Map[String, Option[String]]] = Option( + // SET SERDE serde_classname WITH SERDEPROPERTIES + serdeArgs.map(_.tail).map { props => + if (props.isEmpty) { + null + } else { + extractTableProps(props.head) + } + }.getOrElse { + // SET SERDEPROPERTIES + serdeProps.map { + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) => + extractTableProps(args.head) + }.getOrElse(null) + } + ) + + AlterTableSerDeProperties( + tableIdent, + serdeClassName, + serdeProperties, + partition)(node.source) + } else if (bucketSpec.isDefined) { + val (buckets, noClustered, noSorted) = bucketSpec.map { + case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) => + clusterAndSoryByArgs match { + case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) => + val bucketCols = bucketArgs.head.children.map(_.text) + + val (sortCols, sortDirections, numBuckets) = { + if (bucketArgs(1).text == "TOK_TABCOLNAME") { + val cols = bucketArgs(1).children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => + (colName, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => + (colName, Descending) + } + (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt) + } else { + (Nil, Nil, bucketArgs(1).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) + } + }.getOrElse((None, false, false)) // should not reach here + + AlterTableStoreProperties( + tableIdent, + buckets, + noClustered, + noSorted)(node.source) + } else if (bucketNum.isDefined) { + val num = bucketNum.get.children.head.text.toInt + val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) + AlterTableStoreProperties( + tableIdent, + buckets, + false, + false)(node.source) + } else if (tableSkewed.isDefined) { + // Alter Table not skewed + // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed. + val notSkewed = if (tableSkewed.get.children.size == 0) { + true + } else { + false + } + + val (notStoredAsDirs, skewedArgs) = tableSkewed.map { + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => + // Alter Table not stored as directories + (true, None) + case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) => + val (cols, values, storedAsDirs) = skewedArgs match { + case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => + val cols = skewedCols.children.map(_.text) + val values = skewedValues match { + case Token("TOK_TABCOLVALUE", values) => Seq(values.map(_.text)) + case Token("TOK_TABCOLVALUE_PAIR", pairs) => + pairs.map { + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => vals.map(_.text) + } + } + } + + val storedAsDirs = stored match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false + } + + (cols, values, storedAsDirs) + } + (false, Some((cols, values, storedAsDirs))) + }.get + + if (skewedArgs.isDefined) { + AlterTableSkewed( + tableIdent, + skewedArgs.get._1, + skewedArgs.get._2, + skewedArgs.get._3, notSkewed, notStoredAsDirs)(node.source) + } else { + AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source) + } + } else if (tableSkewedLocation.isDefined) { + val skewedMaps = tableSkewedLocation.get.children(0) match { + case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) => + locationList match { + case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) => + locationMaps.map { + case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => + val k = key match { + case Token(const, Nil) => Seq(const) + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => vals.map(_.text) + } + } + (k, value.text) + }.toMap + } + } + AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) + } else if (addParts.isDefined) { + val allowExisting = getClauseOption("TOK_IFNOTEXISTS", addParts.get.children) + val parts = if (allowExisting.isDefined) { + addParts.get.children.tail + } else { + addParts.get.children + } + + 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.isDefined)(node.source) + } else if (renamePart.isDefined) { + val newPartition = parsePartitionSpec(renamePart.get.children(0)) + AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source) + } else if (exchangePart.isDefined) { + val Seq(Some(partSpec), Some(fromTable)) = + getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), exchangePart.get.children) + val partition = parsePartitionSpec(partSpec).get + val fromTableIdent = extractTableIdent(fromTable) + AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) + } else if (dropParts.isDefined) { + val parts = dropParts.get.children.collect { + case Token("TOK_PARTSPEC", partitions) => + partitions.map { + case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => + (cleanIdentifier(ident.text), op.text, cleanIdentifier(constant.text)) + } + } + + val allowExisting = getClauseOption("TOK_IFEXISTS", dropParts.get.children) + val purge = getClauseOption("PURGE", dropParts.get.children) + + val replication = getClauseOption("TOK_REPLICATION", dropParts.get.children).map { + case Token("TOK_REPLICATION", replId :: metadata :: Nil) => + (replId.text, true) + case Token("TOK_REPLICATION", replId :: Nil) => + (replId.text, false) + } + + AlterTableDropPartition( + tableIdent, + parts, + allowExisting.isDefined, + purge.isDefined, + replication)(node.source) + } else if (archivePart.isDefined) { + val partition = parsePartitionSpec(archivePart.get.children(0)).get + AlterTableArchivePartition(tableIdent, partition)(node.source) + } else if (unarchivePart.isDefined) { + val partition = parsePartitionSpec(unarchivePart.get.children(0)).get + AlterTableUnarchivePartition(tableIdent, partition)(node.source) + } else if (setFileFormat.isDefined) { + val Seq(fileFormat, genericFormat) = + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), + setFileFormat.get.children) + val fFormat = fileFormat.map(_.children.map(_.text)) + val gFormat = genericFormat.map(_.children(0).text) + AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + } else if (setLocation.isDefined) { + val loc = cleanIdentifier(setLocation.get.children(0).text) + AlterTableSetLocation(tableIdent, partition, loc)(node.source) + } else if (touch.isDefined) { + val part = getClauseOption("TOK_PARTSPEC", touch.get.children).flatMap(parsePartitionSpec) + AlterTableTouch(tableIdent, part)(node.source) + } else if (compact.isDefined) { + val compactType = cleanIdentifier(compact.get.children(0).text) + AlterTableCompact(tableIdent, partition, compactType)(node.source) + } else if (merge.isDefined) { + AlterTableMerge(tableIdent, partition)(node.source) + } + else if (renameCol.isDefined) { + val oldName = renameCol.get.children(0).text + val newName = renameCol.get.children(1).text + val dataType = nodeToDataType(renameCol.get.children(2)) + val afterPos = + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", renameCol.get.children) + val afterPosCol = afterPos.map { ap => + ap.children match { + case Token(col, Nil) :: Nil => col + case _ => null + } + } + + val restrict = getClauseOption("TOK_RESTRICT", renameCol.get.children) + val cascade = getClauseOption("TOK_CASCADE", renameCol.get.children) + + val comment = if (renameCol.get.children.size > 3) { + renameCol.get.children(3) match { + case Token(commentStr, Nil) + if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && + commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => Some(commentStr) + case _ => + None + } + } else { + None + } + + AlterTableChangeCol( + tableIdent, + partition, + oldName, + newName, + dataType, + comment, + afterPos.isDefined, + afterPosCol, + restrict.isDefined, + cascade.isDefined)(node.source) + } else if (addCol.isDefined || replaceCol.isDefined) { + val thisNode = if (addCol.isDefined) { + addCol.get + } else { + replaceCol.get + } + + val tableCols = getClause("TOK_TABCOLLIST", thisNode.children) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + } + + val restrict = getClauseOption("TOK_RESTRICT", thisNode.children) + val cascade = getClauseOption("TOK_CASCADE", thisNode.children) + + if (addCol.isDefined) { + AlterTableAddCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + } else { + AlterTableReplaceCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + } + } else { + nodeToDescribeFallback(node) + } + case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( temp, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 956aa527eaf93..f670f880c21b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -22,7 +22,8 @@ import java.util.NoSuchElementException import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical @@ -447,3 +448,283 @@ case class CreateFunction( override val output: Seq[Attribute] = Seq(AttributeReference("result", StringType, nullable = false)()) } + +case class AlterTableRename( + tableName: TableIdentifier, + renameTableName: Option[TableIdentifier])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableProperties( + tableName: TableIdentifier, + setProperties: Option[Map[String, Option[String]]], + dropProperties: Option[Map[String, Option[String]]], + allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSerDeProperties( + tableName: TableIdentifier, + serdeClassName: Option[String], + serdeProperties: Option[Map[String, Option[String]]], + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableStoreProperties( + tableName: TableIdentifier, + buckets: Option[BucketSpec], + noClustered: Boolean, + noSorted: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSkewed( + tableName: TableIdentifier, + skewedCols: Seq[String], + skewedValues: Seq[Seq[String]], + storedAsDirs: Boolean, + notSkewed: Boolean, + notStoredAsDirs: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSkewedLocation( + tableName: TableIdentifier, + skewedMap: Map[Seq[String], String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableAddPartition( + tableName: TableIdentifier, + partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], + allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableRenamePartition( + tableName: TableIdentifier, + oldPartition: Map[String, Option[String]], + newPartition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableExchangePartition( + tableName: TableIdentifier, + fromTableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableDropPartition( + tableName: TableIdentifier, + partitions: Seq[Seq[(String, String, String)]], + allowExisting: Boolean, + purge: Boolean, + replication: Option[(String, Boolean)])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableArchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableUnarchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSetFileFormat( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + fileFormat: Option[Seq[String]], + genericFormat: Option[String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSetLocation( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + location: String)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableTouch( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableCompact( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + compactType: String)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableMerge( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +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 RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableAddCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableReplaceCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} 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/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala index 067a184c0addd..ef931e41265c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala @@ -33,4 +33,117 @@ class SparkQlSuite extends PlanTest { "'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', " + "FILE 'path/to/file'") } + + test("alter table") { + // Rename table + parser.parsePlan("ALTER TABLE table_name RENAME TO new_table_name") + + // Alter table properties + parser.parsePlan("ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')") + parser.parsePlan("ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')") + parser.parsePlan("ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')") + + // Alter table SerDe properties + parser.parsePlan("ALTER TABLE table_name SET SERDE 'org.apache.class'") + parser.parsePlan("ALTER TABLE table_name SET SERDE 'org.apache.class' " + + "WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')") + parser.parsePlan("ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', " + + "'field.delim' = ',')") + parser.parsePlan("ALTER TABLE table_name PARTITION (test, dt='2008-08-08', " + + "country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', " + + "'field.delim' = ',')") + parser.parsePlan("ALTER TABLE table_name PARTITION (test, dt='2008-08-08', " + + "country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')") + + // Alter Table storage properties + parser.parsePlan("ALTER TABLE table_name CLUSTERED BY (dt, country) " + + "INTO 10 BUCKETS") + parser.parsePlan("ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + + "(dt, country DESC) INTO 10 BUCKETS") + parser.parsePlan("ALTER TABLE table_name INTO 20 BUCKETS") + parser.parsePlan("ALTER TABLE table_name NOT CLUSTERED") + parser.parsePlan("ALTER TABLE table_name NOT SORTED") + + // Alter Table skewed + parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + + "(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES") + parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + + "('2008-08-08', 'us') STORED AS DIRECTORIES") + parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + + "(('2008-08-08', 'us'), ('2009-09-09', 'uk'))") + + // Alter Table skewed location + parser.parsePlan("ALTER TABLE table_name SET SKEWED LOCATION " + + "('123'='location1', 'test'='location2')") + parser.parsePlan("ALTER TABLE table_name SET SKEWED LOCATION " + + "(('2008-08-08', 'us')='location1', 'test'='location2')") + + // Alter Table add partition + parser.parsePlan("ALTER TABLE table_name ADD IF NOT EXISTS PARTITION " + + "(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION " + + "(dt='2009-09-09', country='uk')") + + // Alter Table rename partition + parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "RENAME TO PARTITION (dt='2008-09-09', country='uk')") + + // Alter Table exchange partition + parser.parsePlan("ALTER TABLE table_name_1 EXCHANGE PARTITION " + + "(dt='2008-08-08', country='us') WITH TABLE table_name_2") + + // Alter Table drop partitions + parser.parsePlan("ALTER TABLE table_name DROP IF EXISTS PARTITION " + + "(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk')") + parser.parsePlan("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 archive partition + parser.parsePlan("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") + + // Alter Table unarchive partition + parser.parsePlan("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") + + // Alter Table set file format + parser.parsePlan("ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + + "OUTPUTFORMAT 'test' SERDE 'test' INPUTDRIVER 'test' OUTPUTDRIVER 'test'") + parser.parsePlan("ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + + "OUTPUTFORMAT 'test' SERDE 'test'") + parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET FILEFORMAT PARQUET") + + // Alter Table set location + parser.parsePlan("ALTER TABLE table_name SET LOCATION 'new location'") + parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET LOCATION 'new location'") + + // Alter Table touch + parser.parsePlan("ALTER TABLE table_name TOUCH") + parser.parsePlan("ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')") + + // Alter Table compact + parser.parsePlan("ALTER TABLE table_name COMPACT 'compaction_type'") + parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "COMPACT 'MAJOR'") + + // Alter Table concatenate + parser.parsePlan("ALTER TABLE table_name CONCATENATE") + parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE") + + // Alter Table change column name/type/position/comment + parser.parsePlan("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") + + parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + + "COMMENT 'col_comment' FIRST CASCADE") + parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + + "COMMENT 'col_comment' AFTER column_name RESTRICT") + + // Alter Table add/replace columns + parser.parsePlan("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") + parser.parsePlan("ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT " + + "COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT") + } } 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 8e06f6cf981d8..764b4a7ad13b3 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 @@ -246,7 +246,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 f9df49425b4f7..19aa65213ae90 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 @@ -86,21 +86,8 @@ 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", From f9c13976132febea2a6d8443aa5d144a2a107648 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Feb 2016 13:19:51 +0000 Subject: [PATCH 04/13] Fix it. --- .../scala/org/apache/spark/sql/sources/BucketedReadSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 9ba645626fe72..ef070a0ec9ed7 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 @@ -75,7 +75,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet originalDataFrame: DataFrame): Unit = { 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 eb1fab78319432184e3eed34721adae6da230188 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Feb 2016 14:08:20 +0000 Subject: [PATCH 05/13] Fix style. --- .../scala/org/apache/spark/sql/execution/commands.scala | 2 +- .../org/apache/spark/sql/execution/SparkQlSuite.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index f670f880c21b7..3a4d26145cfc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -22,12 +22,12 @@ import java.util.NoSuchElementException import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext} -import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala index ef931e41265c6..2af2ff68dfa5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala @@ -72,7 +72,7 @@ class SparkQlSuite extends PlanTest { "('2008-08-08', 'us') STORED AS DIRECTORIES") parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + "(('2008-08-08', 'us'), ('2009-09-09', 'uk'))") - + // Alter Table skewed location parser.parsePlan("ALTER TABLE table_name SET SKEWED LOCATION " + "('123'='location1', 'test'='location2')") @@ -94,10 +94,10 @@ class SparkQlSuite extends PlanTest { // Alter Table drop partitions parser.parsePlan("ALTER TABLE table_name DROP IF EXISTS PARTITION " + - "(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk')") + "(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk')") parser.parsePlan("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')") + "PURGE FOR METADATA REPLICATION ('test')") // Alter Table archive partition parser.parsePlan("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") @@ -133,7 +133,7 @@ class SparkQlSuite extends PlanTest { // Alter Table change column name/type/position/comment parser.parsePlan("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") - + parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + "COMMENT 'col_comment' FIRST CASCADE") parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + From 3db2e1d867b5750eb157c751a303c542bdad110e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 9 Feb 2016 09:13:42 +0000 Subject: [PATCH 06/13] List TOK_ALTERTABLE as not explanable command. --- .../main/scala/org/apache/spark/sql/execution/SparkQl.scala | 5 +++-- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 3 ++- 2 files changed, 5 insertions(+), 3 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 1835bb78cb799..467c61518e1d8 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 @@ -30,7 +30,8 @@ import org.apache.spark.sql.types.StructType private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) { /** 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 override def nodeToPlan(node: ASTNode): LogicalPlan = { node match { @@ -55,7 +56,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs) ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined) - case Token("TOK_EXPLAIN", explainArgs) => + case Token("TOK_EXPLAIN", explainArgs) if "TOK_QUERY" == explainArgs.head.text => // Ignore FORMATTED if present. val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) 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 19aa65213ae90..070bf35632c81 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 @@ -147,7 +147,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 /** From 2e78f52c0e8810afef09198dfe399df43cd2cdc4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Feb 2016 07:11:37 +0000 Subject: [PATCH 07/13] Add execution.commands package. --- .../org/apache/spark/sql/DataFrame.scala | 3 +- .../org/apache/spark/sql/SQLContext.scala | 1 + .../apache/spark/sql/execution/SparkQl.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 3 +- .../apache/spark/sql/execution/commands.scala | 655 ----------------- .../sql/execution/commands/commands.scala | 686 ++++++++++++++++++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 1 + .../spark/sql/hive/HiveStrategies.scala | 3 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 2 +- 11 files changed, 698 insertions(+), 661 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 76c09a285dc40..8140a0c11eb16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{ExplainCommand, FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution} +import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution} +import org.apache.spark.sql.execution.commands.ExplainCommand import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython 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 c7d1096a1384c..0de4a7375ea37 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 @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.commands._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.sources.BaseRelation 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 467c61518e1d8..1c1a8bdd52efd 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.execution.commands._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 042c99db4dcff..cd7da1c2a33a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -25,8 +25,9 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} +import org.apache.spark.sql.execution.commands.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.execution.commands._ import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTempTableUsing, DescribeCommand => LogicalDescribeCommand, _} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 3a4d26145cfc0..06684422f6f6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -73,658 +73,3 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan override def argString: String = cmd.toString } - - -case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging { - - private def keyValueOutput: Seq[Attribute] = { - val schema = StructType( - StructField("key", StringType, false) :: - StructField("value", StringType, false) :: Nil) - schema.toAttributes - } - - private val (_output, runFunc): (Seq[Attribute], SQLContext => Seq[Row]) = kv match { - // Configures the deprecated "mapred.reduce.tasks" property. - case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") - if (value.toInt < 1) { - val msg = - s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " + - "determining the number of reducers is not supported." - throw new IllegalArgumentException(msg) - } else { - sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS.key, value) - Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, value)) - } - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.EXTERNAL_SORT, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.EXTERNAL_SORT} is deprecated and will be ignored. " + - s"External sort will continue to be used.") - Seq(Row(SQLConf.Deprecated.EXTERNAL_SORT, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.USE_SQL_AGGREGATE2, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} is deprecated and " + - s"will be ignored. ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} will " + - s"continue to be true.") - Seq(Row(SQLConf.Deprecated.USE_SQL_AGGREGATE2, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.TUNGSTEN_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.TUNGSTEN_ENABLED} is deprecated and " + - s"will be ignored. Tungsten will continue to be used.") - Seq(Row(SQLConf.Deprecated.TUNGSTEN_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.CODEGEN_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.CODEGEN_ENABLED} is deprecated and " + - s"will be ignored. Codegen will continue to be used.") - Seq(Row(SQLConf.Deprecated.CODEGEN_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.UNSAFE_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.UNSAFE_ENABLED} is deprecated and " + - s"will be ignored. Unsafe mode will continue to be used.") - Seq(Row(SQLConf.Deprecated.UNSAFE_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.SORTMERGE_JOIN, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.SORTMERGE_JOIN} is deprecated and " + - s"will be ignored. Sort merge join will continue to be used.") - Seq(Row(SQLConf.Deprecated.SORTMERGE_JOIN, "true")) - } - (keyValueOutput, runFunc) - - // Configures a single property. - case Some((key, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - sqlContext.setConf(key, value) - Seq(Row(key, value)) - } - (keyValueOutput, runFunc) - - // (In Hive, "SET" returns all changed properties while "SET -v" returns all properties.) - // Queries all key-value pairs that are set in the SQLConf of the sqlContext. - case None => - val runFunc = (sqlContext: SQLContext) => { - sqlContext.getAllConfs.map { case (k, v) => Row(k, v) }.toSeq - } - (keyValueOutput, runFunc) - - // Queries all properties along with their default values and docs that are defined in the - // SQLConf of the sqlContext. - case Some(("-v", None)) => - val runFunc = (sqlContext: SQLContext) => { - sqlContext.conf.getAllDefinedConfs.map { case (key, defaultValue, doc) => - Row(key, defaultValue, doc) - } - } - val schema = StructType( - StructField("key", StringType, false) :: - StructField("default", StringType, false) :: - StructField("meaning", StringType, false) :: Nil) - (schema.toAttributes, runFunc) - - // Queries the deprecated "mapred.reduce.tasks" property. - case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") - Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, sqlContext.conf.numShufflePartitions.toString)) - } - (keyValueOutput, runFunc) - - // Queries a single property. - case Some((key, None)) => - val runFunc = (sqlContext: SQLContext) => { - val value = - try sqlContext.getConf(key) catch { - case _: NoSuchElementException => "" - } - Seq(Row(key, value)) - } - (keyValueOutput, runFunc) - } - - override val output: Seq[Attribute] = _output - - override def run(sqlContext: SQLContext): Seq[Row] = runFunc(sqlContext) - -} - -/** - * An explain command for users to see how a command will be executed. - * - * Note that this command takes in a logical plan, runs the optimizer on the logical plan - * (but do NOT actually execute it). - */ -case class ExplainCommand( - logicalPlan: LogicalPlan, - override val output: Seq[Attribute] = - Seq(AttributeReference("plan", StringType, nullable = true)()), - extended: Boolean = false) - extends RunnableCommand { - - // Run through the optimizer to generate the physical plan. - override def run(sqlContext: SQLContext): Seq[Row] = try { - // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. - val queryExecution = sqlContext.executePlan(logicalPlan) - val outputString = if (extended) queryExecution.toString else queryExecution.simpleString - - outputString.split("\n").map(Row(_)) - } catch { case cause: TreeNodeException[_] => - ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) - } -} - - -case class CacheTableCommand( - tableName: String, - plan: Option[LogicalPlan], - isLazy: Boolean) - extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - plan.foreach { logicalPlan => - sqlContext.registerDataFrameAsTable(DataFrame(sqlContext, logicalPlan), tableName) - } - sqlContext.cacheTable(tableName) - - if (!isLazy) { - // Performs eager caching - sqlContext.table(tableName).count() - } - - Seq.empty[Row] - } - - override def output: Seq[Attribute] = Seq.empty -} - - -case class UncacheTableCommand(tableName: String) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.table(tableName).unpersist(blocking = false) - Seq.empty[Row] - } - - override def output: Seq[Attribute] = Seq.empty -} - -/** - * Clear all cached data from the in-memory cache. - */ -case object ClearCacheCommand extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.clearCache() - Seq.empty[Row] - } - - override def output: Seq[Attribute] = Seq.empty -} - - -case class DescribeCommand( - child: SparkPlan, - override val output: Seq[Attribute], - isExtended: Boolean) - extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - child.schema.fields.map { field => - val cmtKey = "comment" - val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else "" - Row(field.name, field.dataType.simpleString, comment) - } - } -} - -/** - * A command for users to get tables in the given database. - * If a databaseName is not given, the current database will be used. - * The syntax of using this command in SQL is: - * {{{ - * SHOW TABLES [IN databaseName] - * }}} - */ -case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand { - - // The result of SHOW TABLES has two columns, tableName and isTemporary. - override val output: Seq[Attribute] = { - val schema = StructType( - StructField("tableName", StringType, false) :: - StructField("isTemporary", BooleanType, false) :: Nil) - - schema.toAttributes - } - - override def run(sqlContext: SQLContext): Seq[Row] = { - // Since we need to return a Seq of rows, we will call getTables directly - // instead of calling tables in sqlContext. - val rows = sqlContext.catalog.getTables(databaseName).map { - case (tableName, isTemporary) => Row(tableName, isTemporary) - } - - rows - } -} - -/** - * A command for users to list all of the registered functions. - * The syntax of using this command in SQL is: - * {{{ - * SHOW FUNCTIONS - * }}} - * TODO currently we are simply ignore the db - */ -case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand { - override val output: Seq[Attribute] = { - val schema = StructType( - StructField("function", StringType, nullable = false) :: Nil) - - schema.toAttributes - } - - override def run(sqlContext: SQLContext): Seq[Row] = pattern match { - case Some(p) => - try { - val regex = java.util.regex.Pattern.compile(p) - sqlContext.functionRegistry.listFunction().filter(regex.matcher(_).matches()).map(Row(_)) - } catch { - // probably will failed in the regex that user provided, then returns empty row. - case _: Throwable => Seq.empty[Row] - } - case None => - sqlContext.functionRegistry.listFunction().map(Row(_)) - } -} - -/** - * A command for users to get the usage of a registered function. - * The syntax of using this command in SQL is - * {{{ - * DESCRIBE FUNCTION [EXTENDED] upper; - * }}} - */ -case class DescribeFunction( - functionName: String, - isExtended: Boolean) extends RunnableCommand { - - override val output: Seq[Attribute] = { - val schema = StructType( - StructField("function_desc", StringType, nullable = false) :: Nil) - - schema.toAttributes - } - - private def replaceFunctionName(usage: String, functionName: String): String = { - if (usage == null) { - "To be added." - } else { - usage.replaceAll("_FUNC_", functionName) - } - } - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.functionRegistry.lookupFunction(functionName) match { - case Some(info) => - val result = - Row(s"Function: ${info.getName}") :: - Row(s"Class: ${info.getClassName}") :: - Row(s"Usage: ${replaceFunctionName(info.getUsage(), info.getName)}") :: Nil - - if (isExtended) { - result :+ Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, info.getName)}") - } else { - result - } - - case None => Seq(Row(s"Function: $functionName not found.")) - } - } -} - -case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.setCurrentDatabase(databaseName) - Seq.empty[Row] - } - - override val output: Seq[Attribute] = Seq.empty -} - -case class CreateDataBase( - databaseName: String, - allowExisting: Boolean, - path: Option[String], - comment: Option[String], - props: Map[String, String])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class CreateFunction( - functionName: String, - asName: String, - resourcesMap: Map[String, String], - isTemp: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableRename( - tableName: TableIdentifier, - renameTableName: Option[TableIdentifier])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableProperties( - tableName: TableIdentifier, - setProperties: Option[Map[String, Option[String]]], - dropProperties: Option[Map[String, Option[String]]], - allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableSerDeProperties( - tableName: TableIdentifier, - serdeClassName: Option[String], - serdeProperties: Option[Map[String, Option[String]]], - partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableStoreProperties( - tableName: TableIdentifier, - buckets: Option[BucketSpec], - noClustered: Boolean, - noSorted: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableSkewed( - tableName: TableIdentifier, - skewedCols: Seq[String], - skewedValues: Seq[Seq[String]], - storedAsDirs: Boolean, - notSkewed: Boolean, - notStoredAsDirs: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableSkewedLocation( - tableName: TableIdentifier, - skewedMap: Map[Seq[String], String])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableAddPartition( - tableName: TableIdentifier, - partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], - allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableRenamePartition( - tableName: TableIdentifier, - oldPartition: Map[String, Option[String]], - newPartition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableExchangePartition( - tableName: TableIdentifier, - fromTableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableDropPartition( - tableName: TableIdentifier, - partitions: Seq[Seq[(String, String, String)]], - allowExisting: Boolean, - purge: Boolean, - replication: Option[(String, Boolean)])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableArchivePartition( - tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableUnarchivePartition( - tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableSetFileFormat( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - fileFormat: Option[Seq[String]], - genericFormat: Option[String])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableSetLocation( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - location: String)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableTouch( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableCompact( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - compactType: String)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableMerge( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -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 RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableAddCol( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -case class AlterTableReplaceCol( - tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala new file mode 100644 index 0000000000000..1158443578f26 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala @@ -0,0 +1,686 @@ +/* + * 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.commands + +import java.util.NoSuchElementException + +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + +case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging { + + private def keyValueOutput: Seq[Attribute] = { + val schema = StructType( + StructField("key", StringType, false) :: + StructField("value", StringType, false) :: Nil) + schema.toAttributes + } + + private val (_output, runFunc): (Seq[Attribute], SQLContext => Seq[Row]) = kv match { + // Configures the deprecated "mapred.reduce.tasks" property. + case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") + if (value.toInt < 1) { + val msg = + s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " + + "determining the number of reducers is not supported." + throw new IllegalArgumentException(msg) + } else { + sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS.key, value) + Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, value)) + } + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.EXTERNAL_SORT, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.EXTERNAL_SORT} is deprecated and will be ignored. " + + s"External sort will continue to be used.") + Seq(Row(SQLConf.Deprecated.EXTERNAL_SORT, "true")) + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.USE_SQL_AGGREGATE2, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} is deprecated and " + + s"will be ignored. ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} will " + + s"continue to be true.") + Seq(Row(SQLConf.Deprecated.USE_SQL_AGGREGATE2, "true")) + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.TUNGSTEN_ENABLED, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.TUNGSTEN_ENABLED} is deprecated and " + + s"will be ignored. Tungsten will continue to be used.") + Seq(Row(SQLConf.Deprecated.TUNGSTEN_ENABLED, "true")) + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.CODEGEN_ENABLED, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.CODEGEN_ENABLED} is deprecated and " + + s"will be ignored. Codegen will continue to be used.") + Seq(Row(SQLConf.Deprecated.CODEGEN_ENABLED, "true")) + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.UNSAFE_ENABLED, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.UNSAFE_ENABLED} is deprecated and " + + s"will be ignored. Unsafe mode will continue to be used.") + Seq(Row(SQLConf.Deprecated.UNSAFE_ENABLED, "true")) + } + (keyValueOutput, runFunc) + + case Some((SQLConf.Deprecated.SORTMERGE_JOIN, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.SORTMERGE_JOIN} is deprecated and " + + s"will be ignored. Sort merge join will continue to be used.") + Seq(Row(SQLConf.Deprecated.SORTMERGE_JOIN, "true")) + } + (keyValueOutput, runFunc) + + // Configures a single property. + case Some((key, Some(value))) => + val runFunc = (sqlContext: SQLContext) => { + sqlContext.setConf(key, value) + Seq(Row(key, value)) + } + (keyValueOutput, runFunc) + + // (In Hive, "SET" returns all changed properties while "SET -v" returns all properties.) + // Queries all key-value pairs that are set in the SQLConf of the sqlContext. + case None => + val runFunc = (sqlContext: SQLContext) => { + sqlContext.getAllConfs.map { case (k, v) => Row(k, v) }.toSeq + } + (keyValueOutput, runFunc) + + // Queries all properties along with their default values and docs that are defined in the + // SQLConf of the sqlContext. + case Some(("-v", None)) => + val runFunc = (sqlContext: SQLContext) => { + sqlContext.conf.getAllDefinedConfs.map { case (key, defaultValue, doc) => + Row(key, defaultValue, doc) + } + } + val schema = StructType( + StructField("key", StringType, false) :: + StructField("default", StringType, false) :: + StructField("meaning", StringType, false) :: Nil) + (schema.toAttributes, runFunc) + + // Queries the deprecated "mapred.reduce.tasks" property. + case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) => + val runFunc = (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") + Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, sqlContext.conf.numShufflePartitions.toString)) + } + (keyValueOutput, runFunc) + + // Queries a single property. + case Some((key, None)) => + val runFunc = (sqlContext: SQLContext) => { + val value = + try sqlContext.getConf(key) catch { + case _: NoSuchElementException => "" + } + Seq(Row(key, value)) + } + (keyValueOutput, runFunc) + } + + override val output: Seq[Attribute] = _output + + override def run(sqlContext: SQLContext): Seq[Row] = runFunc(sqlContext) + +} + +/** + * An explain command for users to see how a command will be executed. + * + * Note that this command takes in a logical plan, runs the optimizer on the logical plan + * (but do NOT actually execute it). + */ +case class ExplainCommand( + logicalPlan: LogicalPlan, + override val output: Seq[Attribute] = + Seq(AttributeReference("plan", StringType, nullable = true)()), + extended: Boolean = false) + extends RunnableCommand { + + // Run through the optimizer to generate the physical plan. + override def run(sqlContext: SQLContext): Seq[Row] = try { + // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. + val queryExecution = sqlContext.executePlan(logicalPlan) + val outputString = if (extended) queryExecution.toString else queryExecution.simpleString + + outputString.split("\n").map(Row(_)) + } catch { case cause: TreeNodeException[_] => + ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) + } +} + + +case class CacheTableCommand( + tableName: String, + plan: Option[LogicalPlan], + isLazy: Boolean) + extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + plan.foreach { logicalPlan => + sqlContext.registerDataFrameAsTable(DataFrame(sqlContext, logicalPlan), tableName) + } + sqlContext.cacheTable(tableName) + + if (!isLazy) { + // Performs eager caching + sqlContext.table(tableName).count() + } + + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty +} + + +case class UncacheTableCommand(tableName: String) extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.table(tableName).unpersist(blocking = false) + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty +} + +/** + * Clear all cached data from the in-memory cache. + */ +case object ClearCacheCommand extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.clearCache() + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty +} + + +case class DescribeCommand( + child: SparkPlan, + override val output: Seq[Attribute], + isExtended: Boolean) + extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + child.schema.fields.map { field => + val cmtKey = "comment" + val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else "" + Row(field.name, field.dataType.simpleString, comment) + } + } +} + +/** + * A command for users to get tables in the given database. + * If a databaseName is not given, the current database will be used. + * The syntax of using this command in SQL is: + * {{{ + * SHOW TABLES [IN databaseName] + * }}} + */ +case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand { + + // The result of SHOW TABLES has two columns, tableName and isTemporary. + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("tableName", StringType, false) :: + StructField("isTemporary", BooleanType, false) :: Nil) + + schema.toAttributes + } + + override def run(sqlContext: SQLContext): Seq[Row] = { + // Since we need to return a Seq of rows, we will call getTables directly + // instead of calling tables in sqlContext. + val rows = sqlContext.catalog.getTables(databaseName).map { + case (tableName, isTemporary) => Row(tableName, isTemporary) + } + + rows + } +} + +/** + * A command for users to list all of the registered functions. + * The syntax of using this command in SQL is: + * {{{ + * SHOW FUNCTIONS + * }}} + * TODO currently we are simply ignore the db + */ +case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand { + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("function", StringType, nullable = false) :: Nil) + + schema.toAttributes + } + + override def run(sqlContext: SQLContext): Seq[Row] = pattern match { + case Some(p) => + try { + val regex = java.util.regex.Pattern.compile(p) + sqlContext.functionRegistry.listFunction().filter(regex.matcher(_).matches()).map(Row(_)) + } catch { + // probably will failed in the regex that user provided, then returns empty row. + case _: Throwable => Seq.empty[Row] + } + case None => + sqlContext.functionRegistry.listFunction().map(Row(_)) + } +} + +/** + * A command for users to get the usage of a registered function. + * The syntax of using this command in SQL is + * {{{ + * DESCRIBE FUNCTION [EXTENDED] upper; + * }}} + */ +case class DescribeFunction( + functionName: String, + isExtended: Boolean) extends RunnableCommand { + + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("function_desc", StringType, nullable = false) :: Nil) + + schema.toAttributes + } + + private def replaceFunctionName(usage: String, functionName: String): String = { + if (usage == null) { + "To be added." + } else { + usage.replaceAll("_FUNC_", functionName) + } + } + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.functionRegistry.lookupFunction(functionName) match { + case Some(info) => + val result = + Row(s"Function: ${info.getName}") :: + Row(s"Class: ${info.getClassName}") :: + Row(s"Usage: ${replaceFunctionName(info.getUsage(), info.getName)}") :: Nil + + if (isExtended) { + result :+ Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, info.getName)}") + } else { + result + } + + case None => Seq(Row(s"Function: $functionName not found.")) + } + } +} + +case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.setCurrentDatabase(databaseName) + Seq.empty[Row] + } + + override val output: Seq[Attribute] = Seq.empty +} + +case class CreateDataBase( + databaseName: String, + allowExisting: Boolean, + path: Option[String], + comment: Option[String], + props: Map[String, String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class CreateFunction( + functionName: String, + asName: String, + resourcesMap: Map[String, String], + isTemp: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableRename( + tableName: TableIdentifier, + renameTableName: Option[TableIdentifier])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableProperties( + tableName: TableIdentifier, + setProperties: Option[Map[String, Option[String]]], + dropProperties: Option[Map[String, Option[String]]], + allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSerDeProperties( + tableName: TableIdentifier, + serdeClassName: Option[String], + serdeProperties: Option[Map[String, Option[String]]], + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableStoreProperties( + tableName: TableIdentifier, + buckets: Option[BucketSpec], + noClustered: Boolean, + noSorted: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSkewed( + tableName: TableIdentifier, + skewedCols: Seq[String], + skewedValues: Seq[Seq[String]], + storedAsDirs: Boolean, + notSkewed: Boolean, + notStoredAsDirs: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSkewedLocation( + tableName: TableIdentifier, + skewedMap: Map[Seq[String], String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableAddPartition( + tableName: TableIdentifier, + partitionsAndLocs: Seq[(Map[String, Option[String]], Option[String])], + allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableRenamePartition( + tableName: TableIdentifier, + oldPartition: Map[String, Option[String]], + newPartition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableExchangePartition( + tableName: TableIdentifier, + fromTableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableDropPartition( + tableName: TableIdentifier, + partitions: Seq[Seq[(String, String, String)]], + allowExisting: Boolean, + purge: Boolean, + replication: Option[(String, Boolean)])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableArchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableUnarchivePartition( + tableName: TableIdentifier, + partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSetFileFormat( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + fileFormat: Option[Seq[String]], + genericFormat: Option[String])(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableSetLocation( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + location: String)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableTouch( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableCompact( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + compactType: String)(sql: String) extends RunnableCommand with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableMerge( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +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 RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableAddCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + +case class AlterTableReplaceCol( + tableName: TableIdentifier, + partition: Option[Map[String, Option[String]]], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) extends RunnableCommand + with Logging { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 7a0f7abaa1baf..71992e1f7ea2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -24,7 +24,7 @@ import java.util.{Calendar, GregorianCalendar, Properties} import org.h2.jdbc.JdbcSQLException import org.scalatest.{BeforeAndAfter, PrivateMethodTester} -import org.apache.spark.sql.execution.ExplainCommand +import org.apache.spark.sql.execution.commands.ExplainCommand import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.execution.PhysicalRDD 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 ac174aa6bfa6d..a5c2ea1451838 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 @@ -48,6 +48,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.commands._ import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, PreInsertCastAndRename, PreWriteCheck, ResolveDataSource} import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.hive.client._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 3687dd6f5a7ab..95adaa3859b76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.commands.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} import org.apache.spark.sql.hive.execution._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 246108e0d0e11..d61b17717aa3c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.CacheTableCommand +import org.apache.spark.sql.execution.commands.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.HiveNativeCommand diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index af4c44e578c84..87969897c7603 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.{ExplainCommand, SetCommand} +import org.apache.spark.sql.execution.commands.{ExplainCommand, SetCommand} import org.apache.spark.sql.execution.datasources.DescribeCommand import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} import org.apache.spark.sql.hive.test.TestHive From 170bd771655f09337c078f48ace9f18b6d667c9e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Feb 2016 10:15:22 +0000 Subject: [PATCH 08/13] Refactor. --- .../spark/sql/catalyst/CatalystQl.scala | 51 +- .../spark/sql/catalyst/PlanParser.scala | 53 ++ .../apache/spark/sql/execution/SparkQl.scala | 440 +---------- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../sql/execution/commands/commands.scala | 257 ++---- .../sql/execution/commands/parsers.scala | 420 ++++++++++ .../spark/sql/execution/SparkQlSuite.scala | 741 +++++++++++++++--- 7 files changed, 1192 insertions(+), 773 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 8099751900a42..da50d82ebbdaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -30,17 +30,20 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler +abstract class BaseParser(val conf: ParserConf) extends ParserInterface with ParserBase { + val planParsers: Seq[PlanParser] = Nil + + lazy val planParser: PlanParser = planParsers.reduce(_.orElse(_).asInstanceOf[PlanParser]) + + protected def nodeToPlan(node: ASTNode): LogicalPlan = { + planParser.applyOrElse(node, throw new NotImplementedError(node.text)) + } +} + /** * 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 - } - } - +private[sql] class CatalystQl(conf: ParserConf = SimpleParserConf()) extends BaseParser(conf) { /** * The safeParse method allows a user to focus on the parsing/AST transformation logic. This * method will take care of possible errors during the parsing process. @@ -90,7 +93,7 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends } } - protected def getClauses( + def getClauses( clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { var remainingNodes = nodeList @@ -108,11 +111,11 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends clauses } - protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode = + 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] = { + 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 @@ -120,14 +123,14 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends } } - protected def nodeToAttribute(node: ASTNode): Attribute = node match { + 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 { + 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) => @@ -155,7 +158,7 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends noParseRule("DataType", node) } - protected def nodeToStructField(node: ASTNode): StructField = node match { + 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) => @@ -165,7 +168,7 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) extends noParseRule("StructField", node) } - protected def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = { + def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = { tableNameParts.children.map { case Token(part, Nil) => cleanIdentifier(part) } match { @@ -214,7 +217,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C (keys, bitmasks) } - protected def nodeToPlan(node: ASTNode): LogicalPlan = node match { + override protected def nodeToPlan(node: ASTNode): LogicalPlan = node match { case Token("TOK_SHOWFUNCTIONS", args) => // Skip LIKE. val pattern = args match { @@ -621,22 +624,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 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala new file mode 100644 index 0000000000000..3a5cb44f911b6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala @@ -0,0 +1,53 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.parser.ASTNode +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.CurrentOrigin + +trait ParserBase { + object Token { + def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { + CurrentOrigin.setPosition(node.line, node.positionInLine) + node.pattern + } + } + + 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 + } +} + +/** + * Abstract class for a parser that parses ASTNode to LogicalPlan. + */ +abstract class PlanParser extends PartialFunction[ASTNode, LogicalPlan] with ParserBase + 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 1c1a8bdd52efd..00125c5a2bd8a 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 @@ -34,6 +34,26 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command + protected def extractProps(node: ASTNode): Seq[(String, String)] = node match { + 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) + } + + case Token("TOK_DBPROPLIST", props) => + props.map { + case Token("TOK_TABLEPROPERTY", keysAndValue) => + val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + } + + case _ => Seq.empty + } + protected override def nodeToPlan(node: ASTNode): LogicalPlan = { node match { case Token("TOK_SETCONFIG", Nil) => @@ -57,7 +77,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs) ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined) - case Token("TOK_EXPLAIN", explainArgs) if "TOK_QUERY" == explainArgs.head.text => + case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) @@ -86,15 +106,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly } val props: Map[String, String] = dbprops.toSeq.flatMap { case Token("TOK_DATABASEPROPERTIES", propList) => - propList.flatMap { - case Token("TOK_DBPROPLIST", props) => - props.map { - case Token("TOK_TABLEPROPERTY", keysAndValue) => - val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - } - } + propList.flatMap(extractProps) }.toMap CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) @@ -123,401 +135,8 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source) case Token("TOK_ALTERTABLE", alterTableArgs) => - val tabName = getClause("TOK_TABNAME", alterTableArgs) - val rename = getClauseOption("TOK_ALTERTABLE_RENAME", alterTableArgs) - val setTableProps = getClauseOption("TOK_ALTERTABLE_PROPERTIES", alterTableArgs) - val dropTableProps = getClauseOption("TOK_ALTERTABLE_DROPPROPERTIES", alterTableArgs) - val serde = getClauseOption("TOK_ALTERTABLE_SERIALIZER", alterTableArgs) - val serdeProps = getClauseOption("TOK_ALTERTABLE_SERDEPROPERTIES", alterTableArgs) - val partitionSpec = getClauseOption("TOK_PARTSPEC", alterTableArgs) - val bucketSpec = getClauseOption("TOK_ALTERTABLE_CLUSTER_SORT", alterTableArgs) - val bucketNum = getClauseOption("TOK_ALTERTABLE_BUCKETS", alterTableArgs) - val tableSkewed = getClauseOption("TOK_ALTERTABLE_SKEWED", alterTableArgs) - val tableSkewedLocation = getClauseOption("TOK_ALTERTABLE_SKEWED_LOCATION", alterTableArgs) - val addParts = getClauseOption("TOK_ALTERTABLE_ADDPARTS", alterTableArgs) - val renamePart = getClauseOption("TOK_ALTERTABLE_RENAMEPART", alterTableArgs) - val exchangePart = getClauseOption("TOK_ALTERTABLE_EXCHANGEPARTITION", alterTableArgs) - val dropParts = getClauseOption("TOK_ALTERTABLE_DROPPARTS", alterTableArgs) - val archivePart = getClauseOption("TOK_ALTERTABLE_ARCHIVE", alterTableArgs) - val unarchivePart = getClauseOption("TOK_ALTERTABLE_UNARCHIVE", alterTableArgs) - val setFileFormat = getClauseOption("TOK_ALTERTABLE_FILEFORMAT", alterTableArgs) - val setLocation = getClauseOption("TOK_ALTERTABLE_LOCATION", alterTableArgs) - val touch = getClauseOption("TOK_ALTERTABLE_TOUCH", alterTableArgs) - val compact = getClauseOption("TOK_ALTERTABLE_COMPACT", alterTableArgs) - val merge = getClauseOption("TOK_ALTERTABLE_MERGEFILES", alterTableArgs) - val renameCol = getClauseOption("TOK_ALTERTABLE_RENAMECOL", alterTableArgs) - val addCol = getClauseOption("TOK_ALTERTABLE_ADDCOLS", alterTableArgs) - val replaceCol = getClauseOption("TOK_ALTERTABLE_REPLACECOLS", alterTableArgs) - - val tableIdent: TableIdentifier = extractTableIdent(tabName) - - 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) => - (cleanIdentifier(ident.text), Some(cleanIdentifier(constant.text))) - case Token("TOK_PARTVAL", ident :: Nil) => - (cleanIdentifier(ident.text), None) - }.toMap - Some(spec) - case _ => None - } - } - - // Partition Spec - val partition: Option[Map[String, Option[String]]] = - partitionSpec.flatMap(parsePartitionSpec) - - 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 = unquoteString(key.text) - (k, None) - case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => - val k = unquoteString(key.text) - val v = unquoteString(value.text) - (k, Some(v)) - } - }.toMap - } - - if (rename.isDefined) { - // Rename table - val renamedTable = rename.map { - case Token("TOK_ALTERTABLE_RENAME", renameArgs) => - getClause("TOK_TABNAME", renameArgs) - } - val renamedTableIdent: Option[TableIdentifier] = renamedTable.map(extractTableIdent) - - AlterTableRename(tableIdent, renamedTableIdent)(node.source) - - } else if (setTableProps.isDefined || dropTableProps.isDefined) { - // Alter table properties - val setTableProperties = setTableProps.map { - case Token("TOK_ALTERTABLE_PROPERTIES", args :: Nil) => extractTableProps(args) - } - val dropTableProperties = dropTableProps.map { - case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) => - extractTableProps(args.head) - } - val allowExisting = dropTableProps.flatMap { - case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) => - getClauseOption("TOK_IFEXISTS", args) - } - - AlterTableProperties( - tableIdent, - setTableProperties, - dropTableProperties, - allowExisting.isDefined)(node.source) - } else if (serde.isDefined || serdeProps.isDefined) { - val serdeArgs: Option[Seq[ASTNode]] = serde.map { - case Token("TOK_ALTERTABLE_SERIALIZER", args) => - args - } - - val serdeClassName = serdeArgs.map(_.head).map { - case Token(className, Nil) => className - } - - val serdeProperties: Option[Map[String, Option[String]]] = Option( - // SET SERDE serde_classname WITH SERDEPROPERTIES - serdeArgs.map(_.tail).map { props => - if (props.isEmpty) { - null - } else { - extractTableProps(props.head) - } - }.getOrElse { - // SET SERDEPROPERTIES - serdeProps.map { - case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) => - extractTableProps(args.head) - }.getOrElse(null) - } - ) - - AlterTableSerDeProperties( - tableIdent, - serdeClassName, - serdeProperties, - partition)(node.source) - } else if (bucketSpec.isDefined) { - val (buckets, noClustered, noSorted) = bucketSpec.map { - case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) => - clusterAndSoryByArgs match { - case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) => - val bucketCols = bucketArgs.head.children.map(_.text) - - val (sortCols, sortDirections, numBuckets) = { - if (bucketArgs(1).text == "TOK_TABCOLNAME") { - val cols = bucketArgs(1).children.map { - case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => - (colName, Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => - (colName, Descending) - } - (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt) - } else { - (Nil, Nil, bucketArgs(1).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) - } - }.getOrElse((None, false, false)) // should not reach here - - AlterTableStoreProperties( - tableIdent, - buckets, - noClustered, - noSorted)(node.source) - } else if (bucketNum.isDefined) { - val num = bucketNum.get.children.head.text.toInt - val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) - AlterTableStoreProperties( - tableIdent, - buckets, - false, - false)(node.source) - } else if (tableSkewed.isDefined) { - // Alter Table not skewed - // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed. - val notSkewed = if (tableSkewed.get.children.size == 0) { - true - } else { - false - } - - val (notStoredAsDirs, skewedArgs) = tableSkewed.map { - case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => - // Alter Table not stored as directories - (true, None) - case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) => - val (cols, values, storedAsDirs) = skewedArgs match { - case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => - val cols = skewedCols.children.map(_.text) - val values = skewedValues match { - case Token("TOK_TABCOLVALUE", values) => Seq(values.map(_.text)) - case Token("TOK_TABCOLVALUE_PAIR", pairs) => - pairs.map { - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => vals.map(_.text) - } - } - } - - val storedAsDirs = stored match { - case Token("TOK_STOREDASDIRS", Nil) :: Nil => true - case _ => false - } - - (cols, values, storedAsDirs) - } - (false, Some((cols, values, storedAsDirs))) - }.get - - if (skewedArgs.isDefined) { - AlterTableSkewed( - tableIdent, - skewedArgs.get._1, - skewedArgs.get._2, - skewedArgs.get._3, notSkewed, notStoredAsDirs)(node.source) - } else { - AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source) - } - } else if (tableSkewedLocation.isDefined) { - val skewedMaps = tableSkewedLocation.get.children(0) match { - case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) => - locationList match { - case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) => - locationMaps.map { - case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => - val k = key match { - case Token(const, Nil) => Seq(const) - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => vals.map(_.text) - } - } - (k, value.text) - }.toMap - } - } - AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) - } else if (addParts.isDefined) { - val allowExisting = getClauseOption("TOK_IFNOTEXISTS", addParts.get.children) - val parts = if (allowExisting.isDefined) { - addParts.get.children.tail - } else { - addParts.get.children - } - - 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.isDefined)(node.source) - } else if (renamePart.isDefined) { - val newPartition = parsePartitionSpec(renamePart.get.children(0)) - AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source) - } else if (exchangePart.isDefined) { - val Seq(Some(partSpec), Some(fromTable)) = - getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), exchangePart.get.children) - val partition = parsePartitionSpec(partSpec).get - val fromTableIdent = extractTableIdent(fromTable) - AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) - } else if (dropParts.isDefined) { - val parts = dropParts.get.children.collect { - case Token("TOK_PARTSPEC", partitions) => - partitions.map { - case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => - (cleanIdentifier(ident.text), op.text, cleanIdentifier(constant.text)) - } - } - - val allowExisting = getClauseOption("TOK_IFEXISTS", dropParts.get.children) - val purge = getClauseOption("PURGE", dropParts.get.children) - - val replication = getClauseOption("TOK_REPLICATION", dropParts.get.children).map { - case Token("TOK_REPLICATION", replId :: metadata :: Nil) => - (replId.text, true) - case Token("TOK_REPLICATION", replId :: Nil) => - (replId.text, false) - } - - AlterTableDropPartition( - tableIdent, - parts, - allowExisting.isDefined, - purge.isDefined, - replication)(node.source) - } else if (archivePart.isDefined) { - val partition = parsePartitionSpec(archivePart.get.children(0)).get - AlterTableArchivePartition(tableIdent, partition)(node.source) - } else if (unarchivePart.isDefined) { - val partition = parsePartitionSpec(unarchivePart.get.children(0)).get - AlterTableUnarchivePartition(tableIdent, partition)(node.source) - } else if (setFileFormat.isDefined) { - val Seq(fileFormat, genericFormat) = - getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), - setFileFormat.get.children) - val fFormat = fileFormat.map(_.children.map(_.text)) - val gFormat = genericFormat.map(_.children(0).text) - AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) - } else if (setLocation.isDefined) { - val loc = cleanIdentifier(setLocation.get.children(0).text) - AlterTableSetLocation(tableIdent, partition, loc)(node.source) - } else if (touch.isDefined) { - val part = getClauseOption("TOK_PARTSPEC", touch.get.children).flatMap(parsePartitionSpec) - AlterTableTouch(tableIdent, part)(node.source) - } else if (compact.isDefined) { - val compactType = cleanIdentifier(compact.get.children(0).text) - AlterTableCompact(tableIdent, partition, compactType)(node.source) - } else if (merge.isDefined) { - AlterTableMerge(tableIdent, partition)(node.source) - } - else if (renameCol.isDefined) { - val oldName = renameCol.get.children(0).text - val newName = renameCol.get.children(1).text - val dataType = nodeToDataType(renameCol.get.children(2)) - val afterPos = - getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", renameCol.get.children) - val afterPosCol = afterPos.map { ap => - ap.children match { - case Token(col, Nil) :: Nil => col - case _ => null - } - } - - val restrict = getClauseOption("TOK_RESTRICT", renameCol.get.children) - val cascade = getClauseOption("TOK_CASCADE", renameCol.get.children) - - val comment = if (renameCol.get.children.size > 3) { - renameCol.get.children(3) match { - case Token(commentStr, Nil) - if commentStr != "TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION" && - commentStr != "TOK_RESTRICT" && commentStr != "TOK_CASCADE" => Some(commentStr) - case _ => - None - } - } else { - None - } - - AlterTableChangeCol( - tableIdent, - partition, - oldName, - newName, - dataType, - comment, - afterPos.isDefined, - afterPosCol, - restrict.isDefined, - cascade.isDefined)(node.source) - } else if (addCol.isDefined || replaceCol.isDefined) { - val thisNode = if (addCol.isDefined) { - addCol.get - } else { - replaceCol.get - } - - val tableCols = getClause("TOK_TABCOLLIST", thisNode.children) - val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) - } - - val restrict = getClauseOption("TOK_RESTRICT", thisNode.children) - val cascade = getClauseOption("TOK_CASCADE", thisNode.children) - - if (addCol.isDefined) { - AlterTableAddCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) - } else { - AlterTableReplaceCol( - tableIdent, - partition, - columns, - restrict.isDefined, - cascade.isDefined)(node.source) - } - } else { - nodeToDescribeFallback(node) - } + AlterTableCommandParser(this).applyOrElse(node, + (node: ASTNode) => throw new NotImplementedError(node.text)) case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( @@ -545,16 +164,7 @@ 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).toMap val asClause = tableAs.map(nodeToPlan(_)) if (temp.isDefined && allowExisting.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index cd7da1c2a33a0..1b01e822e99ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -25,9 +25,8 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.commands.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation} -import org.apache.spark.sql.execution.commands._ +import org.apache.spark.sql.execution.commands.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTempTableUsing, DescribeCommand => LogicalDescribeCommand, _} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala index 1158443578f26..55261206c9630 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala @@ -31,6 +31,15 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(sql) + } + + override val output: Seq[Attribute] = + Seq(AttributeReference("result", StringType, nullable = false)()) +} + case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging { private def keyValueOutput: Seq[Attribute] = { @@ -381,84 +390,41 @@ case class CreateDataBase( allowExisting: Boolean, path: Option[String], comment: Option[String], - props: Map[String, String])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + isTemp: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging case class AlterTableRename( tableName: TableIdentifier, - renameTableName: Option[TableIdentifier])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + renameTableName: TableIdentifier)(sql: String) extends NativeDDLCommands(sql) + with Logging -case class AlterTableProperties( +case class AlterTableSetProperties( tableName: TableIdentifier, - setProperties: Option[Map[String, Option[String]]], - dropProperties: Option[Map[String, Option[String]]], - allowExisting: Boolean)(sql: String) extends RunnableCommand with Logging { + setProperties: Map[String, Option[String]])(sql: String) extends NativeDDLCommands(sql) + with Logging - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} +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 RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + noSorted: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging case class AlterTableSkewed( tableName: TableIdentifier, @@ -466,171 +432,75 @@ case class AlterTableSkewed( skewedValues: Seq[Seq[String]], storedAsDirs: Boolean, notSkewed: Boolean, - notStoredAsDirs: Boolean)(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + notStoredAsDirs: Boolean)(sql: String) extends NativeDDLCommands(sql) with Logging case class AlterTableSkewedLocation( tableName: TableIdentifier, - skewedMap: Map[Seq[String], String])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + replication: Option[(String, Boolean)])(sql: String) extends NativeDDLCommands(sql) + with Logging case class AlterTableArchivePartition( tableName: TableIdentifier, - partition: Map[String, Option[String]])(sql: String) extends RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + location: String)(sql: String) extends NativeDDLCommands(sql) + with Logging case class AlterTableTouch( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + compactType: String)(sql: String) extends NativeDDLCommands(sql) + with Logging case class AlterTableMerge( tableName: TableIdentifier, - partition: Option[Map[String, Option[String]]])(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + partition: Option[Map[String, Option[String]]])(sql: String) extends NativeDDLCommands(sql) + with Logging case class AlterTableChangeCol( tableName: TableIdentifier, @@ -642,45 +512,22 @@ case class AlterTableChangeCol( afterPos: Boolean, afterPosCol: Option[String], restrict: Boolean, - cascade: Boolean)(sql: String) extends RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand - with Logging { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} + 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 RunnableCommand - with Logging { + cascade: Boolean)(sql: String) extends NativeDDLCommands(sql) + with Logging - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala new file mode 100644 index 0000000000000..0a579a0c25c12 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala @@ -0,0 +1,420 @@ +/* + * 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.commands + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} +import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.execution.commands._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.types.StructType + +case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { + + 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) => + (unquoteString(cleanIdentifier(ident.text)), + Some(unquoteString(cleanIdentifier(constant.text)))) + case Token("TOK_PARTVAL", ident :: Nil) => + (unquoteString(cleanIdentifier(ident.text)), None) + }.toMap + Some(spec) + case _ => None + } + } + + 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 = unquoteString(cleanIdentifier(key.text)) + (k, None) + case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = unquoteString(cleanIdentifier(key.text)) + val v = unquoteString(cleanIdentifier(value.text)) + (k, Some(v)) + } + }.toMap + } + + override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE" + + override def apply(v1: ASTNode): LogicalPlan = v1.children match { + case (tabName @ Token("TOK_TABNAME", _)) :: rest => + val tableIdent: TableIdentifier = base.extractTableIdent(tabName) + val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children) + val partition = partitionSpec.flatMap(parsePartitionSpec) + matchAlterTableCommands(v1, rest, tableIdent, partition) + case _ => + throw new NotImplementedError(v1.text) + } + + 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) :: rest => + val renamedTable = base.getClause("TOK_TABNAME", renameArgs) + val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable) + AlterTableRename(tableIdent, renamedTableIdent)(node.source) + + case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest => + val setTableProperties = extractTableProps(args.head) + AlterTableSetProperties( + tableIdent, + setTableProperties)(node.source) + + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest => + val dropTableProperties = extractTableProps(args.head) + val allowExisting = base.getClauseOption("TOK_IFEXISTS", args) + AlterTableDropProperties( + tableIdent, + dropTableProperties, allowExisting.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest => + val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text)) + + val serdeProperties: Option[Map[String, Option[String]]] = Option( + // SET SERDE serde_classname WITH SERDEPROPERTIES + if (serdeArgs.tail.isEmpty) { + null + } else { + extractTableProps(serdeArgs.tail.head) + } + ) + + AlterTableSerDeProperties( + tableIdent, + Some(serdeClassName), + serdeProperties, + partition)(node.source) + + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest => + val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) + + AlterTableSerDeProperties( + tableIdent, + None, + Some(serdeProperties), + partition)(node.source) + + case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest => + val (buckets, noClustered, noSorted) = bucketSpec match { + case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) => + clusterAndSoryByArgs match { + case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) => + val bucketCols = bucketArgs.head.children.map(_.text) + + val (sortCols, sortDirections, numBuckets) = { + if (bucketArgs(1).text == "TOK_TABCOLNAME") { + val cols = bucketArgs(1).children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => + (colName, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => + (colName, Descending) + } + (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt) + } else { + (Nil, Nil, bucketArgs(1).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) :: rest => + val num = bucketNum.toInt + val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) + AlterTableStoreProperties( + tableIdent, + buckets, + false, + false)(node.source) + + case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest => + // Alter Table not skewed + // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed. + val notSkewed = if (tableSkewed.children.size == 0) { + true + } else { + false + } + + val (notStoredAsDirs, skewedArgs) = tableSkewed match { + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => + // Alter Table not stored as directories + (true, None) + case Token("TOK_ALTERTABLE_SKEWED", skewedArgs :: Nil) => + val (cols, values, storedAsDirs) = skewedArgs match { + case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => + val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text))) + val values = skewedValues match { + case Token("TOK_TABCOLVALUE", values) => + Seq(values.map(n => unquoteString(cleanIdentifier(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 => unquoteString(cleanIdentifier(n.text))) + } + } + } + + val storedAsDirs = stored match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false + } + + (cols, values, storedAsDirs) + } + (false, Some((cols, values, storedAsDirs))) + } + + if (skewedArgs.isDefined) { + AlterTableSkewed( + tableIdent, + skewedArgs.get._1, /* cols */ + skewedArgs.get._2, /* values */ + skewedArgs.get._3, /* storedAsDirs */ + notSkewed, notStoredAsDirs)(node.source) + } else { + AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source) + } + + case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest => + val skewedMaps = args(0) match { + case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) => + locationList match { + case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) => + locationMaps.map { + case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => + val k = key match { + case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const))) + case Token("TOK_TABCOLVALUES", values :: Nil) => + values match { + case Token("TOK_TABCOLVALUE", vals) => + vals.map(n => unquoteString(cleanIdentifier(n.text))) + } + } + (k, unquoteString(cleanIdentifier(value.text))) + }.toMap + } + } + AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) + + case Token("TOK_ALTERTABLE_ADDPARTS", addPartsArgs) :: rest => + val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs) + val parts = if (allowExisting.isDefined) { + addPartsArgs.tail + } else { + 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.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest => + val newPartition = parsePartitionSpec(args(0)) + AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(node.source) + + case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", args) :: rest => + val Seq(Some(partSpec), Some(fromTable)) = + base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args) + val partition = parsePartitionSpec(partSpec).get + val fromTableIdent = base.extractTableIdent(fromTable) + AlterTableExchangePartition(tableIdent, fromTableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_DROPPARTS", args) :: rest => + val parts = args.collect { + case Token("TOK_PARTSPEC", partitions) => + partitions.map { + case Token("TOK_PARTVAL", ident :: op :: constant :: Nil) => + (unquoteString(cleanIdentifier(ident.text)), + op.text, unquoteString(cleanIdentifier(constant.text))) + } + } + + val allowExisting = base.getClauseOption("TOK_IFEXISTS", args) + val purge = base.getClauseOption("PURGE", args) + + val replication = base.getClauseOption("TOK_REPLICATION", args).map { + case Token("TOK_REPLICATION", replId :: metadata :: Nil) => + (unquoteString(cleanIdentifier(replId.text)), true) + case Token("TOK_REPLICATION", replId :: Nil) => + (unquoteString(cleanIdentifier(replId.text)), false) + } + + AlterTableDropPartition( + tableIdent, + parts, + allowExisting.isDefined, + purge.isDefined, + replication)(node.source) + + case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest => + val partition = parsePartitionSpec(args(0)).get + AlterTableArchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest => + val partition = parsePartitionSpec(args(0)).get + AlterTableUnarchivePartition(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest => + val Seq(fileFormat, genericFormat) = + base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), + args) + val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text)))) + val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text))) + AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + + case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: rest => + AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source) + + case Token("TOK_ALTERTABLE_TOUCH", args) :: rest => + val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) + AlterTableTouch(tableIdent, part)(node.source) + + case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: rest => + AlterTableCompact(tableIdent, partition, + unquoteString(cleanIdentifier(compactType)))(node.source) + + case Token("TOK_ALTERTABLE_MERGEFILES", _) :: rest => + AlterTableMerge(tableIdent, partition)(node.source) + + case Token("TOK_ALTERTABLE_RENAMECOL", args) :: rest => + val oldName = args(0).text + val newName = args(1).text + val dataType = base.nodeToDataType(args(2)) + val afterPos = + base.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 = base.getClauseOption("TOK_RESTRICT", args) + val cascade = base.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(unquoteString(cleanIdentifier(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) :: rest => + val tableCols = base.getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(base.nodeToStructField)) + } + + val restrict = base.getClauseOption("TOK_RESTRICT", args) + val cascade = base.getClauseOption("TOK_CASCADE", args) + + AlterTableAddCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + + case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: rest => + val tableCols = base.getClause("TOK_TABCOLLIST", args) + val columns = tableCols match { + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(base.nodeToStructField)) + } + + val restrict = base.getClauseOption("TOK_RESTRICT", args) + val cascade = base.getClauseOption("TOK_CASCADE", args) + + AlterTableReplaceCol( + tableIdent, + partition, + columns, + restrict.isDefined, + cascade.isDefined)(node.source) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala index 2af2ff68dfa5b..a0c5168780591 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala @@ -17,133 +17,636 @@ package org.apache.spark.sql.execution +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.commands._ +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ class SparkQlSuite extends PlanTest { val parser = new SparkQl() test("create database") { - parser.parsePlan("CREATE DATABASE IF NOT EXISTS database_name " + - "COMMENT 'database_comment' LOCATION '/home/user/db' " + - "WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + 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") { - parser.parsePlan("CREATE TEMPORARY FUNCTION helloworld as " + - "'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', " + - "FILE 'path/to/file'") - } - - test("alter table") { - // Rename table - parser.parsePlan("ALTER TABLE table_name RENAME TO new_table_name") - - // Alter table properties - parser.parsePlan("ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + - "'comment' = 'new_comment')") - parser.parsePlan("ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')") - parser.parsePlan("ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')") - - // Alter table SerDe properties - parser.parsePlan("ALTER TABLE table_name SET SERDE 'org.apache.class'") - parser.parsePlan("ALTER TABLE table_name SET SERDE 'org.apache.class' " + - "WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')") - parser.parsePlan("ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', " + - "'field.delim' = ',')") - parser.parsePlan("ALTER TABLE table_name PARTITION (test, dt='2008-08-08', " + - "country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', " + - "'field.delim' = ',')") - parser.parsePlan("ALTER TABLE table_name PARTITION (test, dt='2008-08-08', " + - "country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',')") - - // Alter Table storage properties - parser.parsePlan("ALTER TABLE table_name CLUSTERED BY (dt, country) " + - "INTO 10 BUCKETS") - parser.parsePlan("ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + - "(dt, country DESC) INTO 10 BUCKETS") - parser.parsePlan("ALTER TABLE table_name INTO 20 BUCKETS") - parser.parsePlan("ALTER TABLE table_name NOT CLUSTERED") - parser.parsePlan("ALTER TABLE table_name NOT SORTED") - - // Alter Table skewed - parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + - "(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES") - parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + - "('2008-08-08', 'us') STORED AS DIRECTORIES") - parser.parsePlan("ALTER TABLE table_name SKEWED BY (dt, country) ON " + - "(('2008-08-08', 'us'), ('2009-09-09', 'uk'))") - - // Alter Table skewed location - parser.parsePlan("ALTER TABLE table_name SET SKEWED LOCATION " + - "('123'='location1', 'test'='location2')") - parser.parsePlan("ALTER TABLE table_name SET SKEWED LOCATION " + - "(('2008-08-08', 'us')='location1', 'test'='location2')") - - // Alter Table add partition - parser.parsePlan("ALTER TABLE table_name ADD IF NOT EXISTS PARTITION " + - "(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION " + - "(dt='2009-09-09', country='uk')") - - // Alter Table rename partition - parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "RENAME TO PARTITION (dt='2008-09-09', country='uk')") - - // Alter Table exchange partition - parser.parsePlan("ALTER TABLE table_name_1 EXCHANGE PARTITION " + - "(dt='2008-08-08', country='us') WITH TABLE table_name_2") - - // Alter Table drop partitions - parser.parsePlan("ALTER TABLE table_name DROP IF EXISTS PARTITION " + - "(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk')") - parser.parsePlan("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 archive partition - parser.parsePlan("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") - - // Alter Table unarchive partition - parser.parsePlan("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") - - // Alter Table set file format - parser.parsePlan("ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test' INPUTDRIVER 'test' OUTPUTDRIVER 'test'") - parser.parsePlan("ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test'") - parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET FILEFORMAT PARQUET") - - // Alter Table set location - parser.parsePlan("ALTER TABLE table_name SET LOCATION 'new location'") - parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET LOCATION 'new location'") - - // Alter Table touch - parser.parsePlan("ALTER TABLE table_name TOUCH") - parser.parsePlan("ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')") - - // Alter Table compact - parser.parsePlan("ALTER TABLE table_name COMPACT 'compaction_type'") - parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "COMPACT 'MAJOR'") - - // Alter Table concatenate - parser.parsePlan("ALTER TABLE table_name CONCATENATE") - parser.parsePlan("ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE") - - // Alter Table change column name/type/position/comment - parser.parsePlan("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") - - parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + - "COMMENT 'col_comment' FIRST CASCADE") - parser.parsePlan("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT " + - "COMMENT 'col_comment' AFTER column_name RESTRICT") - - // Alter Table add/replace columns - parser.parsePlan("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") - parser.parsePlan("ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT " + - "COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT") + 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) } } From 4a963317165877036469abe9bc45eebfc725edf1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 16 Feb 2016 13:19:02 +0000 Subject: [PATCH 09/13] Fix scala style. --- .../scala/org/apache/spark/sql/execution/SparkQlSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala index a0c5168780591..092d4d37597dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala @@ -67,7 +67,7 @@ class SparkQlSuite extends PlanTest { } test("alter table: rename table") { - val sql ="ALTER TABLE table_name RENAME TO new_table_name" + val sql = "ALTER TABLE table_name RENAME TO new_table_name" val parsed = parser.parsePlan(sql) val expected = AlterTableRename( TableIdentifier("table_name", None), From 3c2fc25ecb1fdfa77b15db5d163c6fab46c9c8e8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 16 Feb 2016 14:02:11 +0000 Subject: [PATCH 10/13] Fix import. --- .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 8fef22cf777f6..53f4046d02220 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -34,7 +34,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf} -import org.apache.spark.sql.execution.SetCommand +import org.apache.spark.sql.execution.commands.SetCommand import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} import org.apache.spark.sql.types._ import org.apache.spark.util.{Utils => SparkUtils} From bcf3f28bb4ac1f3b7c27bc17e30e86e8a70accd9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 22 Feb 2016 10:07:00 +0000 Subject: [PATCH 11/13] Address comments. --- .../sql/catalyst/AbstractSparkSQLParser.scala | 1 + .../sql/catalyst/parser/CatalystQl.scala | 905 ++++++++++++++++++ .../sql/catalyst/parser/ParserInterface.scala | 36 + .../sql/catalyst/parser/ParserSupport.scala | 134 +++ .../spark/sql/catalyst/CatalystQlSuite.scala | 1 + .../org/apache/spark/sql/SQLContext.scala | 1 + .../apache/spark/sql/execution/SparkQl.scala | 33 +- .../sql/execution/commands/commands.scala | 255 +---- .../spark/sql/execution/commands/ddl.scala | 188 ++++ .../sql/execution/commands/parsers.scala | 305 +++--- .../org/apache/spark/sql/functions.scala | 3 +- .../spark/sql/execution/DDLCommandSuite.scala | 652 +++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../org/apache/spark/sql/hive/HiveQl.scala | 2 + 14 files changed, 2119 insertions(+), 400 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/DDLCommandSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala index 38fa5cb585ee7..55af9112691bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala @@ -23,6 +23,7 @@ import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.combinator.PackratParsers import scala.util.parsing.input.CharArrayReader.EofCh +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical._ private[sql] abstract class AbstractSparkSQLParser 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 new file mode 100644 index 0000000000000..80c8bfbc43891 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala @@ -0,0 +1,905 @@ +/* + * 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 java.sql.Date + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.parser._ +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 + +/** + * This class translates SQL to Catalyst [[LogicalPlan]]s or [[Expression]]s. + */ +private[sql] class CatalystQl(conf: ParserConf = SimpleParserConf()) extends ParserInterface { + import ParserSupport._ + /** + * The safeParse method allows a user to focus on the parsing/AST transformation logic. This + * method will take care of possible errors during the parsing process. + */ + protected def safeParse[T](sql: String, ast: ASTNode)(toResult: ASTNode => T): T = { + try { + toResult(ast) + } catch { + case e: MatchError => throw e + case e: AnalysisException => throw e + case e: Exception => + throw new AnalysisException(e.getMessage) + case e: NotImplementedError => + throw new AnalysisException( + s"""Unsupported language features in query + |== SQL == + |$sql + |== AST == + |${ast.treeString} + |== Error == + |$e + |== Stacktrace == + |${e.getStackTrace.head} + """.stripMargin) + } + } + + /** Creates LogicalPlan for a given SQL string. */ + def parsePlan(sql: String): LogicalPlan = + safeParse(sql, ParseDriver.parsePlan(sql, conf))(nodeToPlan) + + /** Creates Expression for a given SQL string. */ + def parseExpression(sql: String): Expression = + safeParse(sql, ParseDriver.parseExpression(sql, conf))(selExprNodeToExpr(_).get) + + /** Creates TableIdentifier for a given SQL string. */ + 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)) + } + } + + 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) + } + + /** + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 + * Check the following link for details. + * +https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup + * + * The bitmask denotes the grouping expressions validity for a grouping set, + * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) + * e.g. In superset (k1, k2, k3), (bit 2: k1, bit 1: k2, and bit 0: k3), the grouping id of + * GROUPING SETS (k1, k2) and (k2) should be 1 and 5 respectively. + */ + protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { + val (keyASTs, setASTs) = children.partition { + case Token("TOK_GROUPING_SETS_EXPRESSION", _) => false // grouping sets + case _ => true // grouping keys + } + + val keys = keyASTs.map(nodeToExpr) + val keyMap = keyASTs.zipWithIndex.toMap + + val mask = (1 << keys.length) - 1 + val bitmasks: Seq[Int] = setASTs.map { + case Token("TOK_GROUPING_SETS_EXPRESSION", columns) => + columns.foldLeft(mask)((bitmap, col) => { + val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2).getOrElse( + throw new AnalysisException(s"${col.treeString} doesn't show up in the GROUP BY list")) + // 0 means that the column at the given index is a grouping column, 1 means it is not, + // so we unset the bit in bitmap. + bitmap & ~(1 << (keys.length - 1 - keyIndex)) + }) + case _ => sys.error("Expect GROUPING SETS clause") + } + + (keys, bitmasks) + } + + protected def nodeToPlan(node: ASTNode): LogicalPlan = node match { + case Token("TOK_SHOWFUNCTIONS", args) => + // Skip LIKE. + val pattern = args match { + case like :: nodes if like.text.toUpperCase == "LIKE" => nodes + case nodes => nodes + } + + // Extract Database and Function name + pattern match { + case Nil => + ShowFunctions(None, None) + case Token(name, Nil) :: Nil => + ShowFunctions(None, Some(unquoteString(cleanIdentifier(name)))) + case Token(db, Nil) :: Token(name, Nil) :: Nil => + ShowFunctions(Some(unquoteString(cleanIdentifier(db))), + Some(unquoteString(cleanIdentifier(name)))) + case _ => + noParseRule("SHOW FUNCTIONS", node) + } + + case Token("TOK_DESCFUNCTION", Token(functionName, Nil) :: isExtended) => + DescribeFunction(cleanIdentifier(functionName), isExtended.nonEmpty) + + case Token("TOK_QUERY", queryArgs @ Token("TOK_CTE" | "TOK_FROM" | "TOK_INSERT", _) :: _) => + val (fromClause: Option[ASTNode], insertClauses, cteRelations) = + queryArgs match { + case Token("TOK_CTE", ctes) :: Token("TOK_FROM", from) :: inserts => + val cteRelations = ctes.map { node => + val relation = nodeToRelation(node).asInstanceOf[SubqueryAlias] + relation.alias -> relation + } + (Some(from.head), inserts, Some(cteRelations.toMap)) + case Token("TOK_FROM", from) :: inserts => + (Some(from.head), inserts, None) + case Token("TOK_INSERT", _) :: Nil => + (None, queryArgs, None) + } + + // Return one query for each insert clause. + val queries = insertClauses.map { + case Token("TOK_INSERT", singleInsert) => + val ( + intoClause :: + destClause :: + selectClause :: + selectDistinctClause :: + whereClause :: + groupByClause :: + rollupGroupByClause :: + cubeGroupByClause :: + groupingSetsClause :: + orderByClause :: + havingClause :: + sortByClause :: + clusterByClause :: + distributeByClause :: + limitClause :: + lateralViewClause :: + windowClause :: Nil) = { + getClauses( + Seq( + "TOK_INSERT_INTO", + "TOK_DESTINATION", + "TOK_SELECT", + "TOK_SELECTDI", + "TOK_WHERE", + "TOK_GROUPBY", + "TOK_ROLLUP_GROUPBY", + "TOK_CUBE_GROUPBY", + "TOK_GROUPING_SETS", + "TOK_ORDERBY", + "TOK_HAVING", + "TOK_SORTBY", + "TOK_CLUSTERBY", + "TOK_DISTRIBUTEBY", + "TOK_LIMIT", + "TOK_LATERAL_VIEW", + "WINDOW"), + singleInsert) + } + + val relations = fromClause match { + case Some(f) => nodeToRelation(f) + case None => OneRowRelation + } + + val withWhere = whereClause.map { whereNode => + val Seq(whereExpr) = whereNode.children + Filter(nodeToExpr(whereExpr), relations) + }.getOrElse(relations) + + val select = (selectClause orElse selectDistinctClause) + .getOrElse(sys.error("No select clause.")) + + val transformation = nodeToTransformation(select.children.head, withWhere) + + val withLateralView = lateralViewClause.map { lv => + nodeToGenerate(lv.children.head, outer = false, withWhere) + }.getOrElse(withWhere) + + // The projection of the query can either be a normal projection, an aggregation + // (if there is a group by) or a script transformation. + val withProject: LogicalPlan = transformation.getOrElse { + val selectExpressions = + select.children.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)) + Seq( + groupByClause.map(e => e match { + case Token("TOK_GROUPBY", children) => + // Not a transformation so must be either project or aggregation. + Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) + case _ => sys.error("Expect GROUP BY") + }), + groupingSetsClause.map(e => e match { + case Token("TOK_GROUPING_SETS", children) => + val(groupByExprs, masks) = extractGroupingSet(children) + GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) + case _ => sys.error("Expect GROUPING SETS") + }), + rollupGroupByClause.map(e => e match { + case Token("TOK_ROLLUP_GROUPBY", children) => + Aggregate( + Seq(Rollup(children.map(nodeToExpr))), + selectExpressions, + withLateralView) + case _ => sys.error("Expect WITH ROLLUP") + }), + cubeGroupByClause.map(e => e match { + case Token("TOK_CUBE_GROUPBY", children) => + Aggregate( + Seq(Cube(children.map(nodeToExpr))), + selectExpressions, + withLateralView) + case _ => sys.error("Expect WITH CUBE") + }), + Some(Project(selectExpressions, withLateralView))).flatten.head + } + + // Handle HAVING clause. + val withHaving = havingClause.map { h => + val havingExpr = h.children match { case Seq(hexpr) => nodeToExpr(hexpr) } + // Note that we added a cast to boolean. If the expression itself is already boolean, + // the optimizer will get rid of the unnecessary cast. + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + + // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. + val withSort = + (orderByClause, sortByClause, distributeByClause, clusterByClause) match { + case (Some(totalOrdering), None, None, None) => + Sort(totalOrdering.children.map(nodeToSortOrder), global = true, withDistinct) + case (None, Some(perPartitionOrdering), None, None) => + Sort( + perPartitionOrdering.children.map(nodeToSortOrder), + global = false, withDistinct) + case (None, None, Some(partitionExprs), None) => + RepartitionByExpression( + partitionExprs.children.map(nodeToExpr), withDistinct) + case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => + Sort( + perPartitionOrdering.children.map(nodeToSortOrder), global = false, + RepartitionByExpression( + partitionExprs.children.map(nodeToExpr), + withDistinct)) + case (None, None, None, Some(clusterExprs)) => + Sort( + clusterExprs.children.map(nodeToExpr).map(SortOrder(_, Ascending)), + global = false, + RepartitionByExpression( + clusterExprs.children.map(nodeToExpr), + withDistinct)) + case (None, None, None, None) => withDistinct + case _ => sys.error("Unsupported set of ordering / distribution clauses.") + } + + val withLimit = + limitClause.map(l => nodeToExpr(l.children.head)) + .map(Limit(_, withSort)) + .getOrElse(withSort) + + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.children.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + + // TOK_INSERT_INTO means to add files to the table. + // TOK_DESTINATION means to overwrite the table. + val resultDestination = + (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) + val overwrite = intoClause.isEmpty + nodeToDest( + resultDestination, + withWindowDefinitions, + overwrite) + } + + // If there are multiple INSERTS just UNION them together into one query. + val query = if (queries.length == 1) queries.head else Union(queries) + + // return With plan if there is CTE + cteRelations.map(With(query, _)).getOrElse(query) + + case Token("TOK_UNIONALL", left :: right :: Nil) => + Union(nodeToPlan(left), nodeToPlan(right)) + case Token("TOK_UNIONDISTINCT", left :: right :: Nil) => + Distinct(Union(nodeToPlan(left), nodeToPlan(right))) + case Token("TOK_EXCEPT", left :: right :: Nil) => + Except(nodeToPlan(left), nodeToPlan(right)) + case Token("TOK_INTERSECT", left :: right :: Nil) => + Intersect(nodeToPlan(left), nodeToPlan(right)) + + case _ => + noParseRule("Plan", node) + } + + val allJoinTokens = "(TOK_.*JOIN)".r + val laterViewToken = "TOK_LATERAL_VIEW(.*)".r + protected def nodeToRelation(node: ASTNode): LogicalPlan = { + node match { + case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => + SubqueryAlias(cleanIdentifier(alias), nodeToPlan(query)) + + case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => + nodeToGenerate( + selectClause, + outer = isOuter.nonEmpty, + nodeToRelation(relationClause)) + + /* All relations, possibly with aliases or sampling clauses. */ + case Token("TOK_TABREF", clauses) => + // If the last clause is not a token then it's the alias of the table. + val (nonAliasClauses, aliasClause) = + if (clauses.last.text.startsWith("TOK")) { + (clauses, None) + } else { + (clauses.dropRight(1), Some(clauses.last)) + } + + val (Some(tableNameParts) :: + splitSampleClause :: + bucketSampleClause :: Nil) = { + getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), + nonAliasClauses) + } + + val tableIdent = extractTableIdent(tableNameParts) + val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } + val relation = UnresolvedRelation(tableIdent, alias) + + // Apply sampling if requested. + (bucketSampleClause orElse splitSampleClause).map { + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => + Limit(Literal(count.toInt), relation) + case Token("TOK_TABLESPLITSAMPLE", + Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) => + // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling + // function takes X PERCENT as the input and the range of X is [0, 100], we need to + // adjust the fraction. + require( + fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) + && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be on interval [0, 100]") + Sample(0.0, fraction.toDouble / 100, withReplacement = false, + (math.random * 1000).toInt, + relation) + case Token("TOK_TABLEBUCKETSAMPLE", + Token(numerator, Nil) :: + Token(denominator, Nil) :: Nil) => + val fraction = numerator.toDouble / denominator.toDouble + Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) + case a => + noParseRule("Sampling", a) + }.getOrElse(relation) + + case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => + if (!(other.size <= 1)) { + sys.error(s"Unsupported join operation: $other") + } + + val joinType = joinToken match { + case "TOK_JOIN" => Inner + case "TOK_CROSSJOIN" => Inner + case "TOK_RIGHTOUTERJOIN" => RightOuter + case "TOK_LEFTOUTERJOIN" => LeftOuter + case "TOK_FULLOUTERJOIN" => FullOuter + case "TOK_LEFTSEMIJOIN" => LeftSemi + case "TOK_UNIQUEJOIN" => noParseRule("Unique Join", node) + case "TOK_ANTIJOIN" => noParseRule("Anti Join", node) + case "TOK_NATURALJOIN" => NaturalJoin(Inner) + case "TOK_NATURALRIGHTOUTERJOIN" => NaturalJoin(RightOuter) + case "TOK_NATURALLEFTOUTERJOIN" => NaturalJoin(LeftOuter) + case "TOK_NATURALFULLOUTERJOIN" => NaturalJoin(FullOuter) + } + Join(nodeToRelation(relation1), + nodeToRelation(relation2), + joinType, + other.headOption.map(nodeToExpr)) + + case _ => + noParseRule("Relation", node) + } + } + + protected def nodeToSortOrder(node: ASTNode): SortOrder = node match { + case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => + SortOrder(nodeToExpr(sortExpr), Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => + SortOrder(nodeToExpr(sortExpr), Descending) + case _ => + noParseRule("SortOrder", node) + } + + val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r + protected def nodeToDest( + node: ASTNode, + query: LogicalPlan, + overwrite: Boolean): LogicalPlan = node match { + case Token(destinationToken(), + Token("TOK_DIR", + Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => + query + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.children.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable( + UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.children.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable( + UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = true) + + case _ => + noParseRule("Destination", node) + } + + protected def selExprNodeToExpr(node: ASTNode): Option[Expression] = node match { + case Token("TOK_SELEXPR", e :: Nil) => + Some(nodeToExpr(e)) + + case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => + Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) + + case Token("TOK_SELEXPR", e :: aliasChildren) => + val aliasNames = aliasChildren.collect { + case Token(name, Nil) => cleanIdentifier(name) + } + Some(MultiAlias(nodeToExpr(e), aliasNames)) + + /* Hints are ignored */ + case Token("TOK_HINTLIST", _) => None + + case _ => + noParseRule("Select", node) + } + + /* 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) => + UnresolvedAttribute.quoted(cleanIdentifier(name)) + case Token(".", qualifier :: Token(attr, Nil) :: Nil) => + nodeToExpr(qualifier) match { + case UnresolvedAttribute(nameParts) => + UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) + case other => UnresolvedExtractValue(other, Literal(cleanIdentifier(attr))) + } + case Token("TOK_SUBQUERY_EXPR", Token("TOK_SUBQUERY_OP", Nil) :: subquery :: Nil) => + ScalarSubquery(nodeToPlan(subquery)) + + /* Stars (*) */ + case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) + // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only + // has a single child which is tableName. + case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", target) :: Nil) if target.nonEmpty => + UnresolvedStar(Some(target.map(x => cleanIdentifier(x.text)))) + + /* Aggregate Functions */ + case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => + Count(args.map(nodeToExpr)).toAggregateExpression(isDistinct = true) + case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => + Count(Literal(1)).toAggregateExpression() + + /* Casts */ + case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), IntegerType) + case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), LongType) + case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), FloatType) + case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DoubleType) + case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ShortType) + case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), ByteType) + case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), BinaryType) + case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), BooleanType) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: scale :: nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, scale.text.toInt)) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, 0)) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType.USER_DEFAULT) + case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), TimestampType) + case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DateType) + + /* Arithmetic */ + case Token("+", child :: Nil) => nodeToExpr(child) + case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) + case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) + case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) + case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) + case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) + case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token(DIV(), left :: right:: Nil) => + Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) + case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) + case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right)) + case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right)) + case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right)) + + /* Comparisons */ + case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right)) + case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) + case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) + case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) + case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) + case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) + case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) + case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => + IsNotNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => + IsNull(nodeToExpr(child)) + case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => + In(nodeToExpr(value), list.map(nodeToExpr)) + case Token("TOK_FUNCTION", + Token(BETWEEN(), Nil) :: + kw :: + target :: + minValue :: + maxValue :: Nil) => + + val targetExpression = nodeToExpr(target) + val betweenExpr = + And( + GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), + LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) + kw match { + case Token("KW_FALSE", Nil) => betweenExpr + case Token("KW_TRUE", Nil) => Not(betweenExpr) + } + + /* Boolean Logic */ + case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) + case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) + case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) + case Token("!", child :: Nil) => Not(nodeToExpr(child)) + + /* Case statements */ + case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => + CaseWhen.createFromParser(branches.map(nodeToExpr)) + case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => + val keyExpr = nodeToExpr(branches.head) + CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) + + /* Complex datatype manipulation */ + case Token("[", child :: ordinal :: Nil) => + UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) + + /* Window Functions */ + case Token(text, args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = nodeToExpr(node.copy(children = node.children.init)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + + /* UDFs - Must be last otherwise will preempt built in functions */ + case Token("TOK_FUNCTION", Token(name, Nil) :: args) => + UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = false) + // Aggregate function with DISTINCT keyword. + case Token("TOK_FUNCTIONDI", Token(name, Nil) :: args) => + UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = true) + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => + UnresolvedFunction(name, UnresolvedStar(None) :: Nil, isDistinct = false) + + /* Literals */ + case Token("TOK_NULL", Nil) => Literal.create(null, NullType) + case Token(TRUE(), Nil) => Literal.create(true, BooleanType) + case Token(FALSE(), Nil) => Literal.create(false, BooleanType) + case Token("TOK_STRINGLITERALSEQUENCE", strings) => + Literal(strings.map(s => ParseUtils.unescapeSQLString(s.text)).mkString) + + case ast if ast.tokenType == SparkSqlParser.TinyintLiteral => + Literal.create(ast.text.substring(0, ast.text.length() - 1).toByte, ByteType) + + case ast if ast.tokenType == SparkSqlParser.SmallintLiteral => + Literal.create(ast.text.substring(0, ast.text.length() - 1).toShort, ShortType) + + case ast if ast.tokenType == SparkSqlParser.BigintLiteral => + Literal.create(ast.text.substring(0, ast.text.length() - 1).toLong, LongType) + + case ast if ast.tokenType == SparkSqlParser.DoubleLiteral => + Literal(ast.text.toDouble) + + case ast if ast.tokenType == SparkSqlParser.Number => + val text = ast.text + text match { + case INTEGRAL() => + BigDecimal(text) match { + case v if v.isValidInt => + Literal(v.intValue()) + case v if v.isValidLong => + Literal(v.longValue()) + case v => Literal(v.underlying()) + } + case DECIMAL(_*) => + Literal(BigDecimal(text).underlying()) + case _ => + // Convert a scientifically notated decimal into a double. + Literal(text.toDouble) + } + case ast if ast.tokenType == SparkSqlParser.StringLiteral => + Literal(ParseUtils.unescapeSQLString(ast.text)) + + case ast if ast.tokenType == SparkSqlParser.TOK_DATELITERAL => + Literal(Date.valueOf(ast.text.substring(1, ast.text.length - 1))) + + case ast if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL => + Literal(CalendarInterval.fromYearMonthString(ast.children.head.text)) + + case ast if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL => + Literal(CalendarInterval.fromDayTimeString(ast.children.head.text)) + + case Token("TOK_INTERVAL", elements) => + var interval = new CalendarInterval(0, 0) + var updated = false + elements.foreach { + // The interval node will always contain children for all possible time units. A child node + // is only useful when it contains exactly one (numeric) child. + case e @ Token(name, Token(value, Nil) :: Nil) => + val unit = name match { + case "TOK_INTERVAL_YEAR_LITERAL" => "year" + case "TOK_INTERVAL_MONTH_LITERAL" => "month" + case "TOK_INTERVAL_WEEK_LITERAL" => "week" + case "TOK_INTERVAL_DAY_LITERAL" => "day" + case "TOK_INTERVAL_HOUR_LITERAL" => "hour" + case "TOK_INTERVAL_MINUTE_LITERAL" => "minute" + case "TOK_INTERVAL_SECOND_LITERAL" => "second" + case "TOK_INTERVAL_MILLISECOND_LITERAL" => "millisecond" + case "TOK_INTERVAL_MICROSECOND_LITERAL" => "microsecond" + case _ => noParseRule(s"Interval($name)", e) + } + interval = interval.add(CalendarInterval.fromSingleUnitString(unit, value)) + updated = true + case _ => + } + if (!updated) { + throw new AnalysisException("at least one time unit should be given for interval literal") + } + Literal(interval) + + case _ => + noParseRule("Expression", node) + } + + /* Case insensitive matches for Window Specification */ + val PRECEDING = "(?i)preceding".r + val FOLLOWING = "(?i)following".r + val CURRENT = "(?i)current".r + protected def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.children) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.children.map(nodeToExpr), + orderByExpr.children.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.children.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.children.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.children.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + noParseRule("Partition & Ordering", partitionAndOrdering) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: ASTNode): FrameBoundary = node match { + case Token(PRECEDING(), Token(count, Nil) :: Nil) => + if (count.toLowerCase() == "unbounded") { + UnboundedPreceding + } else { + ValuePreceding(count.toInt) + } + case Token(FOLLOWING(), Token(count, Nil) :: Nil) => + if (count.toLowerCase() == "unbounded") { + UnboundedFollowing + } else { + ValueFollowing(count.toInt) + } + case Token(CURRENT(), Nil) => CurrentRow + case _ => + noParseRule("Window Frame Boundary", node) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.children match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + noParseRule("Window Frame", frame) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } + + protected def nodeToTransformation( + node: ASTNode, + child: LogicalPlan): Option[ScriptTransformation] = None + + val explode = "(?i)explode".r + val jsonTuple = "(?i)json_tuple".r + protected def nodeToGenerate(node: ASTNode, outer: Boolean, child: LogicalPlan): Generate = { + val Token("TOK_SELECT", Token("TOK_SELEXPR", clauses) :: Nil) = node + + val alias = cleanIdentifier(getClause("TOK_TABALIAS", clauses).children.head.text) + + val generator = clauses.head match { + case Token("TOK_FUNCTION", Token(explode(), Nil) :: childNode :: Nil) => + Explode(nodeToExpr(childNode)) + case Token("TOK_FUNCTION", Token(jsonTuple(), Nil) :: children) => + JsonTuple(children.map(nodeToExpr)) + case other => + nodeToGenerator(other) + } + + val attributes = clauses.collect { + case Token(a, Nil) => UnresolvedAttribute(a.toLowerCase) + } + + Generate(generator, join = true, outer = outer, Some(alias.toLowerCase), attributes, child) + } + + protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala new file mode 100644 index 0000000000000..7f35d650b9571 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala @@ -0,0 +1,36 @@ +/* + * 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.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * Interface for a parser. + */ +trait ParserInterface { + /** Creates LogicalPlan for a given SQL string. */ + def parsePlan(sqlText: String): LogicalPlan + + /** Creates Expression for a given SQL string. */ + def parseExpression(sqlText: String): Expression + + /** Creates TableIdentifier for a given SQL string. */ + def parseTableIdentifier(sqlText: String): TableIdentifier +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala new file mode 100644 index 0000000000000..10a7f6ff98694 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala @@ -0,0 +1,134 @@ +/* + * 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.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.types._ + +object ParserSupport { + protected val escapedIdentifier = "`(.+)`".r + protected val doubleQuotedString = "\"([^\"]+)\"".r + protected val singleQuotedString = "'([^']+)'".r + + private[sql] def unquoteString(str: String): String = str match { + case singleQuotedString(s) => s + case doubleQuotedString(s) => s + case other => other + } + + /** Strips backticks from ident if present */ + private[sql] def cleanIdentifier(ident: String): String = ident match { + case escapedIdentifier(i) => i + case plainIdent => plainIdent + } + + private[sql] def cleanAndUnquoteString(str: String): String = { + cleanIdentifier(unquoteString(str)) + } + + object Token { + def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { + CurrentOrigin.setPosition(node.line, node.positionInLine) + node.pattern + } + } + + 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) + } + + 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/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index 42147f516f964..f8edd1f644718 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.parser.CatalystQl import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.BooleanType 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 64950a95ee53a..9c64e8d597870 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 @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ 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 00125c5a2bd8a..089e1baab9508 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,10 +19,10 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.execution.commands._ @@ -30,28 +30,23 @@ 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 ParserSupport._ + /** Check if a command should not be explained. */ protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command - protected def extractProps(node: ASTNode): Seq[(String, String)] = node match { - case Token("TOK_TABLEOPTIONS", options) => + protected def extractProps( + node: ASTNode, + firstLevelNodeStr: String, + secondLevelNodeStr: String): Seq[(String, String)] = node match { + case Token(firstLevelNodeStr, options) => options.map { - case Token("TOK_TABLEOPTION", keysAndValue) => - val key = keysAndValue.init.map(_.text).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - } - - case Token("TOK_DBPROPLIST", props) => - props.map { - case Token("TOK_TABLEPROPERTY", keysAndValue) => + case Token(secondLevelNodeStr, keysAndValue) => val key = keysAndValue.init.map(x => unquoteString(x.text)).mkString(".") val value = unquoteString(keysAndValue.last.text) (key, value) } - - case _ => Seq.empty } protected override def nodeToPlan(node: ASTNode): LogicalPlan = { @@ -106,7 +101,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly } val props: Map[String, String] = dbprops.toSeq.flatMap { case Token("TOK_DATABASEPROPERTIES", propList) => - propList.flatMap(extractProps) + propList.flatMap(extractProps(_, "TOK_DBPROPLIST", "TOK_TABLEPROPERTY")) }.toMap CreateDataBase(databaseName, allowExisting.isDefined, location, comment, props)(node.source) @@ -135,8 +130,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly CreateFunction(funcName, asName, resourcesMap, temp.isDefined)(node.source) case Token("TOK_ALTERTABLE", alterTableArgs) => - AlterTableCommandParser(this).applyOrElse(node, - (node: ASTNode) => throw new NotImplementedError(node.text)) + AlterTableCommandParser.parse(node) case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( @@ -164,7 +158,8 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case Token(name, Nil) => name }.mkString(".") - val options: Map[String, String] = tableOpts.toSeq.flatMap(extractProps).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/commands/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala index 55261206c9630..6b38e6be26e62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala @@ -31,15 +31,6 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ -abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.catalog.runNativeCommand(sql) - } - - override val output: Seq[Attribute] = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging { private def keyValueOutput: Seq[Attribute] = { @@ -49,8 +40,47 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm schema.toAttributes } + private val deprecatedProperties: Map[String, SQLContext => Seq[Row]] = + Seq((SQLConf.Deprecated.EXTERNAL_SORT, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.EXTERNAL_SORT} is deprecated and will be ignored. " + + s"External sort will continue to be used.") + Seq(Row(SQLConf.Deprecated.EXTERNAL_SORT, "true")) + }), + (SQLConf.Deprecated.USE_SQL_AGGREGATE2, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} is deprecated and " + + s"will be ignored. ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} will " + + s"continue to be true.") + Seq(Row(SQLConf.Deprecated.USE_SQL_AGGREGATE2, "true")) + }), + (SQLConf.Deprecated.TUNGSTEN_ENABLED, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.TUNGSTEN_ENABLED} is deprecated and " + + s"will be ignored. Tungsten will continue to be used.") + Seq(Row(SQLConf.Deprecated.TUNGSTEN_ENABLED, "true")) + }), + (SQLConf.Deprecated.CODEGEN_ENABLED, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.CODEGEN_ENABLED} is deprecated and " + + s"will be ignored. Codegen will continue to be used.") + Seq(Row(SQLConf.Deprecated.CODEGEN_ENABLED, "true")) + }), + (SQLConf.Deprecated.UNSAFE_ENABLED, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.UNSAFE_ENABLED} is deprecated and " + + s"will be ignored. Unsafe mode will continue to be used.") + Seq(Row(SQLConf.Deprecated.UNSAFE_ENABLED, "true")) + }), + (SQLConf.Deprecated.SORTMERGE_JOIN, (sqlContext: SQLContext) => { + logWarning( + s"Property ${SQLConf.Deprecated.SORTMERGE_JOIN} is deprecated and " + + s"will be ignored. Sort merge join will continue to be used.") + Seq(Row(SQLConf.Deprecated.SORTMERGE_JOIN, "true")) + }) + ).toMap + private val (_output, runFunc): (Seq[Attribute], SQLContext => Seq[Row]) = kv match { - // Configures the deprecated "mapred.reduce.tasks" property. case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => val runFunc = (sqlContext: SQLContext) => { logWarning( @@ -67,61 +97,9 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm } } (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.EXTERNAL_SORT, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.EXTERNAL_SORT} is deprecated and will be ignored. " + - s"External sort will continue to be used.") - Seq(Row(SQLConf.Deprecated.EXTERNAL_SORT, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.USE_SQL_AGGREGATE2, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} is deprecated and " + - s"will be ignored. ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} will " + - s"continue to be true.") - Seq(Row(SQLConf.Deprecated.USE_SQL_AGGREGATE2, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.TUNGSTEN_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.TUNGSTEN_ENABLED} is deprecated and " + - s"will be ignored. Tungsten will continue to be used.") - Seq(Row(SQLConf.Deprecated.TUNGSTEN_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.CODEGEN_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.CODEGEN_ENABLED} is deprecated and " + - s"will be ignored. Codegen will continue to be used.") - Seq(Row(SQLConf.Deprecated.CODEGEN_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.UNSAFE_ENABLED, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.UNSAFE_ENABLED} is deprecated and " + - s"will be ignored. Unsafe mode will continue to be used.") - Seq(Row(SQLConf.Deprecated.UNSAFE_ENABLED, "true")) - } - (keyValueOutput, runFunc) - - case Some((SQLConf.Deprecated.SORTMERGE_JOIN, Some(value))) => - val runFunc = (sqlContext: SQLContext) => { - logWarning( - s"Property ${SQLConf.Deprecated.SORTMERGE_JOIN} is deprecated and " + - s"will be ignored. Sort merge join will continue to be used.") - Seq(Row(SQLConf.Deprecated.SORTMERGE_JOIN, "true")) - } - (keyValueOutput, runFunc) + + case Some((key, Some(value))) if deprecatedProperties.contains(key) => + (keyValueOutput, deprecatedProperties(key)) // Configures a single property. case Some((key, Some(value))) => @@ -384,150 +362,3 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { override val output: Seq[Attribute] = Seq.empty } - -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, - 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/commands/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala new file mode 100644 index 0000000000000..b6f4a135e2db0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/ddl.scala @@ -0,0 +1,188 @@ +/* + * 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.commands + +import java.util.NoSuchElementException + +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + +abstract class NativeDDLCommands(val sql: String) extends RunnableCommand { + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.runNativeCommand(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, + 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/commands/parsers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala index 0a579a0c25c12..17ca19056960c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala @@ -20,27 +20,27 @@ package org.apache.spark.sql.execution.commands import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} -import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} +import org.apache.spark.sql.catalyst.parser.{ASTNode, CatalystQl, ParserConf, ParserSupport, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.execution.commands._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType -case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { +object AlterTableCommandParser { + import ParserSupport._ 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) => - (unquoteString(cleanIdentifier(ident.text)), - Some(unquoteString(cleanIdentifier(constant.text)))) + (cleanAndUnquoteString(ident.text), Some(cleanAndUnquoteString(constant.text))) case Token("TOK_PARTVAL", ident :: Nil) => - (unquoteString(cleanIdentifier(ident.text)), None) + (cleanAndUnquoteString(ident.text), None) }.toMap Some(spec) case _ => None @@ -53,24 +53,22 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { case Token("TOK_TABLEPROPLIST", props) => props.map { case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) :: Nil) => - val k = unquoteString(cleanIdentifier(key.text)) + val k = cleanAndUnquoteString(key.text) (k, None) case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => - val k = unquoteString(cleanIdentifier(key.text)) - val v = unquoteString(cleanIdentifier(value.text)) + val k = cleanAndUnquoteString(key.text) + val v = cleanAndUnquoteString(value.text) (k, Some(v)) } }.toMap } - override def isDefinedAt(node: ASTNode): Boolean = node.text == "TOK_ALTERTABLE" - - override def apply(v1: ASTNode): LogicalPlan = v1.children match { - case (tabName @ Token("TOK_TABNAME", _)) :: rest => - val tableIdent: TableIdentifier = base.extractTableIdent(tabName) - val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children) + 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, rest, tableIdent, partition) + matchAlterTableCommands(v1, restNodes, tableIdent, partition) case _ => throw new NotImplementedError(v1.text) } @@ -80,43 +78,36 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { nodes: Seq[ASTNode], tableIdent: TableIdentifier, partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes match { - case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest => - val renamedTable = base.getClause("TOK_TABNAME", renameArgs) - val renamedTableIdent: TableIdentifier = base.extractTableIdent(renamedTable) + 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) :: rest => + case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => val setTableProperties = extractTableProps(args.head) AlterTableSetProperties( tableIdent, setTableProperties)(node.source) - case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest => + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => val dropTableProperties = extractTableProps(args.head) - val allowExisting = base.getClauseOption("TOK_IFEXISTS", args) + val allowExisting = getClauseOption("TOK_IFEXISTS", args) AlterTableDropProperties( tableIdent, dropTableProperties, allowExisting.isDefined)(node.source) - case Token("TOK_ALTERTABLE_SERIALIZER", serdeArgs) :: rest => - val serdeClassName = unquoteString(cleanIdentifier(serdeArgs.head.text)) - - val serdeProperties: Option[Map[String, Option[String]]] = Option( - // SET SERDE serde_classname WITH SERDEPROPERTIES - if (serdeArgs.tail.isEmpty) { - null - } else { - extractTableProps(serdeArgs.tail.head) - } - ) + 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(serdeClassName), + Some(cleanAndUnquoteString(serdeClassName)), serdeProperties, partition)(node.source) - case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: rest => + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => val serdeProperties: Map[String, Option[String]] = extractTableProps(args.head) AlterTableSerDeProperties( @@ -125,34 +116,31 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { Some(serdeProperties), partition)(node.source) - case (bucketSpec @ Token("TOK_ALTERTABLE_CLUSTER_SORT", _)) :: rest => - val (buckets, noClustered, noSorted) = bucketSpec match { - case Token("TOK_ALTERTABLE_CLUSTER_SORT", clusterAndSoryByArgs :: Nil) => - clusterAndSoryByArgs match { - case Token("TOK_ALTERTABLE_BUCKETS", bucketArgs) => - val bucketCols = bucketArgs.head.children.map(_.text) - - val (sortCols, sortDirections, numBuckets) = { - if (bucketArgs(1).text == "TOK_TABCOLNAME") { - val cols = bucketArgs(1).children.map { - case Token("TOK_TABSORTCOLNAMEASC", Token(colName, Nil) :: Nil) => - (colName, Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", Token(colName, Nil) :: Nil) => - (colName, Descending) - } - (cols.map(_._1), cols.map(_._2), bucketArgs(2).text.toInt) - } else { - (Nil, Nil, bucketArgs(1).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) + 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( @@ -161,7 +149,7 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { noClustered, noSorted)(node.source) - case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: rest => + case Token("TOK_ALTERTABLE_BUCKETS", Token(bucketNum, Nil) :: Nil) :: _ => val num = bucketNum.toInt val buckets = Some(BucketSpec(num, Nil, Nil, Nil)) AlterTableStoreProperties( @@ -170,32 +158,29 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { false, false)(node.source) - case (tableSkewed @ Token("TOK_ALTERTABLE_SKEWED", _)) :: rest => - // Alter Table not skewed - // Token("TOK_ALTERTABLE_SKEWED", Nil) means not skewed. - val notSkewed = if (tableSkewed.children.size == 0) { - true - } else { - false - } + case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => + // ALTER TABLE table_name NOT SKEWED + AlterTableSkewed(tableIdent, Nil, Nil, false, true, false)(node.source) - val (notStoredAsDirs, skewedArgs) = tableSkewed match { - case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) => - // Alter Table not stored as directories - (true, None) + 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) => - val (cols, values, storedAsDirs) = skewedArgs match { + skewedArgs match { case Token("TOK_TABLESKEWED", skewedCols :: skewedValues :: stored) => - val cols = skewedCols.children.map(n => unquoteString(cleanIdentifier(n.text))) + val cols = skewedCols.children.map(n => cleanAndUnquoteString(n.text)) val values = skewedValues match { case Token("TOK_TABCOLVALUE", values) => - Seq(values.map(n => unquoteString(cleanIdentifier(n.text)))) + 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 => unquoteString(cleanIdentifier(n.text))) + vals.map(n => cleanAndUnquoteString(n.text)) } } } @@ -207,47 +192,36 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { (cols, values, storedAsDirs) } - (false, Some((cols, values, storedAsDirs))) } - if (skewedArgs.isDefined) { - AlterTableSkewed( - tableIdent, - skewedArgs.get._1, /* cols */ - skewedArgs.get._2, /* values */ - skewedArgs.get._3, /* storedAsDirs */ - notSkewed, notStoredAsDirs)(node.source) - } else { - AlterTableSkewed(tableIdent, Nil, Nil, false, notSkewed, notStoredAsDirs)(node.source) - } - - case Token("TOK_ALTERTABLE_SKEWED_LOCATION", args) :: rest => - val skewedMaps = args(0) match { - case Token("TOK_SKEWED_LOCATIONS", locationList :: Nil) => - locationList match { - case Token("TOK_SKEWED_LOCATION_LIST", locationMaps) => - locationMaps.map { - case Token("TOK_SKEWED_LOCATION_MAP", key :: value :: Nil) => - val k = key match { - case Token(const, Nil) => Seq(unquoteString(cleanIdentifier(const))) - case Token("TOK_TABCOLVALUES", values :: Nil) => - values match { - case Token("TOK_TABCOLVALUE", vals) => - vals.map(n => unquoteString(cleanIdentifier(n.text))) - } - } - (k, unquoteString(cleanIdentifier(value.text))) - }.toMap + AlterTableSkewed( + tableIdent, + skewedArgs._1, /* cols */ + skewedArgs._2, /* values */ + skewedArgs._3, /* storedAsDirs */ + false, 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) :: rest => - val allowExisting = base.getClauseOption("TOK_IFNOTEXISTS", addPartsArgs) - val parts = if (allowExisting.isDefined) { - addPartsArgs.tail - } else { - addPartsArgs + 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])] = @@ -273,82 +247,79 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { if (currentPart != null) { partitions += ((currentPart, None)) } - AlterTableAddPartition(tableIdent, partitions, allowExisting.isDefined)(node.source) + AlterTableAddPartition(tableIdent, partitions, allowExisting)(node.source) - case Token("TOK_ALTERTABLE_RENAMEPART", args) :: rest => - val newPartition = parsePartitionSpec(args(0)) - AlterTableRenamePartition(tableIdent, partition.get, newPartition.get)(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", args) :: rest => - val Seq(Some(partSpec), Some(fromTable)) = - base.getClauses(Seq("TOK_PARTSPEC", "TOK_TABNAME"), args) - val partition = parsePartitionSpec(partSpec).get - val fromTableIdent = base.extractTableIdent(fromTable) + 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) :: rest => + 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) => - (unquoteString(cleanIdentifier(ident.text)), - op.text, unquoteString(cleanIdentifier(constant.text))) + (cleanAndUnquoteString(ident.text), + op.text, cleanAndUnquoteString(constant.text)) } } - val allowExisting = base.getClauseOption("TOK_IFEXISTS", args) - val purge = base.getClauseOption("PURGE", args) + val allowExisting = getClauseOption("TOK_IFEXISTS", args).isDefined + + val purge = getClauseOption("PURGE", args) - val replication = base.getClauseOption("TOK_REPLICATION", args).map { - case Token("TOK_REPLICATION", replId :: metadata :: Nil) => - (unquoteString(cleanIdentifier(replId.text)), true) - case Token("TOK_REPLICATION", replId :: Nil) => - (unquoteString(cleanIdentifier(replId.text)), false) + val replication = getClauseOption("TOK_REPLICATION", args).map { + case Token("TOK_REPLICATION", replId :: metadata) => + (cleanAndUnquoteString(replId.text), metadata.nonEmpty) } AlterTableDropPartition( tableIdent, parts, - allowExisting.isDefined, + allowExisting, purge.isDefined, replication)(node.source) - case Token("TOK_ALTERTABLE_ARCHIVE", args) :: rest => - val partition = parsePartitionSpec(args(0)).get + case Token("TOK_ALTERTABLE_ARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) AlterTableArchivePartition(tableIdent, partition)(node.source) - case Token("TOK_ALTERTABLE_UNARCHIVE", args) :: rest => - val partition = parsePartitionSpec(args(0)).get + case Token("TOK_ALTERTABLE_UNARCHIVE", partArg :: Nil) :: _ => + val Some(partition) = parsePartitionSpec(partArg) AlterTableUnarchivePartition(tableIdent, partition)(node.source) - case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: rest => + case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => val Seq(fileFormat, genericFormat) = - base.getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), args) - val fFormat = fileFormat.map(_.children.map(n => unquoteString(cleanIdentifier(n.text)))) - val gFormat = genericFormat.map(f => unquoteString(cleanIdentifier(f.children(0).text))) + 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) :: rest => - AlterTableSetLocation(tableIdent, partition, unquoteString(cleanIdentifier(loc)))(node.source) + case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => + AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) - case Token("TOK_ALTERTABLE_TOUCH", args) :: rest => - val part = base.getClauseOption("TOK_PARTSPEC", args).flatMap(parsePartitionSpec) + 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) :: rest => - AlterTableCompact(tableIdent, partition, - unquoteString(cleanIdentifier(compactType)))(node.source) + case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => + AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) - case Token("TOK_ALTERTABLE_MERGEFILES", _) :: rest => + case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => AlterTableMerge(tableIdent, partition)(node.source) - case Token("TOK_ALTERTABLE_RENAMECOL", args) :: rest => + case Token("TOK_ALTERTABLE_RENAMECOL", args) :: _ => val oldName = args(0).text val newName = args(1).text - val dataType = base.nodeToDataType(args(2)) + val dataType = nodeToDataType(args(2)) val afterPos = - base.getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args) val afterPosCol = afterPos.map { ap => ap.children match { case Token(col, Nil) :: Nil => col @@ -356,15 +327,15 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { } } - val restrict = base.getClauseOption("TOK_RESTRICT", args) - val cascade = base.getClauseOption("TOK_CASCADE", args) + 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(unquoteString(cleanIdentifier(commentStr))) + Some(cleanAndUnquoteString(commentStr)) case _ => None } @@ -384,14 +355,14 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { restrict.isDefined, cascade.isDefined)(node.source) - case Token("TOK_ALTERTABLE_ADDCOLS", args) :: rest => - val tableCols = base.getClause("TOK_TABCOLLIST", args) + case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(base.nodeToStructField)) + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) } - val restrict = base.getClauseOption("TOK_RESTRICT", args) - val cascade = base.getClauseOption("TOK_CASCADE", args) + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) AlterTableAddCol( tableIdent, @@ -400,14 +371,14 @@ case class AlterTableCommandParser(base: CatalystQl) extends PlanParser { restrict.isDefined, cascade.isDefined)(node.source) - case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: rest => - val tableCols = base.getClause("TOK_TABCOLLIST", args) + case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => + val tableCols = getClause("TOK_TABCOLLIST", args) val columns = tableCols match { - case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(base.nodeToStructField)) + case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) } - val restrict = base.getClauseOption("TOK_RESTRICT", args) - val cascade = base.getClauseOption("TOK_CASCADE", args) + val restrict = getClauseOption("TOK_RESTRICT", args) + val cascade = getClauseOption("TOK_CASCADE", args) AlterTableReplaceCol( tableIdent, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 97c6992e18753..aeda108152c23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,11 +22,12 @@ import scala.reflect.runtime.universe.{typeTag, TypeTag} import scala.util.Try import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.{CatalystQl, ScalaReflection} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedFunction} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.parser.CatalystQl import org.apache.spark.sql.catalyst.plans.logical.BroadcastHint import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DDLCommandSuite.scala new file mode 100644 index 0000000000000..91e175a4a8626 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DDLCommandSuite.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 + +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.commands._ +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + +class DDLCommandSuite 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 e5d1cac3d0815..7bd0f570d62ad 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 @@ -42,10 +42,11 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql._ import org.apache.spark.sql.SQLConf.SQLConfEntry import org.apache.spark.sql.SQLConf.SQLConfEntry._ -import org.apache.spark.sql.catalyst.{InternalRow, ParserInterface} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.commands._ 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 7d6a751426a84..a1e0b9813501e 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 @@ -81,6 +81,8 @@ 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 ParserSupport._ + protected val nativeCommands = Seq( "TOK_ALTERDATABASE_OWNER", "TOK_ALTERDATABASE_PROPERTIES", From 0c57651f1be2b6a5e04532d49ad504d2b8038ddd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 22 Feb 2016 10:09:46 +0000 Subject: [PATCH 12/13] Delete moved files. --- .../spark/sql/catalyst/CatalystQl.scala | 995 ------------------ .../spark/sql/catalyst/ParserInterface.scala | 35 - .../spark/sql/catalyst/PlanParser.scala | 53 - .../spark/sql/execution/SparkQlSuite.scala | 652 ------------ 4 files changed, 1735 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserInterface.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala deleted file mode 100644 index 55e53d6abf55f..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ /dev/null @@ -1,995 +0,0 @@ -/* - * 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 - -import java.sql.Date - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.Count -import org.apache.spark.sql.catalyst.parser._ -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 - -abstract class BaseParser(val conf: ParserConf) extends ParserInterface with ParserBase { - val planParsers: Seq[PlanParser] = Nil - - lazy val planParser: PlanParser = planParsers.reduce(_.orElse(_).asInstanceOf[PlanParser]) - - protected def nodeToPlan(node: ASTNode): LogicalPlan = { - planParser.applyOrElse(node, throw new NotImplementedError(node.text)) - } -} - -/** - * This class translates SQL to Catalyst [[LogicalPlan]]s or [[Expression]]s. - */ -private[sql] class CatalystQl(conf: ParserConf = SimpleParserConf()) extends BaseParser(conf) { - /** - * The safeParse method allows a user to focus on the parsing/AST transformation logic. This - * method will take care of possible errors during the parsing process. - */ - protected def safeParse[T](sql: String, ast: ASTNode)(toResult: ASTNode => T): T = { - try { - toResult(ast) - } catch { - case e: MatchError => throw e - case e: AnalysisException => throw e - case e: Exception => - throw new AnalysisException(e.getMessage) - case e: NotImplementedError => - throw new AnalysisException( - s"""Unsupported language features in query - |== SQL == - |$sql - |== AST == - |${ast.treeString} - |== Error == - |$e - |== Stacktrace == - |${e.getStackTrace.head} - """.stripMargin) - } - } - - /** Creates LogicalPlan for a given SQL string. */ - def parsePlan(sql: String): LogicalPlan = - safeParse(sql, ParseDriver.parsePlan(sql, conf))(nodeToPlan) - - /** Creates Expression for a given SQL string. */ - def parseExpression(sql: String): Expression = - safeParse(sql, ParseDriver.parseExpression(sql, conf))(selExprNodeToExpr(_).get) - - /** Creates TableIdentifier for a given SQL string. */ - 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)) - } - } - - 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 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) - } - - 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) - } - - 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 - * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 - * Check the following link for details. - * -https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup - * - * The bitmask denotes the grouping expressions validity for a grouping set, - * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 2: k1, bit 1: k2, and bit 0: k3), the grouping id of - * GROUPING SETS (k1, k2) and (k2) should be 1 and 5 respectively. - */ - protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { - val (keyASTs, setASTs) = children.partition { - case Token("TOK_GROUPING_SETS_EXPRESSION", _) => false // grouping sets - case _ => true // grouping keys - } - - val keys = keyASTs.map(nodeToExpr) - val keyMap = keyASTs.zipWithIndex.toMap - - val mask = (1 << keys.length) - 1 - val bitmasks: Seq[Int] = setASTs.map { - case Token("TOK_GROUPING_SETS_EXPRESSION", columns) => - columns.foldLeft(mask)((bitmap, col) => { - val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2).getOrElse( - throw new AnalysisException(s"${col.treeString} doesn't show up in the GROUP BY list")) - // 0 means that the column at the given index is a grouping column, 1 means it is not, - // so we unset the bit in bitmap. - bitmap & ~(1 << (keys.length - 1 - keyIndex)) - }) - case _ => sys.error("Expect GROUPING SETS clause") - } - - (keys, bitmasks) - } - - override protected def nodeToPlan(node: ASTNode): LogicalPlan = node match { - case Token("TOK_SHOWFUNCTIONS", args) => - // Skip LIKE. - val pattern = args match { - case like :: nodes if like.text.toUpperCase == "LIKE" => nodes - case nodes => nodes - } - - // Extract Database and Function name - pattern match { - case Nil => - ShowFunctions(None, None) - case Token(name, Nil) :: Nil => - ShowFunctions(None, Some(unquoteString(cleanIdentifier(name)))) - case Token(db, Nil) :: Token(name, Nil) :: Nil => - ShowFunctions(Some(unquoteString(cleanIdentifier(db))), - Some(unquoteString(cleanIdentifier(name)))) - case _ => - noParseRule("SHOW FUNCTIONS", node) - } - - case Token("TOK_DESCFUNCTION", Token(functionName, Nil) :: isExtended) => - DescribeFunction(cleanIdentifier(functionName), isExtended.nonEmpty) - - case Token("TOK_QUERY", queryArgs @ Token("TOK_CTE" | "TOK_FROM" | "TOK_INSERT", _) :: _) => - val (fromClause: Option[ASTNode], insertClauses, cteRelations) = - queryArgs match { - case Token("TOK_CTE", ctes) :: Token("TOK_FROM", from) :: inserts => - val cteRelations = ctes.map { node => - val relation = nodeToRelation(node).asInstanceOf[SubqueryAlias] - relation.alias -> relation - } - (Some(from.head), inserts, Some(cteRelations.toMap)) - case Token("TOK_FROM", from) :: inserts => - (Some(from.head), inserts, None) - case Token("TOK_INSERT", _) :: Nil => - (None, queryArgs, None) - } - - // Return one query for each insert clause. - val queries = insertClauses.map { - case Token("TOK_INSERT", singleInsert) => - val ( - intoClause :: - destClause :: - selectClause :: - selectDistinctClause :: - whereClause :: - groupByClause :: - rollupGroupByClause :: - cubeGroupByClause :: - groupingSetsClause :: - orderByClause :: - havingClause :: - sortByClause :: - clusterByClause :: - distributeByClause :: - limitClause :: - lateralViewClause :: - windowClause :: Nil) = { - getClauses( - Seq( - "TOK_INSERT_INTO", - "TOK_DESTINATION", - "TOK_SELECT", - "TOK_SELECTDI", - "TOK_WHERE", - "TOK_GROUPBY", - "TOK_ROLLUP_GROUPBY", - "TOK_CUBE_GROUPBY", - "TOK_GROUPING_SETS", - "TOK_ORDERBY", - "TOK_HAVING", - "TOK_SORTBY", - "TOK_CLUSTERBY", - "TOK_DISTRIBUTEBY", - "TOK_LIMIT", - "TOK_LATERAL_VIEW", - "WINDOW"), - singleInsert) - } - - val relations = fromClause match { - case Some(f) => nodeToRelation(f) - case None => OneRowRelation - } - - val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.children - Filter(nodeToExpr(whereExpr), relations) - }.getOrElse(relations) - - val select = (selectClause orElse selectDistinctClause) - .getOrElse(sys.error("No select clause.")) - - val transformation = nodeToTransformation(select.children.head, withWhere) - - val withLateralView = lateralViewClause.map { lv => - nodeToGenerate(lv.children.head, outer = false, withWhere) - }.getOrElse(withWhere) - - // The projection of the query can either be a normal projection, an aggregation - // (if there is a group by) or a script transformation. - val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = - select.children.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)) - Seq( - groupByClause.map(e => e match { - case Token("TOK_GROUPBY", children) => - // Not a transformation so must be either project or aggregation. - Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) - case _ => sys.error("Expect GROUP BY") - }), - groupingSetsClause.map(e => e match { - case Token("TOK_GROUPING_SETS", children) => - val(groupByExprs, masks) = extractGroupingSet(children) - GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) - case _ => sys.error("Expect GROUPING SETS") - }), - rollupGroupByClause.map(e => e match { - case Token("TOK_ROLLUP_GROUPBY", children) => - Aggregate( - Seq(Rollup(children.map(nodeToExpr))), - selectExpressions, - withLateralView) - case _ => sys.error("Expect WITH ROLLUP") - }), - cubeGroupByClause.map(e => e match { - case Token("TOK_CUBE_GROUPBY", children) => - Aggregate( - Seq(Cube(children.map(nodeToExpr))), - selectExpressions, - withLateralView) - case _ => sys.error("Expect WITH CUBE") - }), - Some(Project(selectExpressions, withLateralView))).flatten.head - } - - // Handle HAVING clause. - val withHaving = havingClause.map { h => - val havingExpr = h.children match { case Seq(hexpr) => nodeToExpr(hexpr) } - // Note that we added a cast to boolean. If the expression itself is already boolean, - // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withProject) - }.getOrElse(withProject) - - // Handle SELECT DISTINCT - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving - - // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. - val withSort = - (orderByClause, sortByClause, distributeByClause, clusterByClause) match { - case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.children.map(nodeToSortOrder), global = true, withDistinct) - case (None, Some(perPartitionOrdering), None, None) => - Sort( - perPartitionOrdering.children.map(nodeToSortOrder), - global = false, withDistinct) - case (None, None, Some(partitionExprs), None) => - RepartitionByExpression( - partitionExprs.children.map(nodeToExpr), withDistinct) - case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - Sort( - perPartitionOrdering.children.map(nodeToSortOrder), global = false, - RepartitionByExpression( - partitionExprs.children.map(nodeToExpr), - withDistinct)) - case (None, None, None, Some(clusterExprs)) => - Sort( - clusterExprs.children.map(nodeToExpr).map(SortOrder(_, Ascending)), - global = false, - RepartitionByExpression( - clusterExprs.children.map(nodeToExpr), - withDistinct)) - case (None, None, None, None) => withDistinct - case _ => sys.error("Unsupported set of ordering / distribution clauses.") - } - - val withLimit = - limitClause.map(l => nodeToExpr(l.children.head)) - .map(Limit(_, withSort)) - .getOrElse(withSort) - - // Collect all window specifications defined in the WINDOW clause. - val windowDefinitions = windowClause.map(_.children.collect { - case Token("TOK_WINDOWDEF", - Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => - windowName -> nodesToWindowSpecification(spec) - }.toMap) - // Handle cases like - // window w1 as (partition by p_mfgr order by p_name - // range between 2 preceding and 2 following), - // w2 as w1 - val resolvedCrossReference = windowDefinitions.map { - windowDefMap => windowDefMap.map { - case (windowName, WindowSpecReference(other)) => - (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) - case o => o.asInstanceOf[(String, WindowSpecDefinition)] - } - } - - val withWindowDefinitions = - resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) - - // TOK_INSERT_INTO means to add files to the table. - // TOK_DESTINATION means to overwrite the table. - val resultDestination = - (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - val overwrite = intoClause.isEmpty - nodeToDest( - resultDestination, - withWindowDefinitions, - overwrite) - } - - // If there are multiple INSERTS just UNION them together into one query. - val query = if (queries.length == 1) queries.head else Union(queries) - - // return With plan if there is CTE - cteRelations.map(With(query, _)).getOrElse(query) - - case Token("TOK_UNIONALL", left :: right :: Nil) => - Union(nodeToPlan(left), nodeToPlan(right)) - case Token("TOK_UNIONDISTINCT", left :: right :: Nil) => - Distinct(Union(nodeToPlan(left), nodeToPlan(right))) - case Token("TOK_EXCEPT", left :: right :: Nil) => - Except(nodeToPlan(left), nodeToPlan(right)) - case Token("TOK_INTERSECT", left :: right :: Nil) => - Intersect(nodeToPlan(left), nodeToPlan(right)) - - case _ => - noParseRule("Plan", node) - } - - val allJoinTokens = "(TOK_.*JOIN)".r - val laterViewToken = "TOK_LATERAL_VIEW(.*)".r - protected def nodeToRelation(node: ASTNode): LogicalPlan = { - node match { - case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => - SubqueryAlias(cleanIdentifier(alias), nodeToPlan(query)) - - case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => - nodeToGenerate( - selectClause, - outer = isOuter.nonEmpty, - nodeToRelation(relationClause)) - - /* All relations, possibly with aliases or sampling clauses. */ - case Token("TOK_TABREF", clauses) => - // If the last clause is not a token then it's the alias of the table. - val (nonAliasClauses, aliasClause) = - if (clauses.last.text.startsWith("TOK")) { - (clauses, None) - } else { - (clauses.dropRight(1), Some(clauses.last)) - } - - val (Some(tableNameParts) :: - splitSampleClause :: - bucketSampleClause :: Nil) = { - getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), - nonAliasClauses) - } - - val tableIdent = extractTableIdent(tableNameParts) - val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } - val relation = UnresolvedRelation(tableIdent, alias) - - // Apply sampling if requested. - (bucketSampleClause orElse splitSampleClause).map { - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => - Limit(Literal(count.toInt), relation) - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) => - // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling - // function takes X PERCENT as the input and the range of X is [0, 100], we need to - // adjust the fraction. - require( - fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) - && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), - s"Sampling fraction ($fraction) must be on interval [0, 100]") - Sample(0.0, fraction.toDouble / 100, withReplacement = false, - (math.random * 1000).toInt, - relation) - case Token("TOK_TABLEBUCKETSAMPLE", - Token(numerator, Nil) :: - Token(denominator, Nil) :: Nil) => - val fraction = numerator.toDouble / denominator.toDouble - Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) - case a => - noParseRule("Sampling", a) - }.getOrElse(relation) - - case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => - if (!(other.size <= 1)) { - sys.error(s"Unsupported join operation: $other") - } - - val joinType = joinToken match { - case "TOK_JOIN" => Inner - case "TOK_CROSSJOIN" => Inner - case "TOK_RIGHTOUTERJOIN" => RightOuter - case "TOK_LEFTOUTERJOIN" => LeftOuter - case "TOK_FULLOUTERJOIN" => FullOuter - case "TOK_LEFTSEMIJOIN" => LeftSemi - case "TOK_UNIQUEJOIN" => noParseRule("Unique Join", node) - case "TOK_ANTIJOIN" => noParseRule("Anti Join", node) - case "TOK_NATURALJOIN" => NaturalJoin(Inner) - case "TOK_NATURALRIGHTOUTERJOIN" => NaturalJoin(RightOuter) - case "TOK_NATURALLEFTOUTERJOIN" => NaturalJoin(LeftOuter) - case "TOK_NATURALFULLOUTERJOIN" => NaturalJoin(FullOuter) - } - Join(nodeToRelation(relation1), - nodeToRelation(relation2), - joinType, - other.headOption.map(nodeToExpr)) - - case _ => - noParseRule("Relation", node) - } - } - - protected def nodeToSortOrder(node: ASTNode): SortOrder = node match { - case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Descending) - case _ => - noParseRule("SortOrder", node) - } - - val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r - protected def nodeToDest( - node: ASTNode, - query: LogicalPlan, - overwrite: Boolean): LogicalPlan = node match { - case Token(destinationToken(), - Token("TOK_DIR", - Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => - query - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.children.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable( - UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = false) - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: - Token("TOK_IFNOTEXISTS", - ifNotExists) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.children.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable( - UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = true) - - case _ => - noParseRule("Destination", node) - } - - protected def selExprNodeToExpr(node: ASTNode): Option[Expression] = node match { - case Token("TOK_SELEXPR", e :: Nil) => - Some(nodeToExpr(e)) - - case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) - - case Token("TOK_SELEXPR", e :: aliasChildren) => - val aliasNames = aliasChildren.collect { - case Token(name, Nil) => cleanIdentifier(name) - } - Some(MultiAlias(nodeToExpr(e), aliasNames)) - - /* Hints are ignored */ - case Token("TOK_HINTLIST", _) => None - - case _ => - noParseRule("Select", node) - } - - /* 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) => - UnresolvedAttribute.quoted(cleanIdentifier(name)) - case Token(".", qualifier :: Token(attr, Nil) :: Nil) => - nodeToExpr(qualifier) match { - case UnresolvedAttribute(nameParts) => - UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) - case other => UnresolvedExtractValue(other, Literal(cleanIdentifier(attr))) - } - case Token("TOK_SUBQUERY_EXPR", Token("TOK_SUBQUERY_OP", Nil) :: subquery :: Nil) => - ScalarSubquery(nodeToPlan(subquery)) - - /* Stars (*) */ - case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) - // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only - // has a single child which is tableName. - case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", target) :: Nil) if target.nonEmpty => - UnresolvedStar(Some(target.map(x => cleanIdentifier(x.text)))) - - /* Aggregate Functions */ - case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => - Count(args.map(nodeToExpr)).toAggregateExpression(isDistinct = true) - case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => - Count(Literal(1)).toAggregateExpression() - - /* Casts */ - case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), IntegerType) - case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), LongType) - case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DoubleType) - case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ShortType) - case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ByteType) - case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), BinaryType) - case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), BooleanType) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: scale :: nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, scale.text.toInt)) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, 0)) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType.USER_DEFAULT) - case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), TimestampType) - case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DateType) - - /* Arithmetic */ - case Token("+", child :: Nil) => nodeToExpr(child) - case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) - case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) - case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) - case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) - case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) - case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token(DIV(), left :: right:: Nil) => - Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) - case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) - case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right)) - case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right)) - case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right)) - - /* Comparisons */ - case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) - case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) - case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right)) - case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) - case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) - case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) - case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) - case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) - case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => - IsNotNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => - IsNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => - In(nodeToExpr(value), list.map(nodeToExpr)) - case Token("TOK_FUNCTION", - Token(BETWEEN(), Nil) :: - kw :: - target :: - minValue :: - maxValue :: Nil) => - - val targetExpression = nodeToExpr(target) - val betweenExpr = - And( - GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), - LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) - kw match { - case Token("KW_FALSE", Nil) => betweenExpr - case Token("KW_TRUE", Nil) => Not(betweenExpr) - } - - /* Boolean Logic */ - case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) - case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) - case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) - case Token("!", child :: Nil) => Not(nodeToExpr(child)) - - /* Case statements */ - case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => - CaseWhen.createFromParser(branches.map(nodeToExpr)) - case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val keyExpr = nodeToExpr(branches.head) - CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) - - /* Complex datatype manipulation */ - case Token("[", child :: ordinal :: Nil) => - UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) - - /* Window Functions */ - case Token(text, args :+ Token("TOK_WINDOWSPEC", spec)) => - val function = nodeToExpr(node.copy(children = node.children.init)) - nodesToWindowSpecification(spec) match { - case reference: WindowSpecReference => - UnresolvedWindowExpression(function, reference) - case definition: WindowSpecDefinition => - WindowExpression(function, definition) - } - - /* UDFs - Must be last otherwise will preempt built in functions */ - case Token("TOK_FUNCTION", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = false) - // Aggregate function with DISTINCT keyword. - case Token("TOK_FUNCTIONDI", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = true) - case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => - UnresolvedFunction(name, UnresolvedStar(None) :: Nil, isDistinct = false) - - /* Literals */ - case Token("TOK_NULL", Nil) => Literal.create(null, NullType) - case Token(TRUE(), Nil) => Literal.create(true, BooleanType) - case Token(FALSE(), Nil) => Literal.create(false, BooleanType) - case Token("TOK_STRINGLITERALSEQUENCE", strings) => - Literal(strings.map(s => ParseUtils.unescapeSQLString(s.text)).mkString) - - case ast if ast.tokenType == SparkSqlParser.TinyintLiteral => - Literal.create(ast.text.substring(0, ast.text.length() - 1).toByte, ByteType) - - case ast if ast.tokenType == SparkSqlParser.SmallintLiteral => - Literal.create(ast.text.substring(0, ast.text.length() - 1).toShort, ShortType) - - case ast if ast.tokenType == SparkSqlParser.BigintLiteral => - Literal.create(ast.text.substring(0, ast.text.length() - 1).toLong, LongType) - - case ast if ast.tokenType == SparkSqlParser.DoubleLiteral => - Literal(ast.text.toDouble) - - case ast if ast.tokenType == SparkSqlParser.Number => - val text = ast.text - text match { - case INTEGRAL() => - BigDecimal(text) match { - case v if v.isValidInt => - Literal(v.intValue()) - case v if v.isValidLong => - Literal(v.longValue()) - case v => Literal(v.underlying()) - } - case DECIMAL(_*) => - Literal(BigDecimal(text).underlying()) - case _ => - // Convert a scientifically notated decimal into a double. - Literal(text.toDouble) - } - case ast if ast.tokenType == SparkSqlParser.StringLiteral => - Literal(ParseUtils.unescapeSQLString(ast.text)) - - case ast if ast.tokenType == SparkSqlParser.TOK_DATELITERAL => - Literal(Date.valueOf(ast.text.substring(1, ast.text.length - 1))) - - case ast if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL => - Literal(CalendarInterval.fromYearMonthString(ast.children.head.text)) - - case ast if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL => - Literal(CalendarInterval.fromDayTimeString(ast.children.head.text)) - - case Token("TOK_INTERVAL", elements) => - var interval = new CalendarInterval(0, 0) - var updated = false - elements.foreach { - // The interval node will always contain children for all possible time units. A child node - // is only useful when it contains exactly one (numeric) child. - case e @ Token(name, Token(value, Nil) :: Nil) => - val unit = name match { - case "TOK_INTERVAL_YEAR_LITERAL" => "year" - case "TOK_INTERVAL_MONTH_LITERAL" => "month" - case "TOK_INTERVAL_WEEK_LITERAL" => "week" - case "TOK_INTERVAL_DAY_LITERAL" => "day" - case "TOK_INTERVAL_HOUR_LITERAL" => "hour" - case "TOK_INTERVAL_MINUTE_LITERAL" => "minute" - case "TOK_INTERVAL_SECOND_LITERAL" => "second" - case "TOK_INTERVAL_MILLISECOND_LITERAL" => "millisecond" - case "TOK_INTERVAL_MICROSECOND_LITERAL" => "microsecond" - case _ => noParseRule(s"Interval($name)", e) - } - interval = interval.add(CalendarInterval.fromSingleUnitString(unit, value)) - updated = true - case _ => - } - if (!updated) { - throw new AnalysisException("at least one time unit should be given for interval literal") - } - Literal(interval) - - case _ => - noParseRule("Expression", node) - } - - /* Case insensitive matches for Window Specification */ - val PRECEDING = "(?i)preceding".r - val FOLLOWING = "(?i)following".r - val CURRENT = "(?i)current".r - protected def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { - case Token(windowName, Nil) :: Nil => - // Refer to a window spec defined in the window clause. - WindowSpecReference(windowName) - case Nil => - // OVER() - WindowSpecDefinition( - partitionSpec = Nil, - orderSpec = Nil, - frameSpecification = UnspecifiedFrame) - case spec => - val (partitionClause :: rowFrame :: rangeFrame :: Nil) = - getClauses( - Seq( - "TOK_PARTITIONINGSPEC", - "TOK_WINDOWRANGE", - "TOK_WINDOWVALUES"), - spec) - - // Handle Partition By and Order By. - val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => - val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = - getClauses( - Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), - partitionAndOrdering.children) - - (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { - case (Some(partitionByExpr), Some(orderByExpr), None) => - (partitionByExpr.children.map(nodeToExpr), - orderByExpr.children.map(nodeToSortOrder)) - case (Some(partitionByExpr), None, None) => - (partitionByExpr.children.map(nodeToExpr), Nil) - case (None, Some(orderByExpr), None) => - (Nil, orderByExpr.children.map(nodeToSortOrder)) - case (None, None, Some(clusterByExpr)) => - val expressions = clusterByExpr.children.map(nodeToExpr) - (expressions, expressions.map(SortOrder(_, Ascending))) - case _ => - noParseRule("Partition & Ordering", partitionAndOrdering) - } - }.getOrElse { - (Nil, Nil) - } - - // Handle Window Frame - val windowFrame = - if (rowFrame.isEmpty && rangeFrame.isEmpty) { - UnspecifiedFrame - } else { - val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) - def nodeToBoundary(node: ASTNode): FrameBoundary = node match { - case Token(PRECEDING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedPreceding - } else { - ValuePreceding(count.toInt) - } - case Token(FOLLOWING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedFollowing - } else { - ValueFollowing(count.toInt) - } - case Token(CURRENT(), Nil) => CurrentRow - case _ => - noParseRule("Window Frame Boundary", node) - } - - rowFrame.orElse(rangeFrame).map { frame => - frame.children match { - case precedingNode :: followingNode :: Nil => - SpecifiedWindowFrame( - frameType, - nodeToBoundary(precedingNode), - nodeToBoundary(followingNode)) - case precedingNode :: Nil => - SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) - case _ => - noParseRule("Window Frame", frame) - } - }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) - } - - WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) - } - - protected def nodeToTransformation( - node: ASTNode, - child: LogicalPlan): Option[ScriptTransformation] = None - - val explode = "(?i)explode".r - val jsonTuple = "(?i)json_tuple".r - protected def nodeToGenerate(node: ASTNode, outer: Boolean, child: LogicalPlan): Generate = { - val Token("TOK_SELECT", Token("TOK_SELEXPR", clauses) :: Nil) = node - - val alias = cleanIdentifier(getClause("TOK_TABALIAS", clauses).children.head.text) - - val generator = clauses.head match { - case Token("TOK_FUNCTION", Token(explode(), Nil) :: childNode :: Nil) => - Explode(nodeToExpr(childNode)) - case Token("TOK_FUNCTION", Token(jsonTuple(), Nil) :: children) => - JsonTuple(children.map(nodeToExpr)) - case other => - nodeToGenerator(other) - } - - val attributes = clauses.collect { - case Token(a, Nil) => UnresolvedAttribute(a.toLowerCase) - } - - Generate(generator, join = true, outer = outer, Some(alias.toLowerCase), attributes, child) - } - - 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/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserInterface.scala deleted file mode 100644 index 24ec452c4d2ef..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ParserInterface.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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 - -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * Interface for a parser. - */ -trait ParserInterface { - /** Creates LogicalPlan for a given SQL string. */ - def parsePlan(sqlText: String): LogicalPlan - - /** Creates Expression for a given SQL string. */ - def parseExpression(sqlText: String): Expression - - /** Creates TableIdentifier for a given SQL string. */ - def parseTableIdentifier(sqlText: String): TableIdentifier -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala deleted file mode 100644 index 3a5cb44f911b6..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/PlanParser.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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 - -import org.apache.spark.sql.catalyst.parser.ASTNode -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.CurrentOrigin - -trait ParserBase { - object Token { - def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { - CurrentOrigin.setPosition(node.line, node.positionInLine) - node.pattern - } - } - - 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 - } -} - -/** - * Abstract class for a parser that parses ASTNode to LogicalPlan. - */ -abstract class PlanParser extends PartialFunction[ASTNode, LogicalPlan] with ParserBase - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala deleted file mode 100644 index 092d4d37597dd..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkQlSuite.scala +++ /dev/null @@ -1,652 +0,0 @@ -/* - * 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 - -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.commands._ -import org.apache.spark.sql.execution.datasources.BucketSpec -import org.apache.spark.sql.types._ - -class SparkQlSuite 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) - } -} From 54c0578db8bb19439723b392d91b4cd5c20da962 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 23 Feb 2016 01:24:20 +0000 Subject: [PATCH 13/13] Fix scala style. --- .../org/apache/spark/sql/catalyst/parser/ParserSupport.scala | 2 +- .../org/apache/spark/sql/execution/commands/commands.scala | 2 +- .../org/apache/spark/sql/execution/commands/parsers.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala index 10a7f6ff98694..5b69e858b72f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserSupport.scala @@ -37,7 +37,7 @@ object ParserSupport { private[sql] def cleanIdentifier(ident: String): String = ident match { case escapedIdentifier(i) => i case plainIdent => plainIdent - } + } private[sql] def cleanAndUnquoteString(str: String): String = { cleanIdentifier(unquoteString(str)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala index 6b38e6be26e62..65524ea461d6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/commands.scala @@ -97,7 +97,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm } } (keyValueOutput, runFunc) - + case Some((key, Some(value))) if deprecatedProperties.contains(key) => (keyValueOutput, deprecatedProperties(key)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala index 17ca19056960c..3aab2e164003c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala @@ -120,7 +120,7 @@ object AlterTableCommandParser { 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 { @@ -134,7 +134,7 @@ object AlterTableCommandParser { (Nil, Nil, bucketArgs.head.text.toInt) } } - + (Some(BucketSpec(numBuckets, bucketCols, sortCols, sortDirections)), false, false) case Token("TOK_NOT_CLUSTERED", Nil) =>