You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/03/31 18:25:16 UTC

[1/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Repository: spark
Updated Branches:
  refs/heads/master 26445c2e4 -> a9b93e073


http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala
----------------------------------------------------------------------
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
deleted file mode 100644
index 6fe0475..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala
+++ /dev/null
@@ -1,387 +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.{AnalysisException, SaveMode}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.parser._
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.types.StructType
-
-private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
-  import ParserUtils._
-
-  /** Check if a command should not be explained. */
-  protected def isNoExplainCommand(command: String): Boolean = {
-    "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command
-  }
-
-  /**
-   * For each node, extract properties in the form of a list
-   * ['key_part1', 'key_part2', 'key_part3', 'value']
-   * into a pair (key_part1.key_part2.key_part3, value).
-   *
-   * Example format:
-   *
-   *   TOK_TABLEPROPERTY
-   *   :- 'k1'
-   *   +- 'v1'
-   *   TOK_TABLEPROPERTY
-   *   :- 'k2'
-   *   +- 'v2'
-   *   TOK_TABLEPROPERTY
-   *   :- 'k3'
-   *   +- 'v3'
-   */
-  private def extractProps(
-      props: Seq[ASTNode],
-      expectedNodeText: String): Seq[(String, String)] = {
-    props.map {
-      case Token(x, keysAndValue) if x == expectedNodeText =>
-        val key = keysAndValue.init.map { x => unquoteString(x.text) }.mkString(".")
-        val value = unquoteString(keysAndValue.last.text)
-        (key, value)
-      case p =>
-        parseFailed(s"Expected property '$expectedNodeText' in command", p)
-    }
-  }
-
-  protected override def nodeToPlan(node: ASTNode): LogicalPlan = {
-    node match {
-      case Token("TOK_SETCONFIG", Nil) =>
-        val keyValueSeparatorIndex = node.remainder.indexOf('=')
-        if (keyValueSeparatorIndex >= 0) {
-          val key = node.remainder.substring(0, keyValueSeparatorIndex).trim
-          val value = node.remainder.substring(keyValueSeparatorIndex + 1).trim
-          SetCommand(Some(key -> Option(value)))
-        } else if (node.remainder.nonEmpty) {
-          SetCommand(Some(node.remainder -> None))
-        } else {
-          SetCommand(None)
-        }
-
-      // Just fake explain for any of the native commands.
-      case Token("TOK_EXPLAIN", explainArgs) if isNoExplainCommand(explainArgs.head.text) =>
-        ExplainCommand(OneRowRelation)
-
-      case Token("TOK_EXPLAIN", explainArgs) if "TOK_CREATETABLE" == explainArgs.head.text =>
-        val Some(crtTbl) :: _ :: extended :: Nil =
-          getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs)
-        ExplainCommand(nodeToPlan(crtTbl), extended = extended.isDefined)
-
-      case Token("TOK_EXPLAIN", explainArgs) =>
-        // Ignore FORMATTED if present.
-        val Some(query) :: _ :: extended :: Nil =
-          getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs)
-        ExplainCommand(nodeToPlan(query), extended = extended.isDefined)
-
-      case Token("TOK_REFRESHTABLE", nameParts :: Nil) =>
-        val tableIdent = extractTableIdent(nameParts)
-        RefreshTable(tableIdent)
-
-      // CREATE DATABASE [IF NOT EXISTS] database_name [COMMENT database_comment]
-      // [LOCATION path] [WITH DBPROPERTIES (key1=val1, key2=val2, ...)];
-      case Token("TOK_CREATEDATABASE", Token(dbName, Nil) :: args) =>
-        val databaseName = cleanIdentifier(dbName)
-        val Seq(ifNotExists, dbLocation, databaseComment, dbprops) = getClauses(Seq(
-          "TOK_IFNOTEXISTS",
-          "TOK_DATABASELOCATION",
-          "TOK_DATABASECOMMENT",
-          "TOK_DATABASEPROPERTIES"), args)
-        val location = dbLocation.map {
-          case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc)
-          case _ => parseFailed("Invalid CREATE DATABASE command", node)
-        }
-        val comment = databaseComment.map {
-          case Token("TOK_DATABASECOMMENT", Token(com, Nil) :: Nil) => unquoteString(com)
-          case _ => parseFailed("Invalid CREATE DATABASE command", node)
-        }
-        val props = dbprops.toSeq.flatMap {
-          case Token("TOK_DATABASEPROPERTIES", Token("TOK_DBPROPLIST", propList) :: Nil) =>
-            // Example format:
-            //
-            //   TOK_DATABASEPROPERTIES
-            //   +- TOK_DBPROPLIST
-            //      :- TOK_TABLEPROPERTY
-            //      :  :- 'k1'
-            //      :  +- 'v1'
-            //      :- TOK_TABLEPROPERTY
-            //         :- 'k2'
-            //         +- 'v2'
-            extractProps(propList, "TOK_TABLEPROPERTY")
-          case _ => parseFailed("Invalid CREATE DATABASE command", node)
-        }.toMap
-        CreateDatabase(databaseName, ifNotExists.isDefined, location, comment, props)
-
-      // DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE];
-      case Token("TOK_DROPDATABASE", Token(dbName, Nil) :: otherArgs) =>
-        // Example format:
-        //
-        //   TOK_DROPDATABASE
-        //   :- database_name
-        //   :- TOK_IFEXISTS
-        //   +- TOK_RESTRICT/TOK_CASCADE
-        val databaseName = cleanIdentifier(dbName)
-        // The default is RESTRICT
-        val Seq(ifExists, _, cascade) = getClauses(Seq(
-          "TOK_IFEXISTS", "TOK_RESTRICT", "TOK_CASCADE"), otherArgs)
-        DropDatabase(databaseName, ifExists.isDefined, cascade.isDefined)
-
-      // ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)
-      case Token("TOK_ALTERDATABASE_PROPERTIES", Token(dbName, Nil) :: args) =>
-        val databaseName = cleanIdentifier(dbName)
-        val dbprops = getClause("TOK_DATABASEPROPERTIES", args)
-        val props = dbprops match {
-          case Token("TOK_DATABASEPROPERTIES", Token("TOK_DBPROPLIST", propList) :: Nil) =>
-            // Example format:
-            //
-            //   TOK_DATABASEPROPERTIES
-            //   +- TOK_DBPROPLIST
-            //      :- TOK_TABLEPROPERTY
-            //      :  :- 'k1'
-            //      :  +- 'v1'
-            //      :- TOK_TABLEPROPERTY
-            //         :- 'k2'
-            //         +- 'v2'
-            extractProps(propList, "TOK_TABLEPROPERTY")
-          case _ => parseFailed("Invalid ALTER DATABASE command", node)
-        }
-        AlterDatabaseProperties(databaseName, props.toMap)
-
-      // DESCRIBE DATABASE [EXTENDED] db_name
-      case Token("TOK_DESCDATABASE", Token(dbName, Nil) :: describeArgs) =>
-        val databaseName = cleanIdentifier(dbName)
-        val extended = getClauseOption("EXTENDED", describeArgs)
-        DescribeDatabase(databaseName, extended.isDefined)
-
-      // CREATE [TEMPORARY] FUNCTION [db_name.]function_name AS class_name
-      // [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] ];
-      case Token("TOK_CREATEFUNCTION", args) =>
-        // Example format:
-        //
-        //   TOK_CREATEFUNCTION
-        //   :- db_name
-        //   :- func_name
-        //   :- alias
-        //   +- TOK_RESOURCE_LIST
-        //      :- TOK_RESOURCE_URI
-        //      :  :- TOK_JAR
-        //      :  +- '/path/to/jar'
-        //      +- TOK_RESOURCE_URI
-        //         :- TOK_FILE
-        //         +- 'path/to/file'
-        val (funcNameArgs, otherArgs) = args.partition {
-          case Token("TOK_RESOURCE_LIST", _) => false
-          case Token("TOK_TEMPORARY", _) => false
-          case Token(_, Nil) => true
-          case _ => parseFailed("Invalid CREATE FUNCTION command", node)
-        }
-        // If database name is specified, there are 3 tokens, otherwise 2.
-        val (dbName, funcName, alias) = funcNameArgs match {
-          case Token(dbName, Nil) :: Token(fname, Nil) :: Token(aname, Nil) :: Nil =>
-            (Some(unquoteString(dbName)), unquoteString(fname), unquoteString(aname))
-          case Token(fname, Nil) :: Token(aname, Nil) :: Nil =>
-            (None, unquoteString(fname), unquoteString(aname))
-          case _ =>
-            parseFailed("Invalid CREATE FUNCTION command", node)
-        }
-        // Extract other keywords, if they exist
-        val Seq(rList, temp) = getClauses(Seq("TOK_RESOURCE_LIST", "TOK_TEMPORARY"), otherArgs)
-        val resources: Seq[(String, String)] = rList.toSeq.flatMap {
-          case Token("TOK_RESOURCE_LIST", resList) =>
-            resList.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"
-                  case Token(f, _) => parseFailed(s"Unexpected resource format '$f'", node)
-                }
-                (resourceType, unquoteString(rPath))
-              case _ => parseFailed("Invalid CREATE FUNCTION command", node)
-            }
-          case _ => parseFailed("Invalid CREATE FUNCTION command", node)
-        }
-        CreateFunction(dbName, funcName, alias, resources, temp.isDefined)(node.source)
-
-      // DROP [TEMPORARY] FUNCTION [IF EXISTS] function_name;
-      case Token("TOK_DROPFUNCTION", args) =>
-        // Example format:
-        //
-        //   TOK_DROPFUNCTION
-        //   :- db_name
-        //   :- func_name
-        //   :- TOK_IFEXISTS
-        //   +- TOK_TEMPORARY
-        val (funcNameArgs, otherArgs) = args.partition {
-          case Token("TOK_IFEXISTS", _) => false
-          case Token("TOK_TEMPORARY", _) => false
-          case Token(_, Nil) => true
-          case _ => parseFailed("Invalid DROP FUNCTION command", node)
-        }
-        // If database name is specified, there are 2 tokens, otherwise 1.
-        val (dbName, funcName) = funcNameArgs match {
-          case Token(dbName, Nil) :: Token(fname, Nil) :: Nil =>
-            (Some(unquoteString(dbName)), unquoteString(fname))
-          case Token(fname, Nil) :: Nil =>
-            (None, unquoteString(fname))
-          case _ =>
-            parseFailed("Invalid DROP FUNCTION command", node)
-        }
-
-        val Seq(ifExists, temp) = getClauses(Seq(
-          "TOK_IFEXISTS", "TOK_TEMPORARY"), otherArgs)
-
-        DropFunction(dbName, funcName, ifExists.isDefined, temp.isDefined)(node.source)
-
-      case Token("TOK_ALTERTABLE", alterTableArgs) =>
-        AlterTableCommandParser.parse(node)
-
-      case Token("TOK_CREATETABLEUSING", createTableArgs) =>
-        val Seq(
-          temp,
-          ifNotExists,
-          Some(tabName),
-          tableCols,
-          Some(Token("TOK_TABLEPROVIDER", providerNameParts)),
-          tableOpts,
-          tableAs) = getClauses(Seq(
-          "TEMPORARY",
-          "TOK_IFNOTEXISTS",
-          "TOK_TABNAME", "TOK_TABCOLLIST",
-          "TOK_TABLEPROVIDER",
-          "TOK_TABLEOPTIONS",
-          "TOK_QUERY"), createTableArgs)
-        val tableIdent: TableIdentifier = extractTableIdent(tabName)
-        val columns = tableCols.map {
-          case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField))
-          case _ => parseFailed("Invalid CREATE TABLE command", node)
-        }
-        val provider = providerNameParts.map {
-          case Token(name, Nil) => name
-          case _ => parseFailed("Invalid CREATE TABLE command", node)
-        }.mkString(".")
-        val options = tableOpts.toSeq.flatMap {
-          case Token("TOK_TABLEOPTIONS", opts) => extractProps(opts, "TOK_TABLEOPTION")
-          case _ => parseFailed("Invalid CREATE TABLE command", node)
-        }.toMap
-        val asClause = tableAs.map(nodeToPlan)
-
-        if (temp.isDefined && ifNotExists.isDefined) {
-          throw new AnalysisException(
-            "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.")
-        }
-
-        if (asClause.isDefined) {
-          if (columns.isDefined) {
-            throw new AnalysisException(
-              "a CREATE TABLE AS SELECT statement does not allow column definitions.")
-          }
-
-          val mode = if (ifNotExists.isDefined) {
-            SaveMode.Ignore
-          } else if (temp.isDefined) {
-            SaveMode.Overwrite
-          } else {
-            SaveMode.ErrorIfExists
-          }
-
-          CreateTableUsingAsSelect(tableIdent,
-            provider,
-            temp.isDefined,
-            Array.empty[String],
-            bucketSpec = None,
-            mode,
-            options,
-            asClause.get)
-        } else {
-          CreateTableUsing(
-            tableIdent,
-            columns,
-            provider,
-            temp.isDefined,
-            options,
-            ifNotExists.isDefined,
-            managedIfNoPath = false)
-        }
-
-      case Token("TOK_SWITCHDATABASE", Token(database, Nil) :: Nil) =>
-        SetDatabaseCommand(cleanIdentifier(database))
-
-      case Token("TOK_DESCTABLE", describeArgs) =>
-        // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
-        val Some(tableType) :: formatted :: extended :: pretty :: Nil =
-          getClauses(Seq("TOK_TABTYPE", "FORMATTED", "EXTENDED", "PRETTY"), describeArgs)
-        if (formatted.isDefined || pretty.isDefined) {
-          // FORMATTED and PRETTY are not supported and this statement will be treated as
-          // a Hive native command.
-          nodeToDescribeFallback(node)
-        } else {
-          tableType match {
-            case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts) :: Nil) =>
-              nameParts match {
-                case Token(dbName, Nil) :: Token(tableName, Nil) :: Nil =>
-                  // It is describing a table with the format like "describe db.table".
-                  // TODO: Actually, a user may mean tableName.columnName. Need to resolve this
-                  // issue.
-                  val tableIdent = TableIdentifier(
-                    cleanIdentifier(tableName), Some(cleanIdentifier(dbName)))
-                  datasources.DescribeCommand(tableIdent, isExtended = extended.isDefined)
-                case Token(dbName, Nil) :: Token(tableName, Nil) :: Token(colName, Nil) :: Nil =>
-                  // It is describing a column with the format like "describe db.table column".
-                  nodeToDescribeFallback(node)
-                case tableName :: Nil =>
-                  // It is describing a table with the format like "describe table".
-                  datasources.DescribeCommand(
-                    TableIdentifier(cleanIdentifier(tableName.text)),
-                    isExtended = extended.isDefined)
-                case _ =>
-                  nodeToDescribeFallback(node)
-              }
-            // All other cases.
-            case _ =>
-              nodeToDescribeFallback(node)
-          }
-        }
-
-      case Token("TOK_CACHETABLE", Token(tableName, Nil) :: args) =>
-       val Seq(lzy, selectAst) = getClauses(Seq("LAZY", "TOK_QUERY"), args)
-        CacheTableCommand(tableName, selectAst.map(nodeToPlan), lzy.isDefined)
-
-      case Token("TOK_UNCACHETABLE", Token(tableName, Nil) :: Nil) =>
-        UncacheTableCommand(tableName)
-
-      case Token("TOK_CLEARCACHE", Nil) =>
-        ClearCacheCommand
-
-      case Token("TOK_SHOWTABLES", args) =>
-        val databaseName = args match {
-          case Nil => None
-          case Token("TOK_FROM", Token(dbName, Nil) :: Nil) :: Nil => Option(dbName)
-          case _ => noParseRule("SHOW TABLES", node)
-        }
-        ShowTablesCommand(databaseName)
-
-      case _ =>
-        super.nodeToPlan(node)
-    }
-  }
-
-  protected def nodeToDescribeFallback(node: ASTNode): LogicalPlan = noParseRule("Describe", node)
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index 8333074..b4687c9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -20,8 +20,8 @@ import scala.collection.JavaConverters._
 
 import org.apache.spark.sql.SaveMode
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.parser.ng.{AbstractSqlParser, AstBuilder, ParseException}
-import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, AstBuilder, ParseException}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
 import org.apache.spark.sql.execution.command.{DescribeCommand => _, _}
 import org.apache.spark.sql.execution.datasources._
@@ -37,7 +37,7 @@ object SparkSqlParser extends AbstractSqlParser{
  * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
  */
 class SparkSqlAstBuilder extends AstBuilder {
-  import org.apache.spark.sql.catalyst.parser.ng.ParserUtils._
+  import org.apache.spark.sql.catalyst.parser.ParserUtils._
 
   /**
    * Create a [[SetCommand]] logical plan.

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala
deleted file mode 100644
index 9fbe6db..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala
+++ /dev/null
@@ -1,431 +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.command
-
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec
-import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending, SortDirection}
-import org.apache.spark.sql.catalyst.parser._
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.types.StructType
-
-
-/**
- * Helper object to parse alter table commands.
- */
-object AlterTableCommandParser {
-  import ParserUtils._
-
-  /**
-   * Parse the given node assuming it is an alter table command.
-   */
-  def parse(node: ASTNode): LogicalPlan = {
-    node.children match {
-      case (tabName @ Token("TOK_TABNAME", _)) :: otherNodes =>
-        val tableIdent = extractTableIdent(tabName)
-        val partSpec = getClauseOption("TOK_PARTSPEC", node.children).map(parsePartitionSpec)
-        matchAlterTableCommands(node, otherNodes, tableIdent, partSpec)
-      case _ =>
-        parseFailed("Could not parse ALTER TABLE command", node)
-    }
-  }
-
-  private def cleanAndUnquoteString(s: String): String = {
-    cleanIdentifier(unquoteString(s))
-  }
-
-  /**
-   * Extract partition spec from the given [[ASTNode]] as a map, assuming it exists.
-   *
-   * Example format:
-   *
-   *   TOK_PARTSPEC
-   *   :- TOK_PARTVAL
-   *   :  :- dt
-   *   :  +- '2008-08-08'
-   *   +- TOK_PARTVAL
-   *      :- country
-   *      +- 'us'
-   */
-  private def parsePartitionSpec(node: ASTNode): Map[String, String] = {
-    node match {
-      case Token("TOK_PARTSPEC", partitions) =>
-        partitions.map {
-          // Note: sometimes there's a "=", "<" or ">" between the key and the value
-          // (e.g. when dropping all partitions with value > than a certain constant)
-          case Token("TOK_PARTVAL", ident :: conj :: constant :: Nil) =>
-            (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text))
-          case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
-            (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text))
-          case Token("TOK_PARTVAL", ident :: Nil) =>
-            (cleanAndUnquoteString(ident.text), null)
-          case _ =>
-            parseFailed("Invalid ALTER TABLE command", node)
-        }.toMap
-      case _ =>
-        parseFailed("Expected partition spec in ALTER TABLE command", node)
-    }
-  }
-
-  /**
-   * Extract table properties from the given [[ASTNode]] as a map, assuming it exists.
-   *
-   * Example format:
-   *
-   *   TOK_TABLEPROPERTIES
-   *   +- TOK_TABLEPROPLIST
-   *      :- TOK_TABLEPROPERTY
-   *      :  :- 'test'
-   *      :  +- 'value'
-   *      +- TOK_TABLEPROPERTY
-   *         :- 'comment'
-   *         +- 'new_comment'
-   */
-  private def extractTableProps(node: ASTNode): Map[String, String] = {
-    node match {
-      case Token("TOK_TABLEPROPERTIES", propsList) =>
-        propsList.flatMap {
-          case Token("TOK_TABLEPROPLIST", props) =>
-            props.map { case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
-              val k = cleanAndUnquoteString(key.text)
-              val v = value match {
-                case Token("TOK_NULL", Nil) => null
-                case _ => cleanAndUnquoteString(value.text)
-              }
-              (k, v)
-            }
-          case _ =>
-            parseFailed("Invalid ALTER TABLE command", node)
-        }.toMap
-      case _ =>
-        parseFailed("Expected table properties in ALTER TABLE command", node)
-    }
-  }
-
-  /**
-   * Parse an alter table command from a [[ASTNode]] into a [[LogicalPlan]].
-   * This follows https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL.
-   *
-   * @param node the original [[ASTNode]] to parse.
-   * @param otherNodes the other [[ASTNode]]s after the first one containing the table name.
-   * @param tableIdent identifier of the table, parsed from the first [[ASTNode]].
-   * @param partition spec identifying the partition this command is concerned with, if any.
-   */
-  // TODO: This method is massive. Break it down.
-  private def matchAlterTableCommands(
-      node: ASTNode,
-      otherNodes: Seq[ASTNode],
-      tableIdent: TableIdentifier,
-      partition: Option[TablePartitionSpec]): LogicalPlan = {
-    otherNodes match {
-      // ALTER TABLE table_name RENAME TO new_table_name;
-      case Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ =>
-        val tableNameClause = getClause("TOK_TABNAME", renameArgs)
-        val newTableIdent = extractTableIdent(tableNameClause)
-        AlterTableRename(tableIdent, newTableIdent)(node.source)
-
-      // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment);
-      case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ =>
-        val properties = extractTableProps(args.head)
-        AlterTableSetProperties(tableIdent, properties)(node.source)
-
-      // ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'key');
-      case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ =>
-        val properties = extractTableProps(args.head)
-        val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined
-        AlterTableUnsetProperties(tableIdent, properties, ifExists)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] SET SERDE serde_name [WITH SERDEPROPERTIES props];
-      case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ =>
-        AlterTableSerDeProperties(
-          tableIdent,
-          Some(cleanAndUnquoteString(serdeClassName)),
-          serdeArgs.headOption.map(extractTableProps),
-          partition)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] SET SERDEPROPERTIES serde_properties;
-      case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ =>
-        AlterTableSerDeProperties(
-          tableIdent,
-          None,
-          Some(extractTableProps(args.head)),
-          partition)(node.source)
-
-      // ALTER TABLE table_name CLUSTERED BY (col, ...) [SORTED BY (col, ...)] INTO n BUCKETS;
-      case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_ALTERTABLE_BUCKETS", b) :: Nil) :: _ =>
-        val clusterCols: Seq[String] = b.head match {
-          case Token("TOK_TABCOLNAME", children) => children.map(_.text)
-          case _ => parseFailed("Invalid ALTER TABLE command", node)
-        }
-        // If sort columns are specified, num buckets should be the third arg.
-        // If sort columns are not specified, num buckets should be the second arg.
-        // TODO: actually use `sortDirections` once we actually store that in the metastore
-        val (sortCols: Seq[String], sortDirections: Seq[SortDirection], numBuckets: Int) = {
-          b.tail match {
-            case Token("TOK_TABCOLNAME", children) :: numBucketsNode :: Nil =>
-              val (cols, directions) = children.map {
-                case Token("TOK_TABSORTCOLNAMEASC", Token(col, Nil) :: Nil) => (col, Ascending)
-                case Token("TOK_TABSORTCOLNAMEDESC", Token(col, Nil) :: Nil) => (col, Descending)
-              }.unzip
-              (cols, directions, numBucketsNode.text.toInt)
-            case numBucketsNode :: Nil =>
-              (Nil, Nil, numBucketsNode.text.toInt)
-            case _ =>
-              parseFailed("Invalid ALTER TABLE command", node)
-          }
-        }
-        AlterTableStorageProperties(
-          tableIdent,
-          BucketSpec(numBuckets, clusterCols, sortCols))(node.source)
-
-      // ALTER TABLE table_name NOT CLUSTERED
-      case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ =>
-        AlterTableNotClustered(tableIdent)(node.source)
-
-      // ALTER TABLE table_name NOT SORTED
-      case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_SORTED", Nil) :: Nil) :: _ =>
-        AlterTableNotSorted(tableIdent)(node.source)
-
-      // ALTER TABLE table_name SKEWED BY (col1, col2)
-      //   ON ((col1_value, col2_value) [, (col1_value, col2_value), ...])
-      //   [STORED AS DIRECTORIES];
-      case Token("TOK_ALTERTABLE_SKEWED",
-          Token("TOK_TABLESKEWED",
-          Token("TOK_TABCOLNAME", colNames) :: colValues :: rest) :: Nil) :: _ =>
-        // Example format:
-        //
-        //   TOK_ALTERTABLE_SKEWED
-        //   :- TOK_TABLESKEWED
-        //   :  :- TOK_TABCOLNAME
-        //   :  :  :- dt
-        //   :  :  +- country
-        //   :- TOK_TABCOLVALUE_PAIR
-        //   :  :- TOK_TABCOLVALUES
-        //   :  :  :- TOK_TABCOLVALUE
-        //   :  :  :  :- '2008-08-08'
-        //   :  :  :  +- 'us'
-        //   :  :- TOK_TABCOLVALUES
-        //   :  :  :- TOK_TABCOLVALUE
-        //   :  :  :  :- '2009-09-09'
-        //   :  :  :  +- 'uk'
-        //   +- TOK_STOREASDIR
-        val names = colNames.map { n => cleanAndUnquoteString(n.text) }
-        val values = colValues match {
-          case Token("TOK_TABCOLVALUE", vals) =>
-            Seq(vals.map { n => cleanAndUnquoteString(n.text) })
-          case Token("TOK_TABCOLVALUE_PAIR", pairs) =>
-            pairs.map {
-              case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", vals) :: Nil) =>
-                vals.map { n => cleanAndUnquoteString(n.text) }
-              case _ =>
-                parseFailed("Invalid ALTER TABLE command", node)
-            }
-          case _ =>
-            parseFailed("Invalid ALTER TABLE command", node)
-        }
-        val storedAsDirs = rest match {
-          case Token("TOK_STOREDASDIRS", Nil) :: Nil => true
-          case _ => false
-        }
-        AlterTableSkewed(
-          tableIdent,
-          names,
-          values,
-          storedAsDirs)(node.source)
-
-      // ALTER TABLE table_name NOT SKEWED
-      case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ =>
-        AlterTableNotSkewed(tableIdent)(node.source)
-
-      // ALTER TABLE table_name NOT STORED AS DIRECTORIES
-      case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) :: _ =>
-        AlterTableNotStoredAsDirs(tableIdent)(node.source)
-
-      // ALTER TABLE table_name SET SKEWED LOCATION (col1="loc1" [, (col2, col3)="loc2", ...] );
-      case Token("TOK_ALTERTABLE_SKEWED_LOCATION",
-        Token("TOK_SKEWED_LOCATIONS",
-        Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ =>
-        // Example format:
-        //
-        //   TOK_ALTERTABLE_SKEWED_LOCATION
-        //   +- TOK_SKEWED_LOCATIONS
-        //      +- TOK_SKEWED_LOCATION_LIST
-        //         :- TOK_SKEWED_LOCATION_MAP
-        //         :  :- 'col1'
-        //         :  +- 'loc1'
-        //         +- TOK_SKEWED_LOCATION_MAP
-        //            :- TOK_TABCOLVALUES
-        //            :  +- TOK_TABCOLVALUE
-        //            :     :- 'col2'
-        //            :     +- 'col3'
-        //            +- 'loc2'
-        val skewedMaps = locationMaps.flatMap {
-          case Token("TOK_SKEWED_LOCATION_MAP", col :: loc :: Nil) =>
-            col match {
-              case Token(const, Nil) =>
-                Seq((cleanAndUnquoteString(const), cleanAndUnquoteString(loc.text)))
-              case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", keys) :: Nil) =>
-                keys.map { k => (cleanAndUnquoteString(k.text), cleanAndUnquoteString(loc.text)) }
-            }
-          case _ =>
-            parseFailed("Invalid ALTER TABLE command", node)
-        }.toMap
-        AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source)
-
-      // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION spec [LOCATION 'loc1']
-      // spec [LOCATION 'loc2'] ...;
-      case Token("TOK_ALTERTABLE_ADDPARTS", args) :: _ =>
-        val (ifNotExists, parts) = args.head match {
-          case Token("TOK_IFNOTEXISTS", Nil) => (true, args.tail)
-          case _ => (false, args)
-        }
-        // List of (spec, location) to describe partitions to add
-        // Each partition spec may or may not be followed by a location
-        val parsedParts = new ArrayBuffer[(TablePartitionSpec, Option[String])]
-        parts.foreach {
-          case t @ Token("TOK_PARTSPEC", _) =>
-            parsedParts += ((parsePartitionSpec(t), None))
-          case Token("TOK_PARTITIONLOCATION", loc :: Nil) =>
-            // Update the location of the last partition we just added
-            if (parsedParts.nonEmpty) {
-              val (spec, _) = parsedParts.remove(parsedParts.length - 1)
-              parsedParts += ((spec, Some(unquoteString(loc.text))))
-            }
-          case _ =>
-            parseFailed("Invalid ALTER TABLE command", node)
-        }
-        AlterTableAddPartition(tableIdent, parsedParts, ifNotExists)(node.source)
-
-      // ALTER TABLE table_name PARTITION spec1 RENAME TO PARTITION spec2;
-      case Token("TOK_ALTERTABLE_RENAMEPART", spec :: Nil) :: _ =>
-        val newPartition = parsePartitionSpec(spec)
-        val oldPartition = partition.getOrElse {
-          parseFailed("Expected old partition spec in ALTER TABLE rename partition command", node)
-        }
-        AlterTableRenamePartition(tableIdent, oldPartition, newPartition)(node.source)
-
-      // ALTER TABLE table_name_1 EXCHANGE PARTITION spec WITH TABLE table_name_2;
-      case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", spec :: newTable :: Nil) :: _ =>
-        val parsedSpec = parsePartitionSpec(spec)
-        val newTableIdent = extractTableIdent(newTable)
-        AlterTableExchangePartition(tableIdent, newTableIdent, parsedSpec)(node.source)
-
-      // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE];
-      case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ =>
-        val parts = args.collect { case p @ Token("TOK_PARTSPEC", _) => parsePartitionSpec(p) }
-        val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined
-        val purge = getClauseOption("PURGE", args).isDefined
-        AlterTableDropPartition(tableIdent, parts, ifExists, purge)(node.source)
-
-      // ALTER TABLE table_name ARCHIVE PARTITION spec;
-      case Token("TOK_ALTERTABLE_ARCHIVE", spec :: Nil) :: _ =>
-        AlterTableArchivePartition(tableIdent, parsePartitionSpec(spec))(node.source)
-
-      // ALTER TABLE table_name UNARCHIVE PARTITION spec;
-      case Token("TOK_ALTERTABLE_UNARCHIVE", spec :: Nil) :: _ =>
-        AlterTableUnarchivePartition(tableIdent, parsePartitionSpec(spec))(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] SET FILEFORMAT file_format;
-      case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ =>
-        val Seq(fileFormat, genericFormat) =
-          getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), args)
-        // Note: the AST doesn't contain information about which file format is being set here.
-        // E.g. we can't differentiate between INPUTFORMAT and OUTPUTFORMAT if either is set.
-        // Right now this just stores the values, but we should figure out how to get the keys.
-        val fFormat = fileFormat
-          .map { _.children.map { n => cleanAndUnquoteString(n.text) }}
-          .getOrElse(Seq())
-        val gFormat = genericFormat.map { f => cleanAndUnquoteString(f.children(0).text) }
-        AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] SET LOCATION "loc";
-      case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ =>
-        AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source)
-
-      // ALTER TABLE table_name TOUCH [PARTITION spec];
-      case Token("TOK_ALTERTABLE_TOUCH", args) :: _ =>
-        // Note: the partition spec, if it exists, comes after TOUCH, so `partition` should
-        // always be None here. Instead, we need to parse it from the TOUCH node's children.
-        val part = getClauseOption("TOK_PARTSPEC", args).map(parsePartitionSpec)
-        AlterTableTouch(tableIdent, part)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] COMPACT 'compaction_type';
-      case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ =>
-        AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] CONCATENATE;
-      case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ =>
-        AlterTableMerge(tableIdent, partition)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] CHANGE [COLUMN] col_old_name col_new_name
-      // column_type [COMMENT col_comment] [FIRST|AFTER column_name] [CASCADE|RESTRICT];
-      case Token("TOK_ALTERTABLE_RENAMECOL", oldName :: newName :: dataType :: args) :: _ =>
-        val afterColName: Option[String] =
-          getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args).map { ap =>
-            ap.children match {
-              case Token(col, Nil) :: Nil => col
-              case _ => parseFailed("Invalid ALTER TABLE command", node)
-            }
-          }
-        val restrict = getClauseOption("TOK_RESTRICT", args).isDefined
-        val cascade = getClauseOption("TOK_CASCADE", args).isDefined
-        val comment = args.headOption.map {
-          case Token("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", _) => null
-          case Token("TOK_RESTRICT", _) => null
-          case Token("TOK_CASCADE", _) => null
-          case Token(commentStr, Nil) => cleanAndUnquoteString(commentStr)
-          case _ => parseFailed("Invalid ALTER TABLE command", node)
-        }
-        AlterTableChangeCol(
-          tableIdent,
-          partition,
-          oldName.text,
-          newName.text,
-          nodeToDataType(dataType),
-          comment,
-          afterColName,
-          restrict,
-          cascade)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] ADD COLUMNS (name type [COMMENT comment], ...)
-      // [CASCADE|RESTRICT]
-      case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ =>
-        val columnNodes = getClause("TOK_TABCOLLIST", args).children
-        val columns = StructType(columnNodes.map(nodeToStructField))
-        val restrict = getClauseOption("TOK_RESTRICT", args).isDefined
-        val cascade = getClauseOption("TOK_CASCADE", args).isDefined
-        AlterTableAddCol(tableIdent, partition, columns, restrict, cascade)(node.source)
-
-      // ALTER TABLE table_name [PARTITION spec] REPLACE COLUMNS (name type [COMMENT comment], ...)
-      // [CASCADE|RESTRICT]
-      case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ =>
-        val columnNodes = getClause("TOK_TABCOLLIST", args).children
-        val columns = StructType(columnNodes.map(nodeToStructField))
-        val restrict = getClauseOption("TOK_RESTRICT", args).isDefined
-        val cascade = getClauseOption("TOK_CASCADE", args).isDefined
-        AlterTableReplaceCol(tableIdent, partition, columns, restrict, cascade)(node.source)
-
-      case _ =>
-        parseFailed("Unsupported ALTER TABLE command", node)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index d06e908..6cc72fb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -26,7 +26,6 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter
 
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.CatalystConf
-import org.apache.spark.sql.catalyst.parser.ParserConf
 import org.apache.spark.util.Utils
 
 ////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -500,19 +499,6 @@ object SQLConf {
     doc = "When true, we could use `datasource`.`path` as table in SQL query."
   )
 
-  val PARSER_SUPPORT_QUOTEDID = booleanConf("spark.sql.parser.supportQuotedIdentifiers",
-    defaultValue = Some(true),
-    isPublic = false,
-    doc = "Whether to use quoted identifier.\n  false: default(past) behavior. Implies only" +
-      "alphaNumeric and underscore are valid characters in identifiers.\n" +
-      "  true: implies column names can contain any character.")
-
-  val PARSER_SUPPORT_SQL11_RESERVED_KEYWORDS = booleanConf(
-    "spark.sql.parser.supportSQL11ReservedKeywords",
-    defaultValue = Some(false),
-    isPublic = false,
-    doc = "This flag should be set to true to enable support for SQL2011 reserved keywords.")
-
   val WHOLESTAGE_CODEGEN_ENABLED = booleanConf("spark.sql.codegen.wholeStage",
     defaultValue = Some(true),
     doc = "When true, the whole stage (of multiple operators) will be compiled into single java" +
@@ -573,7 +559,7 @@ object SQLConf {
  *
  * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads).
  */
-class SQLConf extends Serializable with CatalystConf with ParserConf with Logging {
+class SQLConf extends Serializable with CatalystConf with Logging {
   import SQLConf._
 
   /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */
@@ -674,10 +660,6 @@ class SQLConf extends Serializable with CatalystConf with ParserConf with Loggin
 
   def runSQLOnFile: Boolean = getConf(RUN_SQL_ON_FILES)
 
-  def supportQuotedId: Boolean = getConf(PARSER_SUPPORT_QUOTEDID)
-
-  def supportSQL11ReservedKeywords: Boolean = getConf(PARSER_SUPPORT_SQL11_RESERVED_KEYWORDS)
-
   override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL)
 
   override def groupByOrdinal: Boolean = getConf(GROUP_BY_ORDINAL)

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 47c9a22..f148f2d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -21,11 +21,22 @@ import java.io.File
 
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
 import org.apache.spark.sql.catalyst.catalog.CatalogDatabase
-import org.apache.spark.sql.catalyst.parser.ParserUtils._
 import org.apache.spark.sql.test.SharedSQLContext
 
 class DDLSuite extends QueryTest with SharedSQLContext {
 
+  private val escapedIdentifier = "`(.+)`".r
+
+  /**
+   * Strip backticks, if any, from the string.
+   */
+  def cleanIdentifier(ident: String): String = {
+    ident match {
+      case escapedIdentifier(i) => i
+      case plainIdent => plainIdent
+    }
+  }
+
   /**
    * Drops database `databaseName` after calling `f`.
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/pom.xml
----------------------------------------------------------------------
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 22bad93..58efd80 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -225,25 +225,6 @@
           <argLine>-da -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m</argLine>
         </configuration>
       </plugin>
-      <plugin>
-         <groupId>org.codehaus.mojo</groupId>
-         <artifactId>build-helper-maven-plugin</artifactId>
-         <executions>
-           <execution>
-             <id>add-default-sources</id>
-             <phase>generate-sources</phase>
-             <goals>
-               <goal>add-source</goal>
-             </goals>
-             <configuration>
-               <sources>
-                 <source>v${hive.version.short}/src/main/scala</source>
-                 <source>${project.build.directory/generated-sources/antlr</source>
-               </sources>
-             </configuration>
-           </execution>
-         </executions>
-      </plugin>
 
       <!-- Deploy datanucleus jars to the spark/lib_managed/jars directory -->
       <plugin>

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
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 9a5ec98..2cdc931 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
@@ -25,7 +25,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.hive.common.StatsSetupConst
 import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.metastore.{TableType => HiveTableType, Warehouse}
+import org.apache.hadoop.hive.metastore.{TableType => HiveTableType}
 import org.apache.hadoop.hive.metastore.api.FieldSchema
 import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable, _}
 import org.apache.hadoop.hive.ql.plan.TableDesc
@@ -988,3 +988,28 @@ private[hive] object HiveMetastoreTypes {
     case udt: UserDefinedType[_] => toMetastoreType(udt.sqlType)
   }
 }
+
+private[hive] case class CreateTableAsSelect(
+    tableDesc: CatalogTable,
+    child: LogicalPlan,
+    allowExisting: Boolean) extends UnaryNode with Command {
+
+  override def output: Seq[Attribute] = Seq.empty[Attribute]
+  override lazy val resolved: Boolean =
+    tableDesc.identifier.database.isDefined &&
+      tableDesc.schema.nonEmpty &&
+      tableDesc.storage.serde.isDefined &&
+      tableDesc.storage.inputFormat.isDefined &&
+      tableDesc.storage.outputFormat.isDefined &&
+      childrenResolved
+}
+
+private[hive] case class CreateViewAsSelect(
+    tableDesc: CatalogTable,
+    child: LogicalPlan,
+    allowExisting: Boolean,
+    replace: Boolean,
+    sql: String) extends UnaryNode with Command {
+  override def output: Seq[Attribute] = Seq.empty[Attribute]
+  override lazy val resolved: Boolean = false
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
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
deleted file mode 100644
index 052c43a..0000000
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ /dev/null
@@ -1,749 +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.hive
-
-import java.util.Locale
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.hive.common.`type`.HiveDecimal
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars
-import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry}
-import org.apache.hadoop.hive.ql.parse.EximUtil
-import org.apache.hadoop.hive.ql.session.SessionState
-import org.apache.hadoop.hive.serde.serdeConstants
-import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.parser._
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.SparkQl
-import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
-import org.apache.spark.sql.hive.execution._
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.AnalysisException
-
-/**
- * Used when we need to start parsing the AST before deciding that we are going to pass the command
- * back for Hive to execute natively.  Will be replaced with a native command that contains the
- * cmd string.
- */
-private[hive] case object NativePlaceholder extends LogicalPlan {
-  override def children: Seq[LogicalPlan] = Seq.empty
-  override def output: Seq[Attribute] = Seq.empty
-}
-
-private[hive] case class CreateTableAsSelect(
-    tableDesc: CatalogTable,
-    child: LogicalPlan,
-    allowExisting: Boolean) extends UnaryNode with Command {
-
-  override def output: Seq[Attribute] = Seq.empty[Attribute]
-  override lazy val resolved: Boolean =
-    tableDesc.identifier.database.isDefined &&
-    tableDesc.schema.nonEmpty &&
-    tableDesc.storage.serde.isDefined &&
-    tableDesc.storage.inputFormat.isDefined &&
-    tableDesc.storage.outputFormat.isDefined &&
-    childrenResolved
-}
-
-private[hive] case class CreateViewAsSelect(
-    tableDesc: CatalogTable,
-    child: LogicalPlan,
-    allowExisting: Boolean,
-    replace: Boolean,
-    sql: String) extends UnaryNode with Command {
-  override def output: Seq[Attribute] = Seq.empty[Attribute]
-  override lazy val resolved: Boolean = false
-}
-
-/** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */
-private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging {
-  import ParseUtils._
-  import ParserUtils._
-
-  // Token text -> human readable text
-  private val hiveUnsupportedCommands = Map(
-    "TOK_CREATEROLE" -> "CREATE ROLE",
-    "TOK_DROPROLE" -> "DROP ROLE",
-    "TOK_EXPORT" -> "EXPORT TABLE",
-    "TOK_GRANT" -> "GRANT",
-    "TOK_GRANT_ROLE" -> "GRANT",
-    "TOK_IMPORT" -> "IMPORT TABLE",
-    "TOK_REVOKE" -> "REVOKE",
-    "TOK_REVOKE_ROLE" -> "REVOKE",
-    "TOK_SHOW_COMPACTIONS" -> "SHOW COMPACTIONS",
-    "TOK_SHOW_CREATETABLE" -> "SHOW CREATE TABLE",
-    "TOK_SHOW_GRANT" -> "SHOW GRANT",
-    "TOK_SHOW_ROLE_GRANT" -> "SHOW ROLE GRANT",
-    "TOK_SHOW_ROLE_PRINCIPALS" -> "SHOW PRINCIPALS",
-    "TOK_SHOW_ROLES" -> "SHOW ROLES",
-    "TOK_SHOW_SET_ROLE" -> "SHOW CURRENT ROLES / SET ROLE",
-    "TOK_SHOW_TRANSACTIONS" -> "SHOW TRANSACTIONS",
-    "TOK_SHOWINDEXES" -> "SHOW INDEXES",
-    "TOK_SHOWLOCKS" -> "SHOW LOCKS")
-
-  private val nativeCommands = Set(
-    "TOK_ALTERDATABASE_OWNER",
-    "TOK_ALTERINDEX_PROPERTIES",
-    "TOK_ALTERINDEX_REBUILD",
-    "TOK_ALTERTABLE_ALTERPARTS",
-    "TOK_ALTERTABLE_PARTITION",
-    "TOK_ALTERVIEW_ADDPARTS",
-    "TOK_ALTERVIEW_AS",
-    "TOK_ALTERVIEW_DROPPARTS",
-    "TOK_ALTERVIEW_PROPERTIES",
-    "TOK_ALTERVIEW_RENAME",
-
-    "TOK_CREATEINDEX",
-    "TOK_CREATEMACRO",
-
-    "TOK_DROPINDEX",
-    "TOK_DROPMACRO",
-    "TOK_DROPTABLE_PROPERTIES",
-    "TOK_DROPVIEW",
-    "TOK_DROPVIEW_PROPERTIES",
-
-    "TOK_LOAD",
-
-    "TOK_LOCKTABLE",
-
-    "TOK_MSCK",
-
-    "TOK_SHOW_TABLESTATUS",
-    "TOK_SHOW_TBLPROPERTIES",
-    "TOK_SHOWCOLUMNS",
-    "TOK_SHOWDATABASES",
-    "TOK_SHOWPARTITIONS",
-
-    "TOK_UNLOCKTABLE"
-  )
-
-  // Commands that we do not need to explain.
-  private val noExplainCommands = Set(
-    "TOK_DESCTABLE",
-    "TOK_SHOWTABLES",
-    "TOK_TRUNCATETABLE", // truncate table" is a NativeCommand, does not need to explain.
-    "TOK_ALTERTABLE"
-  ) ++ nativeCommands
-
-  /**
-   * Returns the HiveConf
-   */
-  private[this] def hiveConf: HiveConf = {
-    var ss = SessionState.get()
-    // SessionState is lazy initialization, it can be null here
-    if (ss == null) {
-      val original = Thread.currentThread().getContextClassLoader
-      val conf = new HiveConf(classOf[SessionState])
-      conf.setClassLoader(original)
-      ss = new SessionState(conf)
-      SessionState.start(ss)
-    }
-    ss.getConf
-  }
-
-  protected def getProperties(node: ASTNode): Seq[(String, String)] = node match {
-    case Token("TOK_TABLEPROPLIST", list) =>
-      list.map {
-        case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
-          unquoteString(key) -> unquoteString(value)
-      }
-  }
-
-  private def createView(
-      view: ASTNode,
-      viewNameParts: ASTNode,
-      query: ASTNode,
-      schema: Seq[CatalogColumn],
-      properties: Map[String, String],
-      allowExist: Boolean,
-      replace: Boolean): CreateViewAsSelect = {
-    val tableIdentifier = extractTableIdent(viewNameParts)
-    val originalText = query.source
-    val tableDesc = CatalogTable(
-      identifier = tableIdentifier,
-      tableType = CatalogTableType.VIRTUAL_VIEW,
-      schema = schema,
-      storage = CatalogStorageFormat(
-        locationUri = None,
-        inputFormat = None,
-        outputFormat = None,
-        serde = None,
-        serdeProperties = Map.empty[String, String]
-      ),
-      properties = properties,
-      viewOriginalText = Some(originalText),
-      viewText = Some(originalText))
-
-    // We need to keep the original SQL string so that if `spark.sql.nativeView` is
-    // false, we can fall back to use hive native command later.
-    // We can remove this when parser is configurable(can access SQLConf) in the future.
-    val sql = view.source
-    CreateViewAsSelect(tableDesc, nodeToPlan(query), allowExist, replace, sql)
-  }
-
-  /** Creates LogicalPlan for a given SQL string. */
-  override def parsePlan(sql: String): LogicalPlan = {
-    safeParse(sql, ParseDriver.parsePlan(sql, conf)) { ast =>
-      if (nativeCommands.contains(ast.text)) {
-        HiveNativeCommand(sql)
-      } else if (hiveUnsupportedCommands.contains(ast.text)) {
-        val humanReadableText = hiveUnsupportedCommands(ast.text)
-        throw new AnalysisException("Unsupported operation: " + humanReadableText)
-      } else {
-        nodeToPlan(ast) match {
-          case NativePlaceholder => HiveNativeCommand(sql)
-          case plan => plan
-        }
-      }
-    }
-  }
-
-  protected override def isNoExplainCommand(command: String): Boolean =
-    noExplainCommands.contains(command)
-
-  protected override def nodeToPlan(node: ASTNode): LogicalPlan = {
-    node match {
-      case Token("TOK_DFS", Nil) =>
-        HiveNativeCommand(node.source + " " + node.remainder)
-
-      case Token("TOK_ADDFILE", Nil) =>
-        AddFile(node.remainder)
-
-      case Token("TOK_ADDJAR", Nil) =>
-        AddJar(node.remainder)
-
-      // Special drop table that also uncaches.
-      case Token("TOK_DROPTABLE", Token("TOK_TABNAME", tableNameParts) :: ifExists) =>
-        val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".")
-        DropTable(tableName, ifExists.nonEmpty)
-
-      // Support "ANALYZE TABLE tableName COMPUTE STATISTICS noscan"
-      case Token("TOK_ANALYZE",
-        Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: isNoscan) =>
-        // Reference:
-        // https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables
-        if (partitionSpec.nonEmpty) {
-          // Analyze partitions will be treated as a Hive native command.
-          NativePlaceholder
-        } else if (isNoscan.isEmpty) {
-          // If users do not specify "noscan", it will be treated as a Hive native command.
-          NativePlaceholder
-        } else {
-          val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".")
-          AnalyzeTable(tableName)
-        }
-
-      case view @ Token("TOK_ALTERVIEW", children) =>
-        val Some(nameParts) :: maybeQuery :: _ =
-          getClauses(Seq(
-            "TOK_TABNAME",
-            "TOK_QUERY",
-            "TOK_ALTERVIEW_ADDPARTS",
-            "TOK_ALTERVIEW_DROPPARTS",
-            "TOK_ALTERVIEW_PROPERTIES",
-            "TOK_ALTERVIEW_RENAME"), children)
-
-        // if ALTER VIEW doesn't have query part, let hive to handle it.
-        maybeQuery.map { query =>
-          createView(view, nameParts, query, Nil, Map(), allowExist = false, replace = true)
-        }.getOrElse(NativePlaceholder)
-
-      case view @ Token("TOK_CREATEVIEW", children)
-        if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty =>
-        val Seq(
-        Some(viewNameParts),
-        Some(query),
-        maybeComment,
-        replace,
-        allowExisting,
-        maybeProperties,
-        maybeColumns,
-        maybePartCols
-        ) = getClauses(Seq(
-          "TOK_TABNAME",
-          "TOK_QUERY",
-          "TOK_TABLECOMMENT",
-          "TOK_ORREPLACE",
-          "TOK_IFNOTEXISTS",
-          "TOK_TABLEPROPERTIES",
-          "TOK_TABCOLNAME",
-          "TOK_VIEWPARTCOLS"), children)
-
-        // If the view is partitioned, we let hive handle it.
-        if (maybePartCols.isDefined) {
-          NativePlaceholder
-        } else {
-          val schema = maybeColumns.map { cols =>
-            // We can't specify column types when create view, so fill it with null first, and
-            // update it after the schema has been resolved later.
-            nodeToColumns(cols, lowerCase = true).map(_.copy(dataType = null))
-          }.getOrElse(Seq.empty[CatalogColumn])
-
-          val properties = scala.collection.mutable.Map.empty[String, String]
-
-          maybeProperties.foreach {
-            case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
-              properties ++= getProperties(list)
-          }
-
-          maybeComment.foreach {
-            case Token("TOK_TABLECOMMENT", child :: Nil) =>
-              val comment = unescapeSQLString(child.text)
-              if (comment ne null) {
-                properties += ("comment" -> comment)
-              }
-          }
-
-          createView(view, viewNameParts, query, schema, properties.toMap,
-            allowExisting.isDefined, replace.isDefined)
-        }
-
-      case Token("TOK_CREATETABLE", children)
-        if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty =>
-        // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
-        val (
-          Some(tableNameParts) ::
-            _ /* likeTable */ ::
-            externalTable ::
-            Some(query) ::
-            allowExisting +:
-              _) =
-          getClauses(
-            Seq(
-              "TOK_TABNAME",
-              "TOK_LIKETABLE",
-              "EXTERNAL",
-              "TOK_QUERY",
-              "TOK_IFNOTEXISTS",
-              "TOK_TABLECOMMENT",
-              "TOK_TABCOLLIST",
-              "TOK_TABLEPARTCOLS", // Partitioned by
-              "TOK_TABLEBUCKETS", // Clustered by
-              "TOK_TABLESKEWED", // Skewed by
-              "TOK_TABLEROWFORMAT",
-              "TOK_TABLESERIALIZER",
-              "TOK_FILEFORMAT_GENERIC",
-              "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat
-              "TOK_STORAGEHANDLER", // Storage handler
-              "TOK_TABLELOCATION",
-              "TOK_TABLEPROPERTIES"),
-            children)
-        val tableIdentifier = extractTableIdent(tableNameParts)
-
-        // TODO add bucket support
-        var tableDesc: CatalogTable = CatalogTable(
-          identifier = tableIdentifier,
-          tableType =
-            if (externalTable.isDefined) {
-              CatalogTableType.EXTERNAL_TABLE
-            } else {
-              CatalogTableType.MANAGED_TABLE
-            },
-          storage = CatalogStorageFormat(
-            locationUri = None,
-            inputFormat = None,
-            outputFormat = None,
-            serde = None,
-            serdeProperties = Map.empty[String, String]
-          ),
-          schema = Seq.empty[CatalogColumn])
-
-        // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.)
-        val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT)
-        // handle the default format for the storage type abbreviation
-        val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse {
-          HiveSerDe(
-            inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"),
-            outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
-        }
-
-        tableDesc = tableDesc.withNewStorage(
-          inputFormat = hiveSerDe.inputFormat.orElse(tableDesc.storage.inputFormat),
-          outputFormat = hiveSerDe.outputFormat.orElse(tableDesc.storage.outputFormat),
-          serde = hiveSerDe.serde.orElse(tableDesc.storage.serde))
-
-        children.collect {
-          case list @ Token("TOK_TABCOLLIST", _) =>
-            val cols = nodeToColumns(list, lowerCase = true)
-            if (cols != null) {
-              tableDesc = tableDesc.copy(schema = cols)
-            }
-          case Token("TOK_TABLECOMMENT", child :: Nil) =>
-            val comment = unescapeSQLString(child.text)
-            // TODO support the sql text
-            tableDesc = tableDesc.copy(viewText = Option(comment))
-          case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) =>
-            val cols = nodeToColumns(list.head, lowerCase = false)
-            if (cols != null) {
-              tableDesc = tableDesc.copy(partitionColumns = cols)
-            }
-          case Token("TOK_TABLEROWFORMAT", Token("TOK_SERDEPROPS", child :: Nil) :: Nil) =>
-            val serdeParams = new java.util.HashMap[String, String]()
-            child match {
-              case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) =>
-                val fieldDelim = unescapeSQLString (rowChild1.text)
-                serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim)
-                serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim)
-                if (rowChild2.length > 1) {
-                  val fieldEscape = unescapeSQLString (rowChild2.head.text)
-                  serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape)
-                }
-              case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) =>
-                val collItemDelim = unescapeSQLString(rowChild.text)
-                serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim)
-              case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) =>
-                val mapKeyDelim = unescapeSQLString(rowChild.text)
-                serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim)
-              case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) =>
-                val lineDelim = unescapeSQLString(rowChild.text)
-                if (!(lineDelim == "\n") && !(lineDelim == "10")) {
-                  throw new AnalysisException(
-                    s"LINES TERMINATED BY only supports newline '\\n' right now: $rowChild")
-                }
-                serdeParams.put(serdeConstants.LINE_DELIM, lineDelim)
-              case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) =>
-                val nullFormat = unescapeSQLString(rowChild.text)
-              // TODO support the nullFormat
-              case _ => assert(false)
-            }
-            tableDesc = tableDesc.withNewStorage(
-              serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams.asScala)
-          case Token("TOK_TABLELOCATION", child :: Nil) =>
-            val location = EximUtil.relativeToAbsolutePath(hiveConf, unescapeSQLString(child.text))
-            tableDesc = tableDesc.withNewStorage(locationUri = Option(location))
-          case Token("TOK_TABLESERIALIZER", child :: Nil) =>
-            tableDesc = tableDesc.withNewStorage(
-              serde = Option(unescapeSQLString(child.children.head.text)))
-            if (child.numChildren == 2) {
-              // This is based on the readProps(..) method in
-              // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java:
-              val serdeParams = child.children(1).children.head.children.map {
-                case Token(_, Token(prop, Nil) :: valueNode) =>
-                  val value = valueNode.headOption
-                    .map(_.text)
-                    .map(unescapeSQLString)
-                    .orNull
-                  (unescapeSQLString(prop), value)
-              }.toMap
-              tableDesc = tableDesc.withNewStorage(
-                serdeProperties = tableDesc.storage.serdeProperties ++ serdeParams)
-            }
-          case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) =>
-            child.text.toLowerCase(Locale.ENGLISH) match {
-              case "orc" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"),
-                  outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))
-                if (tableDesc.storage.serde.isEmpty) {
-                  tableDesc = tableDesc.withNewStorage(
-                    serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))
-                }
-
-              case "parquet" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat =
-                    Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
-                  outputFormat =
-                    Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
-                if (tableDesc.storage.serde.isEmpty) {
-                  tableDesc = tableDesc.withNewStorage(
-                    serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
-                }
-
-              case "rcfile" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"),
-                  outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
-                if (tableDesc.storage.serde.isEmpty) {
-                  tableDesc = tableDesc.withNewStorage(
-                    serde =
-                      Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))
-                }
-
-              case "textfile" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"),
-                  outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
-
-              case "sequencefile" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"),
-                  outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))
-
-              case "avro" =>
-                tableDesc = tableDesc.withNewStorage(
-                  inputFormat =
-                    Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"),
-                  outputFormat =
-                    Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"))
-                if (tableDesc.storage.serde.isEmpty) {
-                  tableDesc = tableDesc.withNewStorage(
-                    serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))
-                }
-
-              case _ =>
-                throw new AnalysisException(
-                  s"Unrecognized file format in STORED AS clause: ${child.text}")
-            }
-
-          case Token("TOK_TABLESERIALIZER",
-          Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) =>
-            tableDesc = tableDesc.withNewStorage(serde = Option(unquoteString(serdeName)))
-
-            otherProps match {
-              case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil =>
-                tableDesc = tableDesc.withNewStorage(
-                  serdeProperties = tableDesc.storage.serdeProperties ++ getProperties(list))
-              case _ =>
-            }
-
-          case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
-            tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list))
-          case list @ Token("TOK_TABLEFILEFORMAT", _) =>
-            tableDesc = tableDesc.withNewStorage(
-              inputFormat = Option(unescapeSQLString(list.children.head.text)),
-              outputFormat = Option(unescapeSQLString(list.children(1).text)))
-          case Token("TOK_STORAGEHANDLER", _) =>
-            throw new AnalysisException(
-              "CREATE TABLE AS SELECT cannot be used for a non-native table")
-          case _ => // Unsupported features
-        }
-
-        CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting.isDefined)
-
-      // If its not a "CTAS" like above then take it as a native command
-      case Token("TOK_CREATETABLE", _) =>
-        NativePlaceholder
-
-      // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]"
-      case Token("TOK_TRUNCATETABLE", Token("TOK_TABLE_PARTITION", table) :: Nil) =>
-        NativePlaceholder
-
-      case _ =>
-        super.nodeToPlan(node)
-    }
-  }
-
-  protected override def nodeToDescribeFallback(node: ASTNode): LogicalPlan = NativePlaceholder
-
-  protected override def nodeToTransformation(
-      node: ASTNode,
-      child: LogicalPlan): Option[logical.ScriptTransformation] = node match {
-    case Token("TOK_SELEXPR",
-      Token("TOK_TRANSFORM",
-      Token("TOK_EXPLIST", inputExprs) ::
-      Token("TOK_SERDE", inputSerdeClause) ::
-      Token("TOK_RECORDWRITER", writerClause) ::
-      // TODO: Need to support other types of (in/out)put
-      Token(script, Nil) ::
-      Token("TOK_SERDE", outputSerdeClause) ::
-      Token("TOK_RECORDREADER", readerClause) ::
-      outputClause) :: Nil) =>
-
-      val (output, schemaLess) = outputClause match {
-        case Token("TOK_ALIASLIST", aliases) :: Nil =>
-          (aliases.map { case Token(name, Nil) =>
-            AttributeReference(cleanIdentifier(name), StringType)() }, false)
-        case Token("TOK_TABCOLLIST", attributes) :: Nil =>
-          (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) =>
-            AttributeReference(cleanIdentifier(name), nodeToDataType(dataType))() }, false)
-        case Nil =>
-          (List(AttributeReference("key", StringType)(),
-            AttributeReference("value", StringType)()), true)
-        case _ =>
-          noParseRule("Transform", node)
-      }
-
-      type SerDeInfo = (
-        Seq[(String, String)],  // Input row format information
-          Option[String],         // Optional input SerDe class
-          Seq[(String, String)],  // Input SerDe properties
-          Boolean                 // Whether to use default record reader/writer
-        )
-
-      def matchSerDe(clause: Seq[ASTNode]): SerDeInfo = clause match {
-        case Token("TOK_SERDEPROPS", propsClause) :: Nil =>
-          val rowFormat = propsClause.map {
-            case Token(name, Token(value, Nil) :: Nil) => (name, value)
-          }
-          (rowFormat, None, Nil, false)
-
-        case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil =>
-          (Nil, Some(unescapeSQLString(serdeClass)), Nil, false)
-
-        case Token("TOK_SERDENAME", Token(serdeClass, Nil) ::
-          Token("TOK_TABLEPROPERTIES",
-          Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil =>
-          val serdeProps = propsClause.map {
-            case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) =>
-              (unescapeSQLString(name), unescapeSQLString(value))
-          }
-
-          // SPARK-10310: Special cases LazySimpleSerDe
-          // TODO Fully supports user-defined record reader/writer classes
-          val unescapedSerDeClass = unescapeSQLString(serdeClass)
-          val useDefaultRecordReaderWriter =
-            unescapedSerDeClass == classOf[LazySimpleSerDe].getCanonicalName
-          (Nil, Some(unescapedSerDeClass), serdeProps, useDefaultRecordReaderWriter)
-
-        case Nil =>
-          // Uses default TextRecordReader/TextRecordWriter, sets field delimiter here
-          val serdeProps = Seq(serdeConstants.FIELD_DELIM -> "\t")
-          (Nil, Option(hiveConf.getVar(ConfVars.HIVESCRIPTSERDE)), serdeProps, true)
-      }
-
-      val (inRowFormat, inSerdeClass, inSerdeProps, useDefaultRecordReader) =
-        matchSerDe(inputSerdeClause)
-
-      val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) =
-        matchSerDe(outputSerdeClause)
-
-      val unescapedScript = unescapeSQLString(script)
-
-      // TODO Adds support for user-defined record reader/writer classes
-      val recordReaderClass = if (useDefaultRecordReader) {
-        Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER))
-      } else {
-        None
-      }
-
-      val recordWriterClass = if (useDefaultRecordWriter) {
-        Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDWRITER))
-      } else {
-        None
-      }
-
-      val schema = HiveScriptIOSchema(
-        inRowFormat, outRowFormat,
-        inSerdeClass, outSerdeClass,
-        inSerdeProps, outSerdeProps,
-        recordReaderClass, recordWriterClass,
-        schemaLess)
-
-      Some(
-        logical.ScriptTransformation(
-          inputExprs.map(nodeToExpr),
-          unescapedScript,
-          output,
-          child, schema))
-    case _ => None
-  }
-
-  protected override def nodeToGenerator(node: ASTNode): Generator = node match {
-    case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) =>
-      val functionInfo: FunctionInfo =
-        Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse(
-          sys.error(s"Couldn't find function $functionName"))
-      val functionClassName = functionInfo.getFunctionClass.getName
-      HiveGenericUDTF(
-        functionName, new HiveFunctionWrapper(functionClassName), children.map(nodeToExpr))
-    case other => super.nodeToGenerator(node)
-  }
-
-  // This is based the getColumns methods in
-  // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
-  protected def nodeToColumns(node: ASTNode, lowerCase: Boolean): Seq[CatalogColumn] = {
-    node.children.map(_.children).collect {
-      case Token(rawColName, Nil) :: colTypeNode :: comment =>
-        val colName = if (!lowerCase) rawColName else rawColName.toLowerCase
-        CatalogColumn(
-          name = cleanIdentifier(colName),
-          dataType = nodeToTypeString(colTypeNode),
-          nullable = true,
-          comment.headOption.map(n => unescapeSQLString(n.text)))
-    }
-  }
-
-  // This is based on the following methods in
-  // ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java:
-  //  getTypeStringFromAST
-  //  getStructTypeStringFromAST
-  //  getUnionTypeStringFromAST
-  protected def nodeToTypeString(node: ASTNode): String = node.tokenType match {
-    case SparkSqlParser.TOK_LIST =>
-      val listType :: Nil = node.children
-      val listTypeString = nodeToTypeString(listType)
-      s"${serdeConstants.LIST_TYPE_NAME}<$listTypeString>"
-
-    case SparkSqlParser.TOK_MAP =>
-      val keyType :: valueType :: Nil = node.children
-      val keyTypeString = nodeToTypeString(keyType)
-      val valueTypeString = nodeToTypeString(valueType)
-      s"${serdeConstants.MAP_TYPE_NAME}<$keyTypeString,$valueTypeString>"
-
-    case SparkSqlParser.TOK_STRUCT =>
-      val typeNode = node.children.head
-      require(typeNode.children.nonEmpty, "Struct must have one or more columns.")
-      val structColStrings = typeNode.children.map { columnNode =>
-        val Token(colName, Nil) :: colTypeNode :: Nil = columnNode.children
-        cleanIdentifier(colName) + ":" + nodeToTypeString(colTypeNode)
-      }
-      s"${serdeConstants.STRUCT_TYPE_NAME}<${structColStrings.mkString(",")}>"
-
-    case SparkSqlParser.TOK_UNIONTYPE =>
-      val typeNode = node.children.head
-      val unionTypesString = typeNode.children.map(nodeToTypeString).mkString(",")
-      s"${serdeConstants.UNION_TYPE_NAME}<$unionTypesString>"
-
-    case SparkSqlParser.TOK_CHAR =>
-      val Token(size, Nil) :: Nil = node.children
-      s"${serdeConstants.CHAR_TYPE_NAME}($size)"
-
-    case SparkSqlParser.TOK_VARCHAR =>
-      val Token(size, Nil) :: Nil = node.children
-      s"${serdeConstants.VARCHAR_TYPE_NAME}($size)"
-
-    case SparkSqlParser.TOK_DECIMAL =>
-      val precisionAndScale = node.children match {
-        case Token(precision, Nil) :: Token(scale, Nil) :: Nil =>
-          precision + "," + scale
-        case Token(precision, Nil) :: Nil =>
-          precision + "," + HiveDecimal.USER_DEFAULT_SCALE
-        case Nil =>
-          HiveDecimal.USER_DEFAULT_PRECISION + "," + HiveDecimal.USER_DEFAULT_SCALE
-        case _ =>
-          noParseRule("Decimal", node)
-      }
-      s"${serdeConstants.DECIMAL_TYPE_NAME}($precisionAndScale)"
-
-    // Simple data types.
-    case SparkSqlParser.TOK_BOOLEAN => serdeConstants.BOOLEAN_TYPE_NAME
-    case SparkSqlParser.TOK_TINYINT => serdeConstants.TINYINT_TYPE_NAME
-    case SparkSqlParser.TOK_SMALLINT => serdeConstants.SMALLINT_TYPE_NAME
-    case SparkSqlParser.TOK_INT => serdeConstants.INT_TYPE_NAME
-    case SparkSqlParser.TOK_BIGINT => serdeConstants.BIGINT_TYPE_NAME
-    case SparkSqlParser.TOK_FLOAT => serdeConstants.FLOAT_TYPE_NAME
-    case SparkSqlParser.TOK_DOUBLE => serdeConstants.DOUBLE_TYPE_NAME
-    case SparkSqlParser.TOK_STRING => serdeConstants.STRING_TYPE_NAME
-    case SparkSqlParser.TOK_BINARY => serdeConstants.BINARY_TYPE_NAME
-    case SparkSqlParser.TOK_DATE => serdeConstants.DATE_TYPE_NAME
-    case SparkSqlParser.TOK_TIMESTAMP => serdeConstants.TIMESTAMP_TYPE_NAME
-    case SparkSqlParser.TOK_INTERVAL_YEAR_MONTH => serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME
-    case SparkSqlParser.TOK_INTERVAL_DAY_TIME => serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME
-    case SparkSqlParser.TOK_DATETIME => serdeConstants.DATETIME_TYPE_NAME
-    case _ => null
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
index d6a08fc..12e4f49 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala
@@ -29,8 +29,8 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
 
 import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType}
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.parser.ng._
-import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
+import org.apache.spark.sql.catalyst.parser._
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
 import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView}
@@ -161,18 +161,10 @@ class HiveSqlAstBuilder extends SparkSqlAstBuilder {
       }
 
       // Create the schema.
-      val schema = Option(ctx.colTypeList).toSeq.flatMap(_.colType.asScala).map { col =>
-        CatalogColumn(
-          col.identifier.getText,
-          col.dataType.getText.toLowerCase, // TODO validate this?
-          nullable = true,
-          Option(col.STRING).map(string))
-      }
+      val schema = Option(ctx.columns).toSeq.flatMap(visitCatalogColumns(_, _.toLowerCase))
 
       // Get the column by which the table is partitioned.
-      val partitionCols = Option(ctx.identifierList).toSeq.flatMap(visitIdentifierList).map {
-        CatalogColumn(_, null, nullable = true, None)
-      }
+      val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitCatalogColumns(_))
 
       // Create the storage.
       def format(fmt: ParserRuleContext): CatalogStorageFormat = {
@@ -439,4 +431,19 @@ class HiveSqlAstBuilder extends SparkSqlAstBuilder {
       }
     EmptyStorageFormat.copy(serdeProperties = entries.toMap)
   }
+
+  /**
+   * Create a sequence of [[CatalogColumn]]s from a column list
+   */
+  private def visitCatalogColumns(
+      ctx: ColTypeListContext,
+      formatter: String => String = identity): Seq[CatalogColumn] = withOrigin(ctx) {
+    ctx.colType.asScala.map { col =>
+      CatalogColumn(
+        formatter(col.identifier.getText),
+        col.dataType.getText.toLowerCase, // TODO validate this?
+        nullable = true,
+        Option(col.STRING).map(string))
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala
index 4b6da7c..d966468 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala
@@ -22,8 +22,8 @@ import scala.util.Try
 import org.scalatest.BeforeAndAfterEach
 
 import org.apache.spark.sql.{AnalysisException, QueryTest}
-import org.apache.spark.sql.catalyst.parser.ParseDriver
 import org.apache.spark.sql.catalyst.util.quietly
+import org.apache.spark.sql.hive.execution.HiveSqlParser
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 
 class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterEach {
@@ -131,7 +131,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd
    * @param token a unique token in the string that should be indicated by the exception
    */
   def positionTest(name: String, query: String, token: String): Unit = {
-    def ast = ParseDriver.parsePlan(query, hiveContext.conf)
+    def ast = HiveSqlParser.parsePlan(query)
     def parseTree = Try(quietly(ast.treeString)).getOrElse("<failed to parse>")
 
     test(name) {

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala
index 0aaf576..75108c6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala
@@ -24,11 +24,11 @@ import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType}
 import org.apache.spark.sql.catalyst.expressions.JsonTuple
-import org.apache.spark.sql.catalyst.parser.SimpleParserConf
 import org.apache.spark.sql.catalyst.plans.logical.Generate
+import org.apache.spark.sql.hive.execution.HiveSqlParser
 
 class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll {
-  val parser = new HiveQl(SimpleParserConf())
+  val parser = HiveSqlParser
 
   private def extractTableDesc(sql: String): (CatalogTable, Boolean) = {
     parser.parsePlan(sql).collect {

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index ae026ed..05318f5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -21,7 +21,6 @@ import scala.reflect.ClassTag
 
 import org.apache.spark.sql.{QueryTest, Row}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.parser.SimpleParserConf
 import org.apache.spark.sql.execution.joins._
 import org.apache.spark.sql.hive.execution._
 import org.apache.spark.sql.hive.test.TestHiveSingleton
@@ -30,11 +29,9 @@ import org.apache.spark.sql.internal.SQLConf
 class StatisticsSuite extends QueryTest with TestHiveSingleton {
   import hiveContext.sql
 
-  val parser = new HiveQl(SimpleParserConf())
-
   test("parse analyze commands") {
     def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) {
-      val parsed = parser.parsePlan(analyzeCommand)
+      val parsed = HiveSqlParser.parsePlan(analyzeCommand)
       val operators = parsed.collect {
         case a: AnalyzeTable => a
         case o => o


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[6/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
new file mode 100644
index 0000000..3b9f82a
--- /dev/null
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -0,0 +1,943 @@
+/*
+ * Licensed 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.
+ *
+ * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar.
+ */
+
+grammar SqlBase;
+
+tokens {
+    DELIMITER
+}
+
+singleStatement
+    : statement EOF
+    ;
+
+singleExpression
+    : namedExpression EOF
+    ;
+
+singleTableIdentifier
+    : tableIdentifier EOF
+    ;
+
+singleDataType
+    : dataType EOF
+    ;
+
+statement
+    : query                                                            #statementDefault
+    | USE db=identifier                                                #use
+    | CREATE DATABASE (IF NOT EXISTS)? identifier
+        (COMMENT comment=STRING)? locationSpec?
+        (WITH DBPROPERTIES tablePropertyList)?                         #createDatabase
+    | ALTER DATABASE identifier SET DBPROPERTIES tablePropertyList     #setDatabaseProperties
+    | DROP DATABASE (IF EXISTS)? identifier (RESTRICT | CASCADE)?      #dropDatabase
+    | createTableHeader ('(' colTypeList ')')? tableProvider
+        (OPTIONS tablePropertyList)?                                   #createTableUsing
+    | createTableHeader tableProvider
+        (OPTIONS tablePropertyList)? AS? query                         #createTableUsing
+    | createTableHeader ('(' columns=colTypeList ')')?
+        (COMMENT STRING)?
+        (PARTITIONED BY '(' partitionColumns=colTypeList ')')?
+        bucketSpec? skewSpec?
+        rowFormat?  createFileFormat? locationSpec?
+        (TBLPROPERTIES tablePropertyList)?
+        (AS? query)?                                                   #createTable
+    | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS
+        (identifier | FOR COLUMNS identifierSeq?)?                     #analyze
+    | ALTER TABLE from=tableIdentifier RENAME TO to=tableIdentifier    #renameTable
+    | ALTER TABLE tableIdentifier
+        SET TBLPROPERTIES tablePropertyList                            #setTableProperties
+    | ALTER TABLE tableIdentifier
+        UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList             #unsetTableProperties
+    | ALTER TABLE tableIdentifier (partitionSpec)?
+        SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)?     #setTableSerDe
+    | ALTER TABLE tableIdentifier (partitionSpec)?
+        SET SERDEPROPERTIES tablePropertyList                          #setTableSerDe
+    | ALTER TABLE tableIdentifier bucketSpec                           #bucketTable
+    | ALTER TABLE tableIdentifier NOT CLUSTERED                        #unclusterTable
+    | ALTER TABLE tableIdentifier NOT SORTED                           #unsortTable
+    | ALTER TABLE tableIdentifier skewSpec                             #skewTable
+    | ALTER TABLE tableIdentifier NOT SKEWED                           #unskewTable
+    | ALTER TABLE tableIdentifier NOT STORED AS DIRECTORIES            #unstoreTable
+    | ALTER TABLE tableIdentifier
+        SET SKEWED LOCATION skewedLocationList                         #setTableSkewLocations
+    | ALTER TABLE tableIdentifier ADD (IF NOT EXISTS)?
+        partitionSpecLocation+                                         #addTablePartition
+    | ALTER TABLE tableIdentifier
+        from=partitionSpec RENAME TO to=partitionSpec                  #renameTablePartition
+    | ALTER TABLE from=tableIdentifier
+        EXCHANGE partitionSpec WITH TABLE to=tableIdentifier           #exchangeTablePartition
+    | ALTER TABLE tableIdentifier
+        DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE?    #dropTablePartitions
+    | ALTER TABLE tableIdentifier ARCHIVE partitionSpec                #archiveTablePartition
+    | ALTER TABLE tableIdentifier UNARCHIVE partitionSpec              #unarchiveTablePartition
+    | ALTER TABLE tableIdentifier partitionSpec?
+        SET FILEFORMAT fileFormat                                      #setTableFileFormat
+    | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec      #setTableLocation
+    | ALTER TABLE tableIdentifier TOUCH partitionSpec?                 #touchTable
+    | ALTER TABLE tableIdentifier partitionSpec? COMPACT STRING        #compactTable
+    | ALTER TABLE tableIdentifier partitionSpec? CONCATENATE           #concatenateTable
+    | ALTER TABLE tableIdentifier partitionSpec?
+        CHANGE COLUMN? oldName=identifier colType
+        (FIRST | AFTER after=identifier)? (CASCADE | RESTRICT)?        #changeColumn
+    | ALTER TABLE tableIdentifier partitionSpec?
+        ADD COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)?          #addColumns
+    | ALTER TABLE tableIdentifier partitionSpec?
+        REPLACE COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)?      #replaceColumns
+    | DROP TABLE (IF EXISTS)? tableIdentifier PURGE?
+        (FOR METADATA? REPLICATION '(' STRING ')')?                    #dropTable
+    | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier
+        identifierCommentList? (COMMENT STRING)?
+        (PARTITIONED ON identifierList)?
+        (TBLPROPERTIES tablePropertyList)? AS query                    #createView
+    | ALTER VIEW tableIdentifier AS? query                             #alterViewQuery
+    | CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING
+        (USING resource (',' resource)*)?                              #createFunction
+    | DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName              #dropFunction
+    | EXPLAIN explainOption* statement                                 #explain
+    | SHOW TABLES ((FROM | IN) db=identifier)?
+        (LIKE (qualifiedName | pattern=STRING))?                       #showTables
+    | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))?         #showFunctions
+    | (DESC | DESCRIBE) FUNCTION EXTENDED? qualifiedName               #describeFunction
+    | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
+        tableIdentifier partitionSpec? describeColName?                #describeTable
+    | (DESC | DESCRIBE) DATABASE EXTENDED? identifier                  #describeDatabase
+    | REFRESH TABLE tableIdentifier                                    #refreshTable
+    | CACHE LAZY? TABLE identifier (AS? query)?                        #cacheTable
+    | UNCACHE TABLE identifier                                         #uncacheTable
+    | CLEAR CACHE                                                      #clearCache
+    | ADD identifier .*?                                               #addResource
+    | SET ROLE .*?                                                     #failNativeCommand
+    | SET .*?                                                          #setConfiguration
+    | kws=unsupportedHiveNativeCommands .*?                            #failNativeCommand
+    | hiveNativeCommands                                               #executeNativeCommand
+    ;
+
+hiveNativeCommands
+    : createTableHeader LIKE tableIdentifier
+        rowFormat?  createFileFormat? locationSpec?
+        (TBLPROPERTIES tablePropertyList)?
+    | DELETE FROM tableIdentifier (WHERE booleanExpression)?
+    | TRUNCATE TABLE tableIdentifier partitionSpec?
+        (COLUMNS identifierList)?
+    | ALTER VIEW from=tableIdentifier AS? RENAME TO to=tableIdentifier
+    | ALTER VIEW from=tableIdentifier AS?
+        SET TBLPROPERTIES tablePropertyList
+    | ALTER VIEW from=tableIdentifier AS?
+        UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList
+    | ALTER VIEW from=tableIdentifier AS?
+        ADD (IF NOT EXISTS)? partitionSpecLocation+
+    | ALTER VIEW from=tableIdentifier AS?
+        DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE?
+    | DROP VIEW (IF EXISTS)? qualifiedName
+    | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM|IN) identifier)?
+    | START TRANSACTION (transactionMode (',' transactionMode)*)?
+    | COMMIT WORK?
+    | ROLLBACK WORK?
+    | SHOW PARTITIONS tableIdentifier partitionSpec?
+    | DFS .*?
+    | (CREATE | ALTER | DROP | SHOW | DESC | DESCRIBE | LOCK | UNLOCK | MSCK | LOAD) .*?
+    ;
+
+unsupportedHiveNativeCommands
+    : kw1=CREATE kw2=ROLE
+    | kw1=DROP kw2=ROLE
+    | kw1=GRANT kw2=ROLE?
+    | kw1=REVOKE kw2=ROLE?
+    | kw1=SHOW kw2=GRANT
+    | kw1=SHOW kw2=ROLE kw3=GRANT?
+    | kw1=SHOW kw2=PRINCIPALS
+    | kw1=SHOW kw2=ROLES
+    | kw1=SHOW kw2=CURRENT kw3=ROLES
+    | kw1=EXPORT kw2=TABLE
+    | kw1=IMPORT kw2=TABLE
+    | kw1=SHOW kw2=COMPACTIONS
+    | kw1=SHOW kw2=CREATE kw3=TABLE
+    | kw1=SHOW kw2=TRANSACTIONS
+    | kw1=SHOW kw2=INDEXES
+    | kw1=SHOW kw2=LOCKS
+    ;
+
+createTableHeader
+    : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? tableIdentifier
+    ;
+
+bucketSpec
+    : CLUSTERED BY identifierList
+      (SORTED BY orderedIdentifierList)?
+      INTO INTEGER_VALUE BUCKETS
+    ;
+
+skewSpec
+    : SKEWED BY identifierList
+      ON (constantList | nestedConstantList)
+      (STORED AS DIRECTORIES)?
+    ;
+
+locationSpec
+    : LOCATION STRING
+    ;
+
+query
+    : ctes? queryNoWith
+    ;
+
+insertInto
+    : INSERT OVERWRITE TABLE tableIdentifier partitionSpec? (IF NOT EXISTS)?
+    | INSERT INTO TABLE? tableIdentifier partitionSpec?
+    ;
+
+partitionSpecLocation
+    : partitionSpec locationSpec?
+    ;
+
+partitionSpec
+    : PARTITION '(' partitionVal (',' partitionVal)* ')'
+    ;
+
+partitionVal
+    : identifier (EQ constant)?
+    ;
+
+describeColName
+    : identifier ('.' (identifier | STRING))*
+    ;
+
+ctes
+    : WITH namedQuery (',' namedQuery)*
+    ;
+
+namedQuery
+    : name=identifier AS? '(' queryNoWith ')'
+    ;
+
+tableProvider
+    : USING qualifiedName
+    ;
+
+tablePropertyList
+    : '(' tableProperty (',' tableProperty)* ')'
+    ;
+
+tableProperty
+    : key=tablePropertyKey (EQ? value=STRING)?
+    ;
+
+tablePropertyKey
+    : looseIdentifier ('.' looseIdentifier)*
+    | STRING
+    ;
+
+constantList
+    : '(' constant (',' constant)* ')'
+    ;
+
+nestedConstantList
+    : '(' constantList (',' constantList)* ')'
+    ;
+
+skewedLocation
+    : (constant | constantList) EQ STRING
+    ;
+
+skewedLocationList
+    : '(' skewedLocation (',' skewedLocation)* ')'
+    ;
+
+createFileFormat
+    : STORED AS fileFormat
+    | STORED BY storageHandler
+    ;
+
+fileFormat
+    : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)?
+      (INPUTDRIVER inDriver=STRING OUTPUTDRIVER outDriver=STRING)?                         #tableFileFormat
+    | identifier                                                                           #genericFileFormat
+    ;
+
+storageHandler
+    : STRING (WITH SERDEPROPERTIES tablePropertyList)?
+    ;
+
+resource
+    : identifier STRING
+    ;
+
+queryNoWith
+    : insertInto? queryTerm queryOrganization                                              #singleInsertQuery
+    | fromClause multiInsertQueryBody+                                                     #multiInsertQuery
+    ;
+
+queryOrganization
+    : (ORDER BY order+=sortItem (',' order+=sortItem)*)?
+      (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)?
+      (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)?
+      (SORT BY sort+=sortItem (',' sort+=sortItem)*)?
+      windows?
+      (LIMIT limit=expression)?
+    ;
+
+multiInsertQueryBody
+    : insertInto?
+      querySpecification
+      queryOrganization
+    ;
+
+queryTerm
+    : queryPrimary                                                                         #queryTermDefault
+    | left=queryTerm operator=(INTERSECT | UNION | EXCEPT) setQuantifier? right=queryTerm  #setOperation
+    ;
+
+queryPrimary
+    : querySpecification                                                    #queryPrimaryDefault
+    | TABLE tableIdentifier                                                 #table
+    | inlineTable                                                           #inlineTableDefault1
+    | '(' queryNoWith  ')'                                                  #subquery
+    ;
+
+sortItem
+    : expression ordering=(ASC | DESC)?
+    ;
+
+querySpecification
+    : (((SELECT kind=TRANSFORM '(' namedExpressionSeq ')'
+        | kind=MAP namedExpressionSeq
+        | kind=REDUCE namedExpressionSeq))
+       inRowFormat=rowFormat?
+       (RECORDWRITER recordWriter=STRING)?
+       USING script=STRING
+       (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))?
+       outRowFormat=rowFormat?
+       (RECORDREADER recordReader=STRING)?
+       fromClause?
+       (WHERE where=booleanExpression)?)
+    | ((kind=SELECT setQuantifier? namedExpressionSeq fromClause?
+       | fromClause (kind=SELECT setQuantifier? namedExpressionSeq)?)
+       lateralView*
+       (WHERE where=booleanExpression)?
+       aggregation?
+       (HAVING having=booleanExpression)?
+       windows?)
+    ;
+
+fromClause
+    : FROM relation (',' relation)* lateralView*
+    ;
+
+aggregation
+    : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* (
+      WITH kind=ROLLUP
+    | WITH kind=CUBE
+    | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')?
+    ;
+
+groupingSet
+    : '(' (expression (',' expression)*)? ')'
+    | expression
+    ;
+
+lateralView
+    : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)?
+    ;
+
+setQuantifier
+    : DISTINCT
+    | ALL
+    ;
+
+relation
+    : left=relation
+      ((CROSS | joinType) JOIN right=relation joinCriteria?
+      | NATURAL joinType JOIN right=relation
+      )                                           #joinRelation
+    | relationPrimary                             #relationDefault
+    ;
+
+joinType
+    : INNER?
+    | LEFT OUTER?
+    | LEFT SEMI
+    | RIGHT OUTER?
+    | FULL OUTER?
+    ;
+
+joinCriteria
+    : ON booleanExpression
+    | USING '(' identifier (',' identifier)* ')'
+    ;
+
+sample
+    : TABLESAMPLE '('
+      ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT)
+      | (expression sampleType=ROWS)
+      | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?))
+      ')'
+    ;
+
+identifierList
+    : '(' identifierSeq ')'
+    ;
+
+identifierSeq
+    : identifier (',' identifier)*
+    ;
+
+orderedIdentifierList
+    : '(' orderedIdentifier (',' orderedIdentifier)* ')'
+    ;
+
+orderedIdentifier
+    : identifier ordering=(ASC | DESC)?
+    ;
+
+identifierCommentList
+    : '(' identifierComment (',' identifierComment)* ')'
+    ;
+
+identifierComment
+    : identifier (COMMENT STRING)?
+    ;
+
+relationPrimary
+    : tableIdentifier sample? (AS? identifier)?                     #tableName
+    | '(' queryNoWith ')' sample? (AS? identifier)?                 #aliasedQuery
+    | '(' relation ')' sample? (AS? identifier)?                    #aliasedRelation
+    | inlineTable                                                   #inlineTableDefault2
+    ;
+
+inlineTable
+    : VALUES expression (',' expression)*  (AS? identifier identifierList?)?
+    ;
+
+rowFormat
+    : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tablePropertyList)?  #rowFormatSerde
+    | ROW FORMAT DELIMITED
+      (FIELDS TERMINATED BY fieldsTerminatedBy=STRING (ESCAPED BY escapedBy=STRING)?)?
+      (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)?
+      (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)?
+      (LINES TERMINATED BY linesSeparatedBy=STRING)?
+      (NULL DEFINED AS nullDefinedAs=STRING)?                                       #rowFormatDelimited
+    ;
+
+tableIdentifier
+    : (db=identifier '.')? table=identifier
+    ;
+
+namedExpression
+    : expression (AS? (identifier | identifierList))?
+    ;
+
+namedExpressionSeq
+    : namedExpression (',' namedExpression)*
+    ;
+
+expression
+    : booleanExpression
+    ;
+
+booleanExpression
+    : predicated                                                   #booleanDefault
+    | NOT booleanExpression                                        #logicalNot
+    | left=booleanExpression operator=AND right=booleanExpression  #logicalBinary
+    | left=booleanExpression operator=OR right=booleanExpression   #logicalBinary
+    | EXISTS '(' query ')'                                         #exists
+    ;
+
+// workaround for:
+//  https://github.com/antlr/antlr4/issues/780
+//  https://github.com/antlr/antlr4/issues/781
+predicated
+    : valueExpression predicate[$valueExpression.ctx]?
+    ;
+
+predicate[ParserRuleContext value]
+    : NOT? BETWEEN lower=valueExpression AND upper=valueExpression        #between
+    | NOT? IN '(' expression (',' expression)* ')'                        #inList
+    | NOT? IN '(' query ')'                                               #inSubquery
+    | NOT? like=(RLIKE | LIKE) pattern=valueExpression                    #like
+    | IS NOT? NULL                                                        #nullPredicate
+    ;
+
+valueExpression
+    : primaryExpression                                                                      #valueExpressionDefault
+    | operator=(MINUS | PLUS | TILDE) valueExpression                                        #arithmeticUnary
+    | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary
+    | left=valueExpression operator=(PLUS | MINUS) right=valueExpression                     #arithmeticBinary
+    | left=valueExpression operator=AMPERSAND right=valueExpression                          #arithmeticBinary
+    | left=valueExpression operator=HAT right=valueExpression                                #arithmeticBinary
+    | left=valueExpression operator=PIPE right=valueExpression                               #arithmeticBinary
+    | left=valueExpression comparisonOperator right=valueExpression                          #comparison
+    ;
+
+primaryExpression
+    : constant                                                                                 #constantDefault
+    | ASTERISK                                                                                 #star
+    | qualifiedName '.' ASTERISK                                                               #star
+    | '(' expression (',' expression)+ ')'                                                     #rowConstructor
+    | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)?  #functionCall
+    | '(' query ')'                                                                            #subqueryExpression
+    | CASE valueExpression whenClause+ (ELSE elseExpression=expression)? END                   #simpleCase
+    | CASE whenClause+ (ELSE elseExpression=expression)? END                                   #searchedCase
+    | CAST '(' expression AS dataType ')'                                                      #cast
+    | value=primaryExpression '[' index=valueExpression ']'                                    #subscript
+    | identifier                                                                               #columnReference
+    | base=primaryExpression '.' fieldName=identifier                                          #dereference
+    | '(' expression ')'                                                                       #parenthesizedExpression
+    ;
+
+constant
+    : NULL                                                                                     #nullLiteral
+    | interval                                                                                 #intervalLiteral
+    | identifier STRING                                                                        #typeConstructor
+    | number                                                                                   #numericLiteral
+    | booleanValue                                                                             #booleanLiteral
+    | STRING+                                                                                  #stringLiteral
+    ;
+
+comparisonOperator
+    : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ
+    ;
+
+booleanValue
+    : TRUE | FALSE
+    ;
+
+interval
+    : INTERVAL intervalField*
+    ;
+
+intervalField
+    : value=intervalValue unit=identifier (TO to=identifier)?
+    ;
+
+intervalValue
+    : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE)
+    | STRING
+    ;
+
+dataType
+    : complex=ARRAY '<' dataType '>'                            #complexDataType
+    | complex=MAP '<' dataType ',' dataType '>'                 #complexDataType
+    | complex=STRUCT ('<' colTypeList? '>' | NEQ)               #complexDataType
+    | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')?  #primitiveDataType
+    ;
+
+colTypeList
+    : colType (',' colType)*
+    ;
+
+colType
+    : identifier ':'? dataType (COMMENT STRING)?
+    ;
+
+whenClause
+    : WHEN condition=expression THEN result=expression
+    ;
+
+windows
+    : WINDOW namedWindow (',' namedWindow)*
+    ;
+
+namedWindow
+    : identifier AS windowSpec
+    ;
+
+windowSpec
+    : name=identifier  #windowRef
+    | '('
+      ( CLUSTER BY partition+=expression (',' partition+=expression)*
+      | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)?
+        ((ORDER | SORT) BY sortItem (',' sortItem)*)?)
+      windowFrame?
+      ')'              #windowDef
+    ;
+
+windowFrame
+    : frameType=RANGE start=frameBound
+    | frameType=ROWS start=frameBound
+    | frameType=RANGE BETWEEN start=frameBound AND end=frameBound
+    | frameType=ROWS BETWEEN start=frameBound AND end=frameBound
+    ;
+
+frameBound
+    : UNBOUNDED boundType=(PRECEDING | FOLLOWING)
+    | boundType=CURRENT ROW
+    | expression boundType=(PRECEDING | FOLLOWING)
+    ;
+
+
+explainOption
+    : LOGICAL | FORMATTED | EXTENDED
+    ;
+
+transactionMode
+    : ISOLATION LEVEL SNAPSHOT            #isolationLevel
+    | READ accessMode=(ONLY | WRITE)      #transactionAccessMode
+    ;
+
+qualifiedName
+    : identifier ('.' identifier)*
+    ;
+
+// Identifier that also allows the use of a number of SQL keywords (mainly for backwards compatibility).
+looseIdentifier
+    : identifier
+    | FROM
+    | TO
+    | TABLE
+    | WITH
+    ;
+
+identifier
+    : IDENTIFIER             #unquotedIdentifier
+    | quotedIdentifier       #quotedIdentifierAlternative
+    | nonReserved            #unquotedIdentifier
+    ;
+
+quotedIdentifier
+    : BACKQUOTED_IDENTIFIER
+    ;
+
+number
+    : DECIMAL_VALUE            #decimalLiteral
+    | SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral
+    | INTEGER_VALUE            #integerLiteral
+    | BIGINT_LITERAL           #bigIntLiteral
+    | SMALLINT_LITERAL         #smallIntLiteral
+    | TINYINT_LITERAL          #tinyIntLiteral
+    | DOUBLE_LITERAL           #doubleLiteral
+    ;
+
+nonReserved
+    : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS
+    | ADD
+    | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | MAP | ARRAY | STRUCT
+    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
+    | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED
+    | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS
+    | GROUPING | CUBE | ROLLUP
+    | EXPLAIN | FORMAT | LOGICAL | FORMATTED
+    | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF
+    | SET
+    | VIEW | REPLACE
+    | IF
+    | NO | DATA
+    | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL
+    | SNAPSHOT | READ | WRITE | ONLY
+    | SORT | CLUSTER | DISTRIBUTE UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION
+    | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST
+    | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT
+    | INPUTDRIVER | OUTPUTDRIVER | DBPROPERTIES | DFS | TRUNCATE | METADATA | REPLICATION | COMPUTE
+    | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER
+    | REVOKE | GRANT | LOCK | UNLOCK | MSCK | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE
+    | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEXES | LOCKS | OPTION
+    ;
+
+SELECT: 'SELECT';
+FROM: 'FROM';
+ADD: 'ADD';
+AS: 'AS';
+ALL: 'ALL';
+DISTINCT: 'DISTINCT';
+WHERE: 'WHERE';
+GROUP: 'GROUP';
+BY: 'BY';
+GROUPING: 'GROUPING';
+SETS: 'SETS';
+CUBE: 'CUBE';
+ROLLUP: 'ROLLUP';
+ORDER: 'ORDER';
+HAVING: 'HAVING';
+LIMIT: 'LIMIT';
+AT: 'AT';
+OR: 'OR';
+AND: 'AND';
+IN: 'IN';
+NOT: 'NOT' | '!';
+NO: 'NO';
+EXISTS: 'EXISTS';
+BETWEEN: 'BETWEEN';
+LIKE: 'LIKE';
+RLIKE: 'RLIKE' | 'REGEXP';
+IS: 'IS';
+NULL: 'NULL';
+TRUE: 'TRUE';
+FALSE: 'FALSE';
+NULLS: 'NULLS';
+ASC: 'ASC';
+DESC: 'DESC';
+FOR: 'FOR';
+INTERVAL: 'INTERVAL';
+CASE: 'CASE';
+WHEN: 'WHEN';
+THEN: 'THEN';
+ELSE: 'ELSE';
+END: 'END';
+JOIN: 'JOIN';
+CROSS: 'CROSS';
+OUTER: 'OUTER';
+INNER: 'INNER';
+LEFT: 'LEFT';
+SEMI: 'SEMI';
+RIGHT: 'RIGHT';
+FULL: 'FULL';
+NATURAL: 'NATURAL';
+ON: 'ON';
+LATERAL: 'LATERAL';
+WINDOW: 'WINDOW';
+OVER: 'OVER';
+PARTITION: 'PARTITION';
+RANGE: 'RANGE';
+ROWS: 'ROWS';
+UNBOUNDED: 'UNBOUNDED';
+PRECEDING: 'PRECEDING';
+FOLLOWING: 'FOLLOWING';
+CURRENT: 'CURRENT';
+ROW: 'ROW';
+WITH: 'WITH';
+VALUES: 'VALUES';
+CREATE: 'CREATE';
+TABLE: 'TABLE';
+VIEW: 'VIEW';
+REPLACE: 'REPLACE';
+INSERT: 'INSERT';
+DELETE: 'DELETE';
+INTO: 'INTO';
+DESCRIBE: 'DESCRIBE';
+EXPLAIN: 'EXPLAIN';
+FORMAT: 'FORMAT';
+LOGICAL: 'LOGICAL';
+CAST: 'CAST';
+SHOW: 'SHOW';
+TABLES: 'TABLES';
+COLUMNS: 'COLUMNS';
+COLUMN: 'COLUMN';
+USE: 'USE';
+PARTITIONS: 'PARTITIONS';
+FUNCTIONS: 'FUNCTIONS';
+DROP: 'DROP';
+UNION: 'UNION';
+EXCEPT: 'EXCEPT';
+INTERSECT: 'INTERSECT';
+TO: 'TO';
+TABLESAMPLE: 'TABLESAMPLE';
+STRATIFY: 'STRATIFY';
+ALTER: 'ALTER';
+RENAME: 'RENAME';
+ARRAY: 'ARRAY';
+MAP: 'MAP';
+STRUCT: 'STRUCT';
+COMMENT: 'COMMENT';
+SET: 'SET';
+DATA: 'DATA';
+START: 'START';
+TRANSACTION: 'TRANSACTION';
+COMMIT: 'COMMIT';
+ROLLBACK: 'ROLLBACK';
+WORK: 'WORK';
+ISOLATION: 'ISOLATION';
+LEVEL: 'LEVEL';
+SNAPSHOT: 'SNAPSHOT';
+READ: 'READ';
+WRITE: 'WRITE';
+ONLY: 'ONLY';
+
+IF: 'IF';
+
+EQ  : '=' | '==';
+NSEQ: '<=>';
+NEQ : '<>';
+NEQJ: '!=';
+LT  : '<';
+LTE : '<=';
+GT  : '>';
+GTE : '>=';
+
+PLUS: '+';
+MINUS: '-';
+ASTERISK: '*';
+SLASH: '/';
+PERCENT: '%';
+DIV: 'DIV';
+TILDE: '~';
+AMPERSAND: '&';
+PIPE: '|';
+HAT: '^';
+
+PERCENTLIT: 'PERCENT';
+BUCKET: 'BUCKET';
+OUT: 'OUT';
+OF: 'OF';
+
+SORT: 'SORT';
+CLUSTER: 'CLUSTER';
+DISTRIBUTE: 'DISTRIBUTE';
+OVERWRITE: 'OVERWRITE';
+TRANSFORM: 'TRANSFORM';
+REDUCE: 'REDUCE';
+USING: 'USING';
+SERDE: 'SERDE';
+SERDEPROPERTIES: 'SERDEPROPERTIES';
+RECORDREADER: 'RECORDREADER';
+RECORDWRITER: 'RECORDWRITER';
+DELIMITED: 'DELIMITED';
+FIELDS: 'FIELDS';
+TERMINATED: 'TERMINATED';
+COLLECTION: 'COLLECTION';
+ITEMS: 'ITEMS';
+KEYS: 'KEYS';
+ESCAPED: 'ESCAPED';
+LINES: 'LINES';
+SEPARATED: 'SEPARATED';
+FUNCTION: 'FUNCTION';
+EXTENDED: 'EXTENDED';
+REFRESH: 'REFRESH';
+CLEAR: 'CLEAR';
+CACHE: 'CACHE';
+UNCACHE: 'UNCACHE';
+LAZY: 'LAZY';
+FORMATTED: 'FORMATTED';
+TEMPORARY: 'TEMPORARY' | 'TEMP';
+OPTIONS: 'OPTIONS';
+UNSET: 'UNSET';
+TBLPROPERTIES: 'TBLPROPERTIES';
+DBPROPERTIES: 'DBPROPERTIES';
+BUCKETS: 'BUCKETS';
+SKEWED: 'SKEWED';
+STORED: 'STORED';
+DIRECTORIES: 'DIRECTORIES';
+LOCATION: 'LOCATION';
+EXCHANGE: 'EXCHANGE';
+ARCHIVE: 'ARCHIVE';
+UNARCHIVE: 'UNARCHIVE';
+FILEFORMAT: 'FILEFORMAT';
+TOUCH: 'TOUCH';
+COMPACT: 'COMPACT';
+CONCATENATE: 'CONCATENATE';
+CHANGE: 'CHANGE';
+FIRST: 'FIRST';
+AFTER: 'AFTER';
+CASCADE: 'CASCADE';
+RESTRICT: 'RESTRICT';
+CLUSTERED: 'CLUSTERED';
+SORTED: 'SORTED';
+PURGE: 'PURGE';
+INPUTFORMAT: 'INPUTFORMAT';
+OUTPUTFORMAT: 'OUTPUTFORMAT';
+INPUTDRIVER: 'INPUTDRIVER';
+OUTPUTDRIVER: 'OUTPUTDRIVER';
+DATABASE: 'DATABASE' | 'SCHEMA';
+DFS: 'DFS';
+TRUNCATE: 'TRUNCATE';
+METADATA: 'METADATA';
+REPLICATION: 'REPLICATION';
+ANALYZE: 'ANALYZE';
+COMPUTE: 'COMPUTE';
+STATISTICS: 'STATISTICS';
+PARTITIONED: 'PARTITIONED';
+EXTERNAL: 'EXTERNAL';
+DEFINED: 'DEFINED';
+REVOKE: 'REVOKE';
+GRANT: 'GRANT';
+LOCK: 'LOCK';
+UNLOCK: 'UNLOCK';
+MSCK: 'MSCK';
+EXPORT: 'EXPORT';
+IMPORT: 'IMPORT';
+LOAD: 'LOAD';
+ROLE: 'ROLE';
+ROLES: 'ROLES';
+COMPACTIONS: 'COMPACTIONS';
+PRINCIPALS: 'PRINCIPALS';
+TRANSACTIONS: 'TRANSACTIONS';
+INDEXES: 'INDEXES';
+LOCKS: 'LOCKS';
+OPTION: 'OPTION';
+
+STRING
+    : '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
+    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
+    ;
+
+BIGINT_LITERAL
+    : DIGIT+ 'L'
+    ;
+
+SMALLINT_LITERAL
+    : DIGIT+ 'S'
+    ;
+
+TINYINT_LITERAL
+    : DIGIT+ 'Y'
+    ;
+
+INTEGER_VALUE
+    : DIGIT+
+    ;
+
+DECIMAL_VALUE
+    : DIGIT+ '.' DIGIT*
+    | '.' DIGIT+
+    ;
+
+SCIENTIFIC_DECIMAL_VALUE
+    : DIGIT+ ('.' DIGIT*)? EXPONENT
+    | '.' DIGIT+ EXPONENT
+    ;
+
+DOUBLE_LITERAL
+    :
+    (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'D'
+    ;
+
+IDENTIFIER
+    : (LETTER | DIGIT | '_')+
+    ;
+
+BACKQUOTED_IDENTIFIER
+    : '`' ( ~'`' | '``' )* '`'
+    ;
+
+fragment EXPONENT
+    : 'E' [+-]? DIGIT+
+    ;
+
+fragment DIGIT
+    : [0-9]
+    ;
+
+fragment LETTER
+    : [A-Z]
+    ;
+
+SIMPLE_COMMENT
+    : '--' ~[\r\n]* '\r'? '\n'? -> channel(HIDDEN)
+    ;
+
+BRACKETED_COMMENT
+    : '/*' .*? '*/' -> channel(HIDDEN)
+    ;
+
+WS
+    : [ \r\n\t]+ -> channel(HIDDEN)
+    ;
+
+// Catch-all for anything we can't recognize.
+// We use this to be able to ignore and recover all the text
+// when splitting statements with DelimiterLexer
+UNRECOGNIZED
+    : .
+    ;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4
deleted file mode 100644
index 4e77b6d..0000000
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4
+++ /dev/null
@@ -1,941 +0,0 @@
-/*
- * Licensed 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.
- *
- * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar.
- */
-
-grammar SqlBase;
-
-tokens {
-    DELIMITER
-}
-
-singleStatement
-    : statement EOF
-    ;
-
-singleExpression
-    : namedExpression EOF
-    ;
-
-singleTableIdentifier
-    : tableIdentifier EOF
-    ;
-
-singleDataType
-    : dataType EOF
-    ;
-
-statement
-    : query                                                            #statementDefault
-    | USE db=identifier                                                #use
-    | CREATE DATABASE (IF NOT EXISTS)? identifier
-        (COMMENT comment=STRING)? locationSpec?
-        (WITH DBPROPERTIES tablePropertyList)?                         #createDatabase
-    | ALTER DATABASE identifier SET DBPROPERTIES tablePropertyList     #setDatabaseProperties
-    | DROP DATABASE (IF EXISTS)? identifier (RESTRICT | CASCADE)?      #dropDatabase
-    | createTableHeader ('(' colTypeList ')')? tableProvider
-        (OPTIONS tablePropertyList)?                                   #createTableUsing
-    | createTableHeader tableProvider
-        (OPTIONS tablePropertyList)? AS? query                         #createTableUsing
-    | createTableHeader ('(' colTypeList ')')? (COMMENT STRING)?
-        (PARTITIONED BY identifierList)? bucketSpec? skewSpec?
-        rowFormat?  createFileFormat? locationSpec?
-        (TBLPROPERTIES tablePropertyList)?
-        (AS? query)?                                                   #createTable
-    | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS
-        (identifier | FOR COLUMNS identifierSeq?)                      #analyze
-    | ALTER TABLE from=tableIdentifier RENAME TO to=tableIdentifier    #renameTable
-    | ALTER TABLE tableIdentifier
-        SET TBLPROPERTIES tablePropertyList                            #setTableProperties
-    | ALTER TABLE tableIdentifier
-        UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList             #unsetTableProperties
-    | ALTER TABLE tableIdentifier (partitionSpec)?
-        SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)?     #setTableSerDe
-    | ALTER TABLE tableIdentifier (partitionSpec)?
-        SET SERDEPROPERTIES tablePropertyList                          #setTableSerDe
-    | ALTER TABLE tableIdentifier bucketSpec                           #bucketTable
-    | ALTER TABLE tableIdentifier NOT CLUSTERED                        #unclusterTable
-    | ALTER TABLE tableIdentifier NOT SORTED                           #unsortTable
-    | ALTER TABLE tableIdentifier skewSpec                             #skewTable
-    | ALTER TABLE tableIdentifier NOT SKEWED                           #unskewTable
-    | ALTER TABLE tableIdentifier NOT STORED AS DIRECTORIES            #unstoreTable
-    | ALTER TABLE tableIdentifier
-        SET SKEWED LOCATION skewedLocationList                         #setTableSkewLocations
-    | ALTER TABLE tableIdentifier ADD (IF NOT EXISTS)?
-        partitionSpecLocation+                                         #addTablePartition
-    | ALTER TABLE tableIdentifier
-        from=partitionSpec RENAME TO to=partitionSpec                  #renameTablePartition
-    | ALTER TABLE from=tableIdentifier
-        EXCHANGE partitionSpec WITH TABLE to=tableIdentifier           #exchangeTablePartition
-    | ALTER TABLE tableIdentifier
-        DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE?    #dropTablePartitions
-    | ALTER TABLE tableIdentifier ARCHIVE partitionSpec                #archiveTablePartition
-    | ALTER TABLE tableIdentifier UNARCHIVE partitionSpec              #unarchiveTablePartition
-    | ALTER TABLE tableIdentifier partitionSpec?
-        SET FILEFORMAT fileFormat                                      #setTableFileFormat
-    | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec      #setTableLocation
-    | ALTER TABLE tableIdentifier TOUCH partitionSpec?                 #touchTable
-    | ALTER TABLE tableIdentifier partitionSpec? COMPACT STRING        #compactTable
-    | ALTER TABLE tableIdentifier partitionSpec? CONCATENATE           #concatenateTable
-    | ALTER TABLE tableIdentifier partitionSpec?
-        CHANGE COLUMN? oldName=identifier colType
-        (FIRST | AFTER after=identifier)? (CASCADE | RESTRICT)?        #changeColumn
-    | ALTER TABLE tableIdentifier partitionSpec?
-        ADD COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)?          #addColumns
-    | ALTER TABLE tableIdentifier partitionSpec?
-        REPLACE COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)?      #replaceColumns
-    | DROP TABLE (IF EXISTS)? tableIdentifier PURGE?
-        (FOR METADATA? REPLICATION '(' STRING ')')?                    #dropTable
-    | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier
-        identifierCommentList? (COMMENT STRING)?
-        (PARTITIONED ON identifierList)?
-        (TBLPROPERTIES tablePropertyList)? AS query                    #createView
-    | ALTER VIEW tableIdentifier AS? query                             #alterViewQuery
-    | CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING
-        (USING resource (',' resource)*)?                              #createFunction
-    | DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName              #dropFunction
-    | EXPLAIN explainOption* statement                                 #explain
-    | SHOW TABLES ((FROM | IN) db=identifier)?
-        (LIKE (qualifiedName | pattern=STRING))?                       #showTables
-    | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))?         #showFunctions
-    | (DESC | DESCRIBE) FUNCTION EXTENDED? qualifiedName               #describeFunction
-    | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
-        tableIdentifier partitionSpec? describeColName?                #describeTable
-    | (DESC | DESCRIBE) DATABASE EXTENDED? identifier                  #describeDatabase
-    | REFRESH TABLE tableIdentifier                                    #refreshTable
-    | CACHE LAZY? TABLE identifier (AS? query)?                        #cacheTable
-    | UNCACHE TABLE identifier                                         #uncacheTable
-    | CLEAR CACHE                                                      #clearCache
-    | ADD identifier .*?                                               #addResource
-    | SET ROLE .*?                                                     #failNativeCommand
-    | SET .*?                                                          #setConfiguration
-    | kws=unsupportedHiveNativeCommands .*?                            #failNativeCommand
-    | hiveNativeCommands                                               #executeNativeCommand
-    ;
-
-hiveNativeCommands
-    : createTableHeader LIKE tableIdentifier
-        rowFormat?  createFileFormat? locationSpec?
-        (TBLPROPERTIES tablePropertyList)?
-    | DELETE FROM tableIdentifier (WHERE booleanExpression)?
-    | TRUNCATE TABLE tableIdentifier partitionSpec?
-        (COLUMNS identifierList)?
-    | ALTER VIEW from=tableIdentifier AS? RENAME TO to=tableIdentifier
-    | ALTER VIEW from=tableIdentifier AS?
-        SET TBLPROPERTIES tablePropertyList
-    | ALTER VIEW from=tableIdentifier AS?
-        UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList
-    | ALTER VIEW from=tableIdentifier AS?
-        ADD (IF NOT EXISTS)? partitionSpecLocation+
-    | ALTER VIEW from=tableIdentifier AS?
-        DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE?
-    | DROP VIEW (IF EXISTS)? qualifiedName
-    | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM|IN) identifier)?
-    | START TRANSACTION (transactionMode (',' transactionMode)*)?
-    | COMMIT WORK?
-    | ROLLBACK WORK?
-    | SHOW PARTITIONS tableIdentifier partitionSpec?
-    | DFS .*?
-    | (CREATE | ALTER | DROP | SHOW | DESC | DESCRIBE | LOCK | UNLOCK | MSCK | LOAD) .*?
-    ;
-
-unsupportedHiveNativeCommands
-    : kw1=CREATE kw2=ROLE
-    | kw1=DROP kw2=ROLE
-    | kw1=GRANT kw2=ROLE?
-    | kw1=REVOKE kw2=ROLE?
-    | kw1=SHOW kw2=GRANT
-    | kw1=SHOW kw2=ROLE kw3=GRANT?
-    | kw1=SHOW kw2=PRINCIPALS
-    | kw1=SHOW kw2=ROLES
-    | kw1=SHOW kw2=CURRENT kw3=ROLES
-    | kw1=EXPORT kw2=TABLE
-    | kw1=IMPORT kw2=TABLE
-    | kw1=SHOW kw2=COMPACTIONS
-    | kw1=SHOW kw2=CREATE kw3=TABLE
-    | kw1=SHOW kw2=TRANSACTIONS
-    | kw1=SHOW kw2=INDEXES
-    | kw1=SHOW kw2=LOCKS
-    ;
-
-createTableHeader
-    : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? tableIdentifier
-    ;
-
-bucketSpec
-    : CLUSTERED BY identifierList
-      (SORTED BY orderedIdentifierList)?
-      INTO INTEGER_VALUE BUCKETS
-    ;
-
-skewSpec
-    : SKEWED BY identifierList
-      ON (constantList | nestedConstantList)
-      (STORED AS DIRECTORIES)?
-    ;
-
-locationSpec
-    : LOCATION STRING
-    ;
-
-query
-    : ctes? queryNoWith
-    ;
-
-insertInto
-    : INSERT OVERWRITE TABLE tableIdentifier partitionSpec? (IF NOT EXISTS)?
-    | INSERT INTO TABLE? tableIdentifier partitionSpec?
-    ;
-
-partitionSpecLocation
-    : partitionSpec locationSpec?
-    ;
-
-partitionSpec
-    : PARTITION '(' partitionVal (',' partitionVal)* ')'
-    ;
-
-partitionVal
-    : identifier (EQ constant)?
-    ;
-
-describeColName
-    : identifier ('.' (identifier | STRING))*
-    ;
-
-ctes
-    : WITH namedQuery (',' namedQuery)*
-    ;
-
-namedQuery
-    : name=identifier AS? '(' queryNoWith ')'
-    ;
-
-tableProvider
-    : USING qualifiedName
-    ;
-
-tablePropertyList
-    : '(' tableProperty (',' tableProperty)* ')'
-    ;
-
-tableProperty
-    : key=tablePropertyKey (EQ? value=STRING)?
-    ;
-
-tablePropertyKey
-    : looseIdentifier ('.' looseIdentifier)*
-    | STRING
-    ;
-
-constantList
-    : '(' constant (',' constant)* ')'
-    ;
-
-nestedConstantList
-    : '(' constantList (',' constantList)* ')'
-    ;
-
-skewedLocation
-    : (constant | constantList) EQ STRING
-    ;
-
-skewedLocationList
-    : '(' skewedLocation (',' skewedLocation)* ')'
-    ;
-
-createFileFormat
-    : STORED AS fileFormat
-    | STORED BY storageHandler
-    ;
-
-fileFormat
-    : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)?
-      (INPUTDRIVER inDriver=STRING OUTPUTDRIVER outDriver=STRING)?                         #tableFileFormat
-    | identifier                                                                           #genericFileFormat
-    ;
-
-storageHandler
-    : STRING (WITH SERDEPROPERTIES tablePropertyList)?
-    ;
-
-resource
-    : identifier STRING
-    ;
-
-queryNoWith
-    : insertInto? queryTerm queryOrganization                                              #singleInsertQuery
-    | fromClause multiInsertQueryBody+                                                     #multiInsertQuery
-    ;
-
-queryOrganization
-    : (ORDER BY order+=sortItem (',' order+=sortItem)*)?
-      (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)?
-      (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)?
-      (SORT BY sort+=sortItem (',' sort+=sortItem)*)?
-      windows?
-      (LIMIT limit=expression)?
-    ;
-
-multiInsertQueryBody
-    : insertInto?
-      querySpecification
-      queryOrganization
-    ;
-
-queryTerm
-    : queryPrimary                                                                         #queryTermDefault
-    | left=queryTerm operator=(INTERSECT | UNION | EXCEPT) setQuantifier? right=queryTerm  #setOperation
-    ;
-
-queryPrimary
-    : querySpecification                                                    #queryPrimaryDefault
-    | TABLE tableIdentifier                                                 #table
-    | inlineTable                                                           #inlineTableDefault1
-    | '(' queryNoWith  ')'                                                  #subquery
-    ;
-
-sortItem
-    : expression ordering=(ASC | DESC)?
-    ;
-
-querySpecification
-    : (((SELECT kind=TRANSFORM '(' namedExpressionSeq ')'
-        | kind=MAP namedExpressionSeq
-        | kind=REDUCE namedExpressionSeq))
-       inRowFormat=rowFormat?
-       (RECORDWRITER recordWriter=STRING)?
-       USING script=STRING
-       (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))?
-       outRowFormat=rowFormat?
-       (RECORDREADER recordReader=STRING)?
-       fromClause?
-       (WHERE where=booleanExpression)?)
-    | ((kind=SELECT setQuantifier? namedExpressionSeq fromClause?
-       | fromClause (kind=SELECT setQuantifier? namedExpressionSeq)?)
-       lateralView*
-       (WHERE where=booleanExpression)?
-       aggregation?
-       (HAVING having=booleanExpression)?
-       windows?)
-    ;
-
-fromClause
-    : FROM relation (',' relation)* lateralView*
-    ;
-
-aggregation
-    : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* (
-      WITH kind=ROLLUP
-    | WITH kind=CUBE
-    | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')?
-    ;
-
-groupingSet
-    : '(' (expression (',' expression)*)? ')'
-    | expression
-    ;
-
-lateralView
-    : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)?
-    ;
-
-setQuantifier
-    : DISTINCT
-    | ALL
-    ;
-
-relation
-    : left=relation
-      ((CROSS | joinType) JOIN right=relation joinCriteria?
-      | NATURAL joinType JOIN right=relation
-      )                                           #joinRelation
-    | relationPrimary                             #relationDefault
-    ;
-
-joinType
-    : INNER?
-    | LEFT OUTER?
-    | LEFT SEMI
-    | RIGHT OUTER?
-    | FULL OUTER?
-    ;
-
-joinCriteria
-    : ON booleanExpression
-    | USING '(' identifier (',' identifier)* ')'
-    ;
-
-sample
-    : TABLESAMPLE '('
-      ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT)
-      | (expression sampleType=ROWS)
-      | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?))
-      ')'
-    ;
-
-identifierList
-    : '(' identifierSeq ')'
-    ;
-
-identifierSeq
-    : identifier (',' identifier)*
-    ;
-
-orderedIdentifierList
-    : '(' orderedIdentifier (',' orderedIdentifier)* ')'
-    ;
-
-orderedIdentifier
-    : identifier ordering=(ASC | DESC)?
-    ;
-
-identifierCommentList
-    : '(' identifierComment (',' identifierComment)* ')'
-    ;
-
-identifierComment
-    : identifier (COMMENT STRING)?
-    ;
-
-relationPrimary
-    : tableIdentifier sample? (AS? identifier)?                     #tableName
-    | '(' queryNoWith ')' sample? (AS? identifier)?                 #aliasedQuery
-    | '(' relation ')' sample? (AS? identifier)?                    #aliasedRelation
-    | inlineTable                                                   #inlineTableDefault2
-    ;
-
-inlineTable
-    : VALUES expression (',' expression)*  (AS? identifier identifierList?)?
-    ;
-
-rowFormat
-    : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tablePropertyList)?  #rowFormatSerde
-    | ROW FORMAT DELIMITED
-      (FIELDS TERMINATED BY fieldsTerminatedBy=STRING (ESCAPED BY escapedBy=STRING)?)?
-      (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)?
-      (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)?
-      (LINES TERMINATED BY linesSeparatedBy=STRING)?
-      (NULL DEFINED AS nullDefinedAs=STRING)?                                       #rowFormatDelimited
-    ;
-
-tableIdentifier
-    : (db=identifier '.')? table=identifier
-    ;
-
-namedExpression
-    : expression (AS? (identifier | identifierList))?
-    ;
-
-namedExpressionSeq
-    : namedExpression (',' namedExpression)*
-    ;
-
-expression
-    : booleanExpression
-    ;
-
-booleanExpression
-    : predicated                                                   #booleanDefault
-    | NOT booleanExpression                                        #logicalNot
-    | left=booleanExpression operator=AND right=booleanExpression  #logicalBinary
-    | left=booleanExpression operator=OR right=booleanExpression   #logicalBinary
-    | EXISTS '(' query ')'                                         #exists
-    ;
-
-// workaround for:
-//  https://github.com/antlr/antlr4/issues/780
-//  https://github.com/antlr/antlr4/issues/781
-predicated
-    : valueExpression predicate[$valueExpression.ctx]?
-    ;
-
-predicate[ParserRuleContext value]
-    : NOT? BETWEEN lower=valueExpression AND upper=valueExpression        #between
-    | NOT? IN '(' expression (',' expression)* ')'                        #inList
-    | NOT? IN '(' query ')'                                               #inSubquery
-    | NOT? like=(RLIKE | LIKE) pattern=valueExpression                    #like
-    | IS NOT? NULL                                                        #nullPredicate
-    ;
-
-valueExpression
-    : primaryExpression                                                                      #valueExpressionDefault
-    | operator=(MINUS | PLUS | TILDE) valueExpression                                        #arithmeticUnary
-    | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary
-    | left=valueExpression operator=(PLUS | MINUS) right=valueExpression                     #arithmeticBinary
-    | left=valueExpression operator=AMPERSAND right=valueExpression                          #arithmeticBinary
-    | left=valueExpression operator=HAT right=valueExpression                                #arithmeticBinary
-    | left=valueExpression operator=PIPE right=valueExpression                               #arithmeticBinary
-    | left=valueExpression comparisonOperator right=valueExpression                          #comparison
-    ;
-
-primaryExpression
-    : constant                                                                                 #constantDefault
-    | ASTERISK                                                                                 #star
-    | qualifiedName '.' ASTERISK                                                               #star
-    | '(' expression (',' expression)+ ')'                                                     #rowConstructor
-    | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)?  #functionCall
-    | '(' query ')'                                                                            #subqueryExpression
-    | CASE valueExpression whenClause+ (ELSE elseExpression=expression)? END                   #simpleCase
-    | CASE whenClause+ (ELSE elseExpression=expression)? END                                   #searchedCase
-    | CAST '(' expression AS dataType ')'                                                      #cast
-    | value=primaryExpression '[' index=valueExpression ']'                                    #subscript
-    | identifier                                                                               #columnReference
-    | base=primaryExpression '.' fieldName=identifier                                          #dereference
-    | '(' expression ')'                                                                       #parenthesizedExpression
-    ;
-
-constant
-    : NULL                                                                                     #nullLiteral
-    | interval                                                                                 #intervalLiteral
-    | identifier STRING                                                                        #typeConstructor
-    | number                                                                                   #numericLiteral
-    | booleanValue                                                                             #booleanLiteral
-    | STRING+                                                                                  #stringLiteral
-    ;
-
-comparisonOperator
-    : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ
-    ;
-
-booleanValue
-    : TRUE | FALSE
-    ;
-
-interval
-    : INTERVAL intervalField*
-    ;
-
-intervalField
-    : value=intervalValue unit=identifier (TO to=identifier)?
-    ;
-
-intervalValue
-    : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE)
-    | STRING
-    ;
-
-dataType
-    : complex=ARRAY '<' dataType '>'                            #complexDataType
-    | complex=MAP '<' dataType ',' dataType '>'                 #complexDataType
-    | complex=STRUCT ('<' colTypeList? '>' | NEQ)               #complexDataType
-    | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')?  #primitiveDataType
-    ;
-
-colTypeList
-    : colType (',' colType)*
-    ;
-
-colType
-    : identifier ':'? dataType (COMMENT STRING)?
-    ;
-
-whenClause
-    : WHEN condition=expression THEN result=expression
-    ;
-
-windows
-    : WINDOW namedWindow (',' namedWindow)*
-    ;
-
-namedWindow
-    : identifier AS windowSpec
-    ;
-
-windowSpec
-    : name=identifier  #windowRef
-    | '('
-      ( CLUSTER BY partition+=expression (',' partition+=expression)*
-      | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)?
-        ((ORDER | SORT) BY sortItem (',' sortItem)*)?)
-      windowFrame?
-      ')'              #windowDef
-    ;
-
-windowFrame
-    : frameType=RANGE start=frameBound
-    | frameType=ROWS start=frameBound
-    | frameType=RANGE BETWEEN start=frameBound AND end=frameBound
-    | frameType=ROWS BETWEEN start=frameBound AND end=frameBound
-    ;
-
-frameBound
-    : UNBOUNDED boundType=(PRECEDING | FOLLOWING)
-    | boundType=CURRENT ROW
-    | expression boundType=(PRECEDING | FOLLOWING)
-    ;
-
-
-explainOption
-    : LOGICAL | FORMATTED | EXTENDED
-    ;
-
-transactionMode
-    : ISOLATION LEVEL SNAPSHOT            #isolationLevel
-    | READ accessMode=(ONLY | WRITE)      #transactionAccessMode
-    ;
-
-qualifiedName
-    : identifier ('.' identifier)*
-    ;
-
-// Identifier that also allows the use of a number of SQL keywords (mainly for backwards compatibility).
-looseIdentifier
-    : identifier
-    | FROM
-    | TO
-    | TABLE
-    | WITH
-    ;
-
-identifier
-    : IDENTIFIER             #unquotedIdentifier
-    | quotedIdentifier       #quotedIdentifierAlternative
-    | nonReserved            #unquotedIdentifier
-    ;
-
-quotedIdentifier
-    : BACKQUOTED_IDENTIFIER
-    ;
-
-number
-    : DECIMAL_VALUE            #decimalLiteral
-    | SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral
-    | INTEGER_VALUE            #integerLiteral
-    | BIGINT_LITERAL           #bigIntLiteral
-    | SMALLINT_LITERAL         #smallIntLiteral
-    | TINYINT_LITERAL          #tinyIntLiteral
-    | DOUBLE_LITERAL           #doubleLiteral
-    ;
-
-nonReserved
-    : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS
-    | ADD
-    | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | MAP | ARRAY | STRUCT
-    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
-    | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED
-    | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS
-    | GROUPING | CUBE | ROLLUP
-    | EXPLAIN | FORMAT | LOGICAL | FORMATTED
-    | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF
-    | SET
-    | VIEW | REPLACE
-    | IF
-    | NO | DATA
-    | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL
-    | SNAPSHOT | READ | WRITE | ONLY
-    | SORT | CLUSTER | DISTRIBUTE UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION
-    | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST
-    | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT
-    | INPUTDRIVER | OUTPUTDRIVER | DBPROPERTIES | DFS | TRUNCATE | METADATA | REPLICATION | COMPUTE
-    | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER
-    | REVOKE | GRANT | LOCK | UNLOCK | MSCK | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE
-    | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEXES | LOCKS | OPTION
-    ;
-
-SELECT: 'SELECT';
-FROM: 'FROM';
-ADD: 'ADD';
-AS: 'AS';
-ALL: 'ALL';
-DISTINCT: 'DISTINCT';
-WHERE: 'WHERE';
-GROUP: 'GROUP';
-BY: 'BY';
-GROUPING: 'GROUPING';
-SETS: 'SETS';
-CUBE: 'CUBE';
-ROLLUP: 'ROLLUP';
-ORDER: 'ORDER';
-HAVING: 'HAVING';
-LIMIT: 'LIMIT';
-AT: 'AT';
-OR: 'OR';
-AND: 'AND';
-IN: 'IN';
-NOT: 'NOT' | '!';
-NO: 'NO';
-EXISTS: 'EXISTS';
-BETWEEN: 'BETWEEN';
-LIKE: 'LIKE';
-RLIKE: 'RLIKE' | 'REGEXP';
-IS: 'IS';
-NULL: 'NULL';
-TRUE: 'TRUE';
-FALSE: 'FALSE';
-NULLS: 'NULLS';
-ASC: 'ASC';
-DESC: 'DESC';
-FOR: 'FOR';
-INTERVAL: 'INTERVAL';
-CASE: 'CASE';
-WHEN: 'WHEN';
-THEN: 'THEN';
-ELSE: 'ELSE';
-END: 'END';
-JOIN: 'JOIN';
-CROSS: 'CROSS';
-OUTER: 'OUTER';
-INNER: 'INNER';
-LEFT: 'LEFT';
-SEMI: 'SEMI';
-RIGHT: 'RIGHT';
-FULL: 'FULL';
-NATURAL: 'NATURAL';
-ON: 'ON';
-LATERAL: 'LATERAL';
-WINDOW: 'WINDOW';
-OVER: 'OVER';
-PARTITION: 'PARTITION';
-RANGE: 'RANGE';
-ROWS: 'ROWS';
-UNBOUNDED: 'UNBOUNDED';
-PRECEDING: 'PRECEDING';
-FOLLOWING: 'FOLLOWING';
-CURRENT: 'CURRENT';
-ROW: 'ROW';
-WITH: 'WITH';
-VALUES: 'VALUES';
-CREATE: 'CREATE';
-TABLE: 'TABLE';
-VIEW: 'VIEW';
-REPLACE: 'REPLACE';
-INSERT: 'INSERT';
-DELETE: 'DELETE';
-INTO: 'INTO';
-DESCRIBE: 'DESCRIBE';
-EXPLAIN: 'EXPLAIN';
-FORMAT: 'FORMAT';
-LOGICAL: 'LOGICAL';
-CAST: 'CAST';
-SHOW: 'SHOW';
-TABLES: 'TABLES';
-COLUMNS: 'COLUMNS';
-COLUMN: 'COLUMN';
-USE: 'USE';
-PARTITIONS: 'PARTITIONS';
-FUNCTIONS: 'FUNCTIONS';
-DROP: 'DROP';
-UNION: 'UNION';
-EXCEPT: 'EXCEPT';
-INTERSECT: 'INTERSECT';
-TO: 'TO';
-TABLESAMPLE: 'TABLESAMPLE';
-STRATIFY: 'STRATIFY';
-ALTER: 'ALTER';
-RENAME: 'RENAME';
-ARRAY: 'ARRAY';
-MAP: 'MAP';
-STRUCT: 'STRUCT';
-COMMENT: 'COMMENT';
-SET: 'SET';
-DATA: 'DATA';
-START: 'START';
-TRANSACTION: 'TRANSACTION';
-COMMIT: 'COMMIT';
-ROLLBACK: 'ROLLBACK';
-WORK: 'WORK';
-ISOLATION: 'ISOLATION';
-LEVEL: 'LEVEL';
-SNAPSHOT: 'SNAPSHOT';
-READ: 'READ';
-WRITE: 'WRITE';
-ONLY: 'ONLY';
-
-IF: 'IF';
-
-EQ  : '=' | '==';
-NSEQ: '<=>';
-NEQ : '<>';
-NEQJ: '!=';
-LT  : '<';
-LTE : '<=';
-GT  : '>';
-GTE : '>=';
-
-PLUS: '+';
-MINUS: '-';
-ASTERISK: '*';
-SLASH: '/';
-PERCENT: '%';
-DIV: 'DIV';
-TILDE: '~';
-AMPERSAND: '&';
-PIPE: '|';
-HAT: '^';
-
-PERCENTLIT: 'PERCENT';
-BUCKET: 'BUCKET';
-OUT: 'OUT';
-OF: 'OF';
-
-SORT: 'SORT';
-CLUSTER: 'CLUSTER';
-DISTRIBUTE: 'DISTRIBUTE';
-OVERWRITE: 'OVERWRITE';
-TRANSFORM: 'TRANSFORM';
-REDUCE: 'REDUCE';
-USING: 'USING';
-SERDE: 'SERDE';
-SERDEPROPERTIES: 'SERDEPROPERTIES';
-RECORDREADER: 'RECORDREADER';
-RECORDWRITER: 'RECORDWRITER';
-DELIMITED: 'DELIMITED';
-FIELDS: 'FIELDS';
-TERMINATED: 'TERMINATED';
-COLLECTION: 'COLLECTION';
-ITEMS: 'ITEMS';
-KEYS: 'KEYS';
-ESCAPED: 'ESCAPED';
-LINES: 'LINES';
-SEPARATED: 'SEPARATED';
-FUNCTION: 'FUNCTION';
-EXTENDED: 'EXTENDED';
-REFRESH: 'REFRESH';
-CLEAR: 'CLEAR';
-CACHE: 'CACHE';
-UNCACHE: 'UNCACHE';
-LAZY: 'LAZY';
-FORMATTED: 'FORMATTED';
-TEMPORARY: 'TEMPORARY' | 'TEMP';
-OPTIONS: 'OPTIONS';
-UNSET: 'UNSET';
-TBLPROPERTIES: 'TBLPROPERTIES';
-DBPROPERTIES: 'DBPROPERTIES';
-BUCKETS: 'BUCKETS';
-SKEWED: 'SKEWED';
-STORED: 'STORED';
-DIRECTORIES: 'DIRECTORIES';
-LOCATION: 'LOCATION';
-EXCHANGE: 'EXCHANGE';
-ARCHIVE: 'ARCHIVE';
-UNARCHIVE: 'UNARCHIVE';
-FILEFORMAT: 'FILEFORMAT';
-TOUCH: 'TOUCH';
-COMPACT: 'COMPACT';
-CONCATENATE: 'CONCATENATE';
-CHANGE: 'CHANGE';
-FIRST: 'FIRST';
-AFTER: 'AFTER';
-CASCADE: 'CASCADE';
-RESTRICT: 'RESTRICT';
-CLUSTERED: 'CLUSTERED';
-SORTED: 'SORTED';
-PURGE: 'PURGE';
-INPUTFORMAT: 'INPUTFORMAT';
-OUTPUTFORMAT: 'OUTPUTFORMAT';
-INPUTDRIVER: 'INPUTDRIVER';
-OUTPUTDRIVER: 'OUTPUTDRIVER';
-DATABASE: 'DATABASE' | 'SCHEMA';
-DFS: 'DFS';
-TRUNCATE: 'TRUNCATE';
-METADATA: 'METADATA';
-REPLICATION: 'REPLICATION';
-ANALYZE: 'ANALYZE';
-COMPUTE: 'COMPUTE';
-STATISTICS: 'STATISTICS';
-PARTITIONED: 'PARTITIONED';
-EXTERNAL: 'EXTERNAL';
-DEFINED: 'DEFINED';
-REVOKE: 'REVOKE';
-GRANT: 'GRANT';
-LOCK: 'LOCK';
-UNLOCK: 'UNLOCK';
-MSCK: 'MSCK';
-EXPORT: 'EXPORT';
-IMPORT: 'IMPORT';
-LOAD: 'LOAD';
-ROLE: 'ROLE';
-ROLES: 'ROLES';
-COMPACTIONS: 'COMPACTIONS';
-PRINCIPALS: 'PRINCIPALS';
-TRANSACTIONS: 'TRANSACTIONS';
-INDEXES: 'INDEXES';
-LOCKS: 'LOCKS';
-OPTION: 'OPTION';
-
-STRING
-    : '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
-    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
-    ;
-
-BIGINT_LITERAL
-    : DIGIT+ 'L'
-    ;
-
-SMALLINT_LITERAL
-    : DIGIT+ 'S'
-    ;
-
-TINYINT_LITERAL
-    : DIGIT+ 'Y'
-    ;
-
-INTEGER_VALUE
-    : DIGIT+
-    ;
-
-DECIMAL_VALUE
-    : DIGIT+ '.' DIGIT*
-    | '.' DIGIT+
-    ;
-
-SCIENTIFIC_DECIMAL_VALUE
-    : DIGIT+ ('.' DIGIT*)? EXPONENT
-    | '.' DIGIT+ EXPONENT
-    ;
-
-DOUBLE_LITERAL
-    :
-    (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'D'
-    ;
-
-IDENTIFIER
-    : (LETTER | DIGIT | '_')+
-    ;
-
-BACKQUOTED_IDENTIFIER
-    : '`' ( ~'`' | '``' )* '`'
-    ;
-
-fragment EXPONENT
-    : 'E' [+-]? DIGIT+
-    ;
-
-fragment DIGIT
-    : [0-9]
-    ;
-
-fragment LETTER
-    : [A-Z]
-    ;
-
-SIMPLE_COMMENT
-    : '--' ~[\r\n]* '\r'? '\n'? -> channel(HIDDEN)
-    ;
-
-BRACKETED_COMMENT
-    : '/*' .*? '*/' -> channel(HIDDEN)
-    ;
-
-WS
-    : [ \r\n\t]+ -> channel(HIDDEN)
-    ;
-
-// Catch-all for anything we can't recognize.
-// We use this to be able to ignore and recover all the text
-// when splitting statements with DelimiterLexer
-UNRECOGNIZED
-    : .
-    ;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala
deleted file mode 100644
index 28f7b10..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala
+++ /dev/null
@@ -1,99 +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.parser
-
-import org.antlr.runtime.{Token, TokenRewriteStream}
-
-import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode}
-
-case class ASTNode(
-    token: Token,
-    startIndex: Int,
-    stopIndex: Int,
-    children: List[ASTNode],
-    stream: TokenRewriteStream) extends TreeNode[ASTNode] {
-  /** Cache the number of children. */
-  val numChildren: Int = children.size
-
-  /** tuple used in pattern matching. */
-  val pattern: Some[(String, List[ASTNode])] = Some((token.getText, children))
-
-  /** Line in which the ASTNode starts. */
-  lazy val line: Int = {
-    val line = token.getLine
-    if (line == 0) {
-      if (children.nonEmpty) children.head.line
-      else 0
-    } else {
-      line
-    }
-  }
-
-  /** Position of the Character at which ASTNode starts. */
-  lazy val positionInLine: Int = {
-    val line = token.getCharPositionInLine
-    if (line == -1) {
-      if (children.nonEmpty) children.head.positionInLine
-      else 0
-    } else {
-      line
-    }
-  }
-
-  /** Origin of the ASTNode. */
-  override val origin: Origin = Origin(Some(line), Some(positionInLine))
-
-  /** Source text. */
-  lazy val source: String = stream.toOriginalString(startIndex, stopIndex)
-
-  /** Get the source text that remains after this token. */
-  lazy val remainder: String = {
-    stream.fill()
-    stream.toOriginalString(stopIndex + 1, stream.size() - 1).trim()
-  }
-
-  def text: String = token.getText
-
-  def tokenType: Int = token.getType
-
-  /**
-   * Checks if this node is equal to another node.
-   *
-   * Right now this function only checks the name, type, text and children of the node
-   * for equality.
-   */
-  def treeEquals(other: ASTNode): Boolean = {
-    def check(f: ASTNode => Any): Boolean = {
-      val l = f(this)
-      val r = f(other)
-      (l == null && r == null) || l.equals(r)
-    }
-    if (other == null) {
-      false
-    } else if (!check(_.token.getType)
-      || !check(_.token.getText)
-      || !check(_.numChildren)) {
-      false
-    } else {
-      children.zip(other.children).forall {
-        case (l, r) => l treeEquals r
-      }
-    }
-  }
-
-  override def simpleString: String = s"$text $line, $startIndex, $stopIndex, $positionInLine "
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSparkSQLParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSparkSQLParser.scala
deleted file mode 100644
index 7b456a6..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSparkSQLParser.scala
+++ /dev/null
@@ -1,145 +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.parser
-
-import scala.language.implicitConversions
-import scala.util.parsing.combinator.lexical.StdLexical
-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.plans.logical._
-
-private[sql] abstract class AbstractSparkSQLParser
-  extends StandardTokenParsers with PackratParsers with ParserInterface {
-
-  def parsePlan(input: String): LogicalPlan = synchronized {
-    // Initialize the Keywords.
-    initLexical
-    phrase(start)(new lexical.Scanner(input)) match {
-      case Success(plan, _) => plan
-      case failureOrError => sys.error(failureOrError.toString)
-    }
-  }
-  /* One time initialization of lexical.This avoid reinitialization of  lexical in parse method */
-  protected lazy val initLexical: Unit = lexical.initialize(reservedWords)
-
-  protected case class Keyword(str: String) {
-    def normalize: String = lexical.normalizeKeyword(str)
-    def parser: Parser[String] = normalize
-  }
-
-  protected implicit def asParser(k: Keyword): Parser[String] = k.parser
-
-  // By default, use Reflection to find the reserved words defined in the sub class.
-  // NOTICE, Since the Keyword properties defined by sub class, we couldn't call this
-  // method during the parent class instantiation, because the sub class instance
-  // isn't created yet.
-  protected lazy val reservedWords: Seq[String] =
-    this
-      .getClass
-      .getMethods
-      .filter(_.getReturnType == classOf[Keyword])
-      .map(_.invoke(this).asInstanceOf[Keyword].normalize)
-
-  // Set the keywords as empty by default, will change that later.
-  override val lexical = new SqlLexical
-
-  protected def start: Parser[LogicalPlan]
-
-  // Returns the whole input string
-  protected lazy val wholeInput: Parser[String] = new Parser[String] {
-    def apply(in: Input): ParseResult[String] =
-      Success(in.source.toString, in.drop(in.source.length()))
-  }
-
-  // Returns the rest of the input string that are not parsed yet
-  protected lazy val restInput: Parser[String] = new Parser[String] {
-    def apply(in: Input): ParseResult[String] =
-      Success(
-        in.source.subSequence(in.offset, in.source.length()).toString,
-        in.drop(in.source.length()))
-  }
-}
-
-class SqlLexical extends StdLexical {
-  case class DecimalLit(chars: String) extends Token {
-    override def toString: String = chars
-  }
-
-  /* This is a work around to support the lazy setting */
-  def initialize(keywords: Seq[String]): Unit = {
-    reserved.clear()
-    reserved ++= keywords
-  }
-
-  /* Normal the keyword string */
-  def normalizeKeyword(str: String): String = str.toLowerCase
-
-  delimiters += (
-    "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
-    ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~", "<=>"
-  )
-
-  protected override def processIdent(name: String) = {
-    val token = normalizeKeyword(name)
-    if (reserved contains token) Keyword(token) else Identifier(name)
-  }
-
-  override lazy val token: Parser[Token] =
-    ( rep1(digit) ~ scientificNotation ^^ { case i ~ s => DecimalLit(i.mkString + s) }
-    | '.' ~> (rep1(digit) ~ scientificNotation) ^^
-      { case i ~ s => DecimalLit("0." + i.mkString + s) }
-    | rep1(digit) ~ ('.' ~> digit.*) ~ scientificNotation ^^
-      { case i1 ~ i2 ~ s => DecimalLit(i1.mkString + "." + i2.mkString + s) }
-    | digit.* ~ identChar ~ (identChar | digit).* ^^
-      { case first ~ middle ~ rest => processIdent((first ++ (middle :: rest)).mkString) }
-    | rep1(digit) ~ ('.' ~> digit.*).? ^^ {
-        case i ~ None => NumericLit(i.mkString)
-        case i ~ Some(d) => DecimalLit(i.mkString + "." + d.mkString)
-      }
-    | '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^
-      { case chars => StringLit(chars mkString "") }
-    | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^
-      { case chars => StringLit(chars mkString "") }
-    | '`' ~> chrExcept('`', '\n', EofCh).* <~ '`' ^^
-      { case chars => Identifier(chars mkString "") }
-    | EofCh ^^^ EOF
-    | '\'' ~> failure("unclosed string literal")
-    | '"' ~> failure("unclosed string literal")
-    | delim
-    | failure("illegal character")
-    )
-
-  override def identChar: Parser[Elem] = letter | elem('_')
-
-  private lazy val scientificNotation: Parser[String] =
-    (elem('e') | elem('E')) ~> (elem('+') | elem('-')).? ~ rep1(digit) ^^ {
-      case s ~ rest => "e" + s.mkString + rest.mkString
-    }
-
-  override def whitespace: Parser[Any] =
-    ( whitespaceChar
-    | '/' ~ '*' ~ comment
-    | '/' ~ '/' ~ chrExcept(EofCh, '\n').*
-    | '#' ~ chrExcept(EofCh, '\n').*
-    | '-' ~ '-' ~ chrExcept(EofCh, '\n').*
-    | '/' ~ '*' ~ failure("unclosed comment")
-    ).*
-}
-


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[2/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
new file mode 100644
index 0000000..a80d29c
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -0,0 +1,497 @@
+/*
+ * 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, Timestamp}
+
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.CalendarInterval
+
+/**
+ * Test basic expression parsing. If a type of expression is supported it should be tested here.
+ *
+ * Please note that some of the expressions test don't have to be sound expressions, only their
+ * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
+ * CheckAnalysis classes.
+ */
+class ExpressionParserSuite extends PlanTest {
+  import CatalystSqlParser._
+  import org.apache.spark.sql.catalyst.dsl.expressions._
+  import org.apache.spark.sql.catalyst.dsl.plans._
+
+  def assertEqual(sqlCommand: String, e: Expression): Unit = {
+    compareExpressions(parseExpression(sqlCommand), e)
+  }
+
+  def intercept(sqlCommand: String, messages: String*): Unit = {
+    val e = intercept[ParseException](parseExpression(sqlCommand))
+    messages.foreach { message =>
+      assert(e.message.contains(message))
+    }
+  }
+
+  test("star expressions") {
+    // Global Star
+    assertEqual("*", UnresolvedStar(None))
+
+    // Targeted Star
+    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
+  }
+
+  // NamedExpression (Alias/Multialias)
+  test("named expressions") {
+    // No Alias
+    val r0 = 'a
+    assertEqual("a", r0)
+
+    // Single Alias.
+    val r1 = 'a as "b"
+    assertEqual("a as b", r1)
+    assertEqual("a b", r1)
+
+    // Multi-Alias
+    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
+    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
+
+    // Numeric literals without a space between the literal qualifier and the alias, should not be
+    // interpreted as such. An unresolved reference should be returned instead.
+    // TODO add the JIRA-ticket number.
+    assertEqual("1SL", Symbol("1SL"))
+
+    // Aliased star is allowed.
+    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
+  }
+
+  test("binary logical expressions") {
+    // And
+    assertEqual("a and b", 'a && 'b)
+
+    // Or
+    assertEqual("a or b", 'a || 'b)
+
+    // Combination And/Or check precedence
+    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
+    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
+
+    // Multiple AND/OR get converted into a balanced tree
+    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
+    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
+  }
+
+  test("long binary logical expressions") {
+    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
+      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
+      val e = parseExpression(sql)
+      assert(e.collect { case _: EqualTo => true }.size === 1000)
+      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
+    }
+    testVeryBinaryExpression(" AND ", classOf[And])
+    testVeryBinaryExpression(" OR ", classOf[Or])
+  }
+
+  test("not expressions") {
+    assertEqual("not a", !'a)
+    assertEqual("!a", !'a)
+    assertEqual("not true > true", Not(GreaterThan(true, true)))
+  }
+
+  test("exists expression") {
+    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
+  }
+
+  test("comparison expressions") {
+    assertEqual("a = b", 'a === 'b)
+    assertEqual("a == b", 'a === 'b)
+    assertEqual("a <=> b", 'a <=> 'b)
+    assertEqual("a <> b", 'a =!= 'b)
+    assertEqual("a != b", 'a =!= 'b)
+    assertEqual("a < b", 'a < 'b)
+    assertEqual("a <= b", 'a <= 'b)
+    assertEqual("a > b", 'a > 'b)
+    assertEqual("a >= b", 'a >= 'b)
+  }
+
+  test("between expressions") {
+    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
+    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
+  }
+
+  test("in expressions") {
+    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
+    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
+  }
+
+  test("in sub-query") {
+    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
+  }
+
+  test("like expressions") {
+    assertEqual("a like 'pattern%'", 'a like "pattern%")
+    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
+    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
+    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
+    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
+    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
+  }
+
+  test("is null expressions") {
+    assertEqual("a is null", 'a.isNull)
+    assertEqual("a is not null", 'a.isNotNull)
+    assertEqual("a = b is null", ('a === 'b).isNull)
+    assertEqual("a = b is not null", ('a === 'b).isNotNull)
+  }
+
+  test("binary arithmetic expressions") {
+    // Simple operations
+    assertEqual("a * b", 'a * 'b)
+    assertEqual("a / b", 'a / 'b)
+    assertEqual("a DIV b", ('a / 'b).cast(LongType))
+    assertEqual("a % b", 'a % 'b)
+    assertEqual("a + b", 'a + 'b)
+    assertEqual("a - b", 'a - 'b)
+    assertEqual("a & b", 'a & 'b)
+    assertEqual("a ^ b", 'a ^ 'b)
+    assertEqual("a | b", 'a | 'b)
+
+    // Check precedences
+    assertEqual(
+      "a * t | b ^ c & d - e + f % g DIV h / i * k",
+      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
+  }
+
+  test("unary arithmetic expressions") {
+    assertEqual("+a", 'a)
+    assertEqual("-a", -'a)
+    assertEqual("~a", ~'a)
+    assertEqual("-+~~a", -(~(~'a)))
+  }
+
+  test("cast expressions") {
+    // Note that DataType parsing is tested elsewhere.
+    assertEqual("cast(a as int)", 'a.cast(IntegerType))
+    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
+    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
+    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
+  }
+
+  test("function expressions") {
+    assertEqual("foo()", 'foo.function())
+    assertEqual("foo.bar()", Symbol("foo.bar").function())
+    assertEqual("foo(*)", 'foo.function(star()))
+    assertEqual("count(*)", 'count.function(1))
+    assertEqual("foo(a, b)", 'foo.function('a, 'b))
+    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
+    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
+    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
+    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
+  }
+
+  test("window function expressions") {
+    val func = 'foo.function(star())
+    def windowed(
+        partitioning: Seq[Expression] = Seq.empty,
+        ordering: Seq[SortOrder] = Seq.empty,
+        frame: WindowFrame = UnspecifiedFrame): Expression = {
+      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
+    }
+
+    // Basic window testing.
+    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
+    assertEqual("foo(*) over ()", windowed())
+    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
+    assertEqual("foo(*) over (distribute by a, b)", windowed(Seq('a, 'b)))
+    assertEqual("foo(*) over (cluster by a, b)", windowed(Seq('a, 'b)))
+    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
+    assertEqual("foo(*) over (sort by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
+    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
+    assertEqual("foo(*) over (distribute by a, b sort by c)", windowed(Seq('a, 'b), Seq('c.asc)))
+
+    // Test use of expressions in window functions.
+    assertEqual(
+      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
+      WindowExpression('sum.function('product + 1),
+        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
+    assertEqual(
+      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
+      WindowExpression('sum.function('product + 1),
+        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
+
+    // Range/Row
+    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
+    val boundaries = Seq(
+      ("10 preceding", ValuePreceding(10), CurrentRow),
+      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
+      ("unbounded preceding", UnboundedPreceding, CurrentRow),
+      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
+      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
+      ("between unbounded preceding and unbounded following",
+        UnboundedPreceding, UnboundedFollowing),
+      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
+      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
+      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
+    )
+    frameTypes.foreach {
+      case (frameTypeSql, frameType) =>
+        boundaries.foreach {
+          case (boundarySql, begin, end) =>
+            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
+            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
+            assertEqual(query, expr)
+        }
+    }
+
+    // We cannot use non integer constants.
+    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
+      "Frame bound value must be a constant integer.")
+
+    // We cannot use an arbitrary expression.
+    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
+      "Frame bound value must be a constant integer.")
+  }
+
+  test("row constructor") {
+    // Note that '(a)' will be interpreted as a nested expression.
+    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
+    assertEqual("(a, b, c)", CreateStruct(Seq('a, 'b, 'c)))
+  }
+
+  test("scalar sub-query") {
+    assertEqual(
+      "(select max(val) from tbl) > current",
+      ScalarSubquery(table("tbl").select('max.function('val))) > 'current)
+    assertEqual(
+      "a = (select b from s)",
+      'a === ScalarSubquery(table("s").select('b)))
+  }
+
+  test("case when") {
+    assertEqual("case a when 1 then b when 2 then c else d end",
+      CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd)))
+    assertEqual("case when a = 1 then b when a = 2 then c else d end",
+      CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd))
+  }
+
+  test("dereference") {
+    assertEqual("a.b", UnresolvedAttribute("a.b"))
+    assertEqual("`select`.b", UnresolvedAttribute("select.b"))
+    assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis.
+    assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b"))
+  }
+
+  test("reference") {
+    // Regular
+    assertEqual("a", 'a)
+
+    // Starting with a digit.
+    assertEqual("1a", Symbol("1a"))
+
+    // Quoted using a keyword.
+    assertEqual("`select`", 'select)
+
+    // Unquoted using an unreserved keyword.
+    assertEqual("columns", 'columns)
+  }
+
+  test("subscript") {
+    assertEqual("a[b]", 'a.getItem('b))
+    assertEqual("a[1 + 1]", 'a.getItem(Literal(1) + 1))
+    assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem('b))
+  }
+
+  test("parenthesis") {
+    assertEqual("(a)", 'a)
+    assertEqual("r * (a + b)", 'r * ('a + 'b))
+  }
+
+  test("type constructors") {
+    // Dates.
+    assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11")))
+    intercept[IllegalArgumentException] {
+      parseExpression("DAtE 'mar 11 2016'")
+    }
+
+    // Timestamps.
+    assertEqual("tImEstAmp '2016-03-11 20:54:00.000'",
+      Literal(Timestamp.valueOf("2016-03-11 20:54:00.000")))
+    intercept[IllegalArgumentException] {
+      parseExpression("timestamP '2016-33-11 20:54:00.000'")
+    }
+
+    // Unsupported datatype.
+    intercept("GEO '(10,-6)'", "Literals of type 'GEO' are currently not supported.")
+  }
+
+  test("literals") {
+    // NULL
+    assertEqual("null", Literal(null))
+
+    // Boolean
+    assertEqual("trUe", Literal(true))
+    assertEqual("False", Literal(false))
+
+    // Integral should have the narrowest possible type
+    assertEqual("787324", Literal(787324))
+    assertEqual("7873247234798249234", Literal(7873247234798249234L))
+    assertEqual("78732472347982492793712334",
+      Literal(BigDecimal("78732472347982492793712334").underlying()))
+
+    // Decimal
+    assertEqual("7873247234798249279371.2334",
+      Literal(BigDecimal("7873247234798249279371.2334").underlying()))
+
+    // Scientific Decimal
+    assertEqual("9.0e1", 90d)
+    assertEqual(".9e+2", 90d)
+    assertEqual("0.9e+2", 90d)
+    assertEqual("900e-1", 90d)
+    assertEqual("900.0E-1", 90d)
+    assertEqual("9.e+1", 90d)
+    intercept(".e3")
+
+    // Tiny Int Literal
+    assertEqual("10Y", Literal(10.toByte))
+    intercept("-1000Y")
+
+    // Small Int Literal
+    assertEqual("10S", Literal(10.toShort))
+    intercept("40000S")
+
+    // Long Int Literal
+    assertEqual("10L", Literal(10L))
+    intercept("78732472347982492793712334L")
+
+    // Double Literal
+    assertEqual("10.0D", Literal(10.0D))
+    // TODO we need to figure out if we should throw an exception here!
+    assertEqual("1E309", Literal(Double.PositiveInfinity))
+  }
+
+  test("strings") {
+    // Single Strings.
+    assertEqual("\"hello\"", "hello")
+    assertEqual("'hello'", "hello")
+
+    // Multi-Strings.
+    assertEqual("\"hello\" 'world'", "helloworld")
+    assertEqual("'hello' \" \" 'world'", "hello world")
+
+    // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
+    // regular '%'; to get the correct result you need to add another escaped '\'.
+    // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
+    assertEqual("'pattern%'", "pattern%")
+    assertEqual("'no-pattern\\%'", "no-pattern\\%")
+    assertEqual("'pattern\\\\%'", "pattern\\%")
+    assertEqual("'pattern\\\\\\%'", "pattern\\\\%")
+
+    // Escaped characters.
+    // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
+    assertEqual("'\\0'", "\u0000") // ASCII NUL (X'00')
+    assertEqual("'\\''", "\'")     // Single quote
+    assertEqual("'\\\"'", "\"")    // Double quote
+    assertEqual("'\\b'", "\b")     // Backspace
+    assertEqual("'\\n'", "\n")     // Newline
+    assertEqual("'\\r'", "\r")     // Carriage return
+    assertEqual("'\\t'", "\t")     // Tab character
+    assertEqual("'\\Z'", "\u001A") // ASCII 26 - CTRL + Z (EOF on windows)
+
+    // Octals
+    assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!")
+
+    // Unicode
+    assertEqual("'\\u0087\\u0111\\u0114\\u0108\\u0100\\u0032\\u0058\\u0041'", "World :)")
+  }
+
+  test("intervals") {
+    def intervalLiteral(u: String, s: String): Literal = {
+      Literal(CalendarInterval.fromSingleUnitString(u, s))
+    }
+
+    // Empty interval statement
+    intercept("interval", "at least one time unit should be given for interval literal")
+
+    // Single Intervals.
+    val units = Seq(
+      "year",
+      "month",
+      "week",
+      "day",
+      "hour",
+      "minute",
+      "second",
+      "millisecond",
+      "microsecond")
+    val forms = Seq("", "s")
+    val values = Seq("0", "10", "-7", "21")
+    units.foreach { unit =>
+      forms.foreach { form =>
+         values.foreach { value =>
+           val expected = intervalLiteral(unit, value)
+           assertEqual(s"interval $value $unit$form", expected)
+           assertEqual(s"interval '$value' $unit$form", expected)
+         }
+      }
+    }
+
+    // Hive nanosecond notation.
+    assertEqual("interval 13.123456789 seconds", intervalLiteral("second", "13.123456789"))
+    assertEqual("interval -13.123456789 second", intervalLiteral("second", "-13.123456789"))
+
+    // Non Existing unit
+    intercept("interval 10 nanoseconds", "No interval can be constructed")
+
+    // Year-Month intervals.
+    val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0")
+    yearMonthValues.foreach { value =>
+      val result = Literal(CalendarInterval.fromYearMonthString(value))
+      assertEqual(s"interval '$value' year to month", result)
+    }
+
+    // Day-Time intervals.
+    val datTimeValues = Seq(
+      "99 11:22:33.123456789",
+      "-99 11:22:33.123456789",
+      "10 9:8:7.123456789",
+      "1 0:0:0",
+      "-1 0:0:0",
+      "1 0:0:1")
+    datTimeValues.foreach { value =>
+      val result = Literal(CalendarInterval.fromDayTimeString(value))
+      assertEqual(s"interval '$value' day to second", result)
+    }
+
+    // Unknown FROM TO intervals
+    intercept("interval 10 month to second", "Intervals FROM month TO second are not supported.")
+
+    // Composed intervals.
+    assertEqual(
+      "interval 3 months 22 seconds 1 millisecond",
+      Literal(new CalendarInterval(3, 22001000L)))
+    assertEqual(
+      "interval 3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second",
+      Literal(new CalendarInterval(14,
+        22 * CalendarInterval.MICROS_PER_DAY + 2 * CalendarInterval.MICROS_PER_SECOND)))
+  }
+
+  test("composed expressions") {
+    assertEqual("1 + r.r As q", (Literal(1) + UnresolvedAttribute("r.r")).as("q"))
+    assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar)))
+    intercept("1 - f('o', o(bar)) hello * world", "mismatched input '*'")
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
new file mode 100644
index 0000000..23f05ce
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
@@ -0,0 +1,429 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types.{BooleanType, IntegerType}
+
+class PlanParserSuite extends PlanTest {
+  import CatalystSqlParser._
+  import org.apache.spark.sql.catalyst.dsl.expressions._
+  import org.apache.spark.sql.catalyst.dsl.plans._
+
+  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
+    comparePlans(parsePlan(sqlCommand), plan)
+  }
+
+  def intercept(sqlCommand: String, messages: String*): Unit = {
+    val e = intercept[ParseException](parsePlan(sqlCommand))
+    messages.foreach { message =>
+      assert(e.message.contains(message))
+    }
+  }
+
+  test("case insensitive") {
+    val plan = table("a").select(star())
+    assertEqual("sELEct * FroM a", plan)
+    assertEqual("select * fRoM a", plan)
+    assertEqual("SELECT * FROM a", plan)
+  }
+
+  test("show functions") {
+    assertEqual("show functions", ShowFunctions(None, None))
+    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
+    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
+    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
+    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
+  }
+
+  test("describe function") {
+    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
+    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
+    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
+    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
+  }
+
+  test("set operations") {
+    val a = table("a").select(star())
+    val b = table("b").select(star())
+
+    assertEqual("select * from a union select * from b", Distinct(a.union(b)))
+    assertEqual("select * from a union distinct select * from b", Distinct(a.union(b)))
+    assertEqual("select * from a union all select * from b", a.union(b))
+    assertEqual("select * from a except select * from b", a.except(b))
+    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
+    assertEqual("select * from a except distinct select * from b", a.except(b))
+    assertEqual("select * from a intersect select * from b", a.intersect(b))
+    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
+    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
+  }
+
+  test("common table expressions") {
+    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
+      val ctes = namedPlans.map {
+        case (name, cte) =>
+          name -> SubqueryAlias(name, cte)
+      }.toMap
+      With(plan, ctes)
+    }
+    assertEqual(
+      "with cte1 as (select * from a) select * from cte1",
+      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
+    assertEqual(
+      "with cte1 (select 1) select * from cte1",
+      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
+    assertEqual(
+      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
+      cte(table("cte2").select(star()),
+        "cte1" -> OneRowRelation.select(1),
+        "cte2" -> table("cte1").select(star())))
+    intercept(
+      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
+      "Name 'cte1' is used for multiple common table expressions")
+  }
+
+  test("simple select query") {
+    assertEqual("select 1", OneRowRelation.select(1))
+    assertEqual("select a, b", OneRowRelation.select('a, 'b))
+    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
+    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
+    assertEqual(
+      "select a, b from db.c having x < 1",
+      table("db", "c").select('a, 'b).where(('x < 1).cast(BooleanType)))
+    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
+    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
+  }
+
+  test("reverse select query") {
+    assertEqual("from a", table("a"))
+    assertEqual("from a select b, c", table("a").select('b, 'c))
+    assertEqual(
+      "from db.a select b, c where d < 1", table("db", "a").where('d < 1).select('b, 'c))
+    assertEqual("from a select distinct b, c", Distinct(table("a").select('b, 'c)))
+    assertEqual(
+      "from (from a union all from b) c select *",
+      table("a").union(table("b")).as("c").select(star()))
+  }
+
+  test("transform query spec") {
+    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
+    assertEqual("select transform(a, b) using 'func' from e where f < 10",
+      p.copy(child = p.child.where('f < 10), output = Seq('key.string, 'value.string)))
+    assertEqual("map a, b using 'func' as c, d from e",
+      p.copy(output = Seq('c.string, 'd.string)))
+    assertEqual("reduce a, b using 'func' as (c: int, d decimal(10, 0)) from e",
+      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
+  }
+
+  test("multi select query") {
+    assertEqual(
+      "from a select * select * where s < 10",
+      table("a").select(star()).union(table("a").where('s < 10).select(star())))
+    intercept(
+      "from a select * select * from x where a.s < 10",
+      "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements")
+    assertEqual(
+      "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
+      table("a").select(star()).insertInto("tbl1").union(
+        table("a").where('s < 10).select(star()).insertInto("tbl2")))
+  }
+
+  test("query organization") {
+    // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
+    val baseSql = "select * from t"
+    val basePlan = table("t").select(star())
+
+    val ws = Map("w1" -> WindowSpecDefinition(Seq.empty, Seq.empty, UnspecifiedFrame))
+    val limitWindowClauses = Seq(
+      ("", (p: LogicalPlan) => p),
+      (" limit 10", (p: LogicalPlan) => p.limit(10)),
+      (" window w1 as ()", (p: LogicalPlan) => WithWindowDefinition(ws, p)),
+      (" window w1 as () limit 10", (p: LogicalPlan) => WithWindowDefinition(ws, p).limit(10))
+    )
+
+    val orderSortDistrClusterClauses = Seq(
+      ("", basePlan),
+      (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)),
+      (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)),
+      (" distribute by a, b", basePlan.distribute('a, 'b)),
+      (" distribute by a sort by b", basePlan.distribute('a).sortBy('b.asc)),
+      (" cluster by a, b", basePlan.distribute('a, 'b).sortBy('a.asc, 'b.asc))
+    )
+
+    orderSortDistrClusterClauses.foreach {
+      case (s1, p1) =>
+        limitWindowClauses.foreach {
+          case (s2, pf2) =>
+            assertEqual(baseSql + s1 + s2, pf2(p1))
+        }
+    }
+
+    val msg = "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported"
+    intercept(s"$baseSql order by a sort by a", msg)
+    intercept(s"$baseSql cluster by a distribute by a", msg)
+    intercept(s"$baseSql order by a cluster by a", msg)
+    intercept(s"$baseSql order by a distribute by a", msg)
+  }
+
+  test("insert into") {
+    val sql = "select * from t"
+    val plan = table("t").select(star())
+    def insert(
+        partition: Map[String, Option[String]],
+        overwrite: Boolean = false,
+        ifNotExists: Boolean = false): LogicalPlan =
+      InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
+
+    // Single inserts
+    assertEqual(s"insert overwrite table s $sql",
+      insert(Map.empty, overwrite = true))
+    assertEqual(s"insert overwrite table s if not exists $sql",
+      insert(Map.empty, overwrite = true, ifNotExists = true))
+    assertEqual(s"insert into s $sql",
+      insert(Map.empty))
+    assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql",
+      insert(Map("c" -> Option("d"), "e" -> Option("1"))))
+    assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql",
+      insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true))
+
+    // Multi insert
+    val plan2 = table("t").where('x > 5).select(star())
+    assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5",
+      InsertIntoTable(
+        table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).union(
+        InsertIntoTable(
+          table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false)))
+  }
+
+  test("aggregation") {
+    val sql = "select a, b, sum(c) as c from d group by a, b"
+
+    // Normal
+    assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c")))
+
+    // Cube
+    assertEqual(s"$sql with cube",
+      table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
+
+    // Rollup
+    assertEqual(s"$sql with rollup",
+      table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
+
+    // Grouping Sets
+    assertEqual(s"$sql grouping sets((a, b), (a), ())",
+      GroupingSets(Seq(0, 1, 3), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c"))))
+    intercept(s"$sql grouping sets((a, b), (c), ())",
+      "c doesn't show up in the GROUP BY list")
+  }
+
+  test("limit") {
+    val sql = "select * from t"
+    val plan = table("t").select(star())
+    assertEqual(s"$sql limit 10", plan.limit(10))
+    assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
+  }
+
+  test("window spec") {
+    // Note that WindowSpecs are testing in the ExpressionParserSuite
+    val sql = "select * from t"
+    val plan = table("t").select(star())
+    val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc),
+      SpecifiedWindowFrame(RowFrame, ValuePreceding(1), ValueFollowing(1)))
+
+    // Test window resolution.
+    val ws1 = Map("w1" -> spec, "w2" -> spec, "w3" -> spec)
+    assertEqual(
+      s"""$sql
+         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
+         |       w2 as w1,
+         |       w3 as w1""".stripMargin,
+      WithWindowDefinition(ws1, plan))
+
+    // Fail with no reference.
+    intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")
+
+    // Fail when resolved reference is not a window spec.
+    intercept(
+      s"""$sql
+         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
+         |       w2 as w1,
+         |       w3 as w2""".stripMargin,
+      "Window reference 'w2' is not a window specification"
+    )
+  }
+
+  test("lateral view") {
+    // Single lateral view
+    assertEqual(
+      "select * from t lateral view explode(x) expl as x",
+      table("t")
+        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
+        .select(star()))
+
+    // Multiple lateral views
+    assertEqual(
+      """select *
+        |from t
+        |lateral view explode(x) expl
+        |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin,
+      table("t")
+        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq.empty)
+        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z"))
+        .select(star()))
+
+    // Multi-Insert lateral views.
+    val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
+    assertEqual(
+      """from t1
+        |lateral view explode(x) expl as x
+        |insert into t2
+        |select *
+        |lateral view json_tuple(x, y) jtup q, z
+        |insert into t3
+        |select *
+        |where s < 10
+      """.stripMargin,
+      Union(from
+        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z"))
+        .select(star())
+        .insertInto("t2"),
+        from.where('s < 10).select(star()).insertInto("t3")))
+
+    // Unsupported generator.
+    intercept(
+      "select * from t lateral view posexplode(x) posexpl as x, y",
+      "Generator function 'posexplode' is not supported")
+  }
+
+  test("joins") {
+    // Test single joins.
+    val testUnconditionalJoin = (sql: String, jt: JoinType) => {
+      assertEqual(
+        s"select * from t as tt $sql u",
+        table("t").as("tt").join(table("u"), jt, None).select(star()))
+    }
+    val testConditionalJoin = (sql: String, jt: JoinType) => {
+      assertEqual(
+        s"select * from t $sql u as uu on a = b",
+        table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star()))
+    }
+    val testNaturalJoin = (sql: String, jt: JoinType) => {
+      assertEqual(
+        s"select * from t tt natural $sql u as uu",
+        table("t").as("tt").join(table("u").as("uu"), NaturalJoin(jt), None).select(star()))
+    }
+    val testUsingJoin = (sql: String, jt: JoinType) => {
+      assertEqual(
+        s"select * from t $sql u using(a, b)",
+        table("t").join(table("u"), UsingJoin(jt, Seq('a.attr, 'b.attr)), None).select(star()))
+    }
+    val testAll = Seq(testUnconditionalJoin, testConditionalJoin, testNaturalJoin, testUsingJoin)
+
+    def test(sql: String, jt: JoinType, tests: Seq[(String, JoinType) => Unit]): Unit = {
+      tests.foreach(_(sql, jt))
+    }
+    test("cross join", Inner, Seq(testUnconditionalJoin))
+    test(",", Inner, Seq(testUnconditionalJoin))
+    test("join", Inner, testAll)
+    test("inner join", Inner, testAll)
+    test("left join", LeftOuter, testAll)
+    test("left outer join", LeftOuter, testAll)
+    test("right join", RightOuter, testAll)
+    test("right outer join", RightOuter, testAll)
+    test("full join", FullOuter, testAll)
+    test("full outer join", FullOuter, testAll)
+
+    // Test multiple consecutive joins
+    assertEqual(
+      "select * from a join b join c right join d",
+      table("a").join(table("b")).join(table("c")).join(table("d"), RightOuter).select(star()))
+  }
+
+  test("sampled relations") {
+    val sql = "select * from t"
+    assertEqual(s"$sql tablesample(100 rows)",
+      table("t").limit(100).select(star()))
+    assertEqual(s"$sql tablesample(43 percent) as x",
+      Sample(0, .43d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
+    assertEqual(s"$sql tablesample(bucket 4 out of 10) as x",
+      Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
+    intercept(s"$sql tablesample(bucket 4 out of 10 on x) as x",
+      "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported")
+    intercept(s"$sql tablesample(bucket 11 out of 10) as x",
+      s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]")
+  }
+
+  test("sub-query") {
+    val plan = table("t0").select('id)
+    assertEqual("select id from (t0)", plan)
+    assertEqual("select id from ((((((t0))))))", plan)
+    assertEqual(
+      "(select * from t1) union distinct (select * from t2)",
+      Distinct(table("t1").select(star()).union(table("t2").select(star()))))
+    assertEqual(
+      "select * from ((select * from t1) union (select * from t2)) t",
+      Distinct(
+        table("t1").select(star()).union(table("t2").select(star()))).as("t").select(star()))
+    assertEqual(
+      """select  id
+        |from (((select id from t0)
+        |       union all
+        |       (select  id from t0))
+        |      union all
+        |      (select id from t0)) as u_1
+      """.stripMargin,
+      plan.union(plan).union(plan).as("u_1").select('id))
+  }
+
+  test("scalar sub-query") {
+    assertEqual(
+      "select (select max(b) from s) ss from t",
+      table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss")))
+    assertEqual(
+      "select * from t where a = (select b from s)",
+      table("t").where('a === ScalarSubquery(table("s").select('b))).select(star()))
+    assertEqual(
+      "select g from t group by g having a > (select b from s)",
+      table("t")
+        .groupBy('g)('g)
+        .where(('a > ScalarSubquery(table("s").select('b))).cast(BooleanType)))
+  }
+
+  test("table reference") {
+    assertEqual("table t", table("t"))
+    assertEqual("table d.t", table("d", "t"))
+  }
+
+  test("inline table") {
+    assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
+      Seq('col1.int),
+      Seq(1, 2, 3, 4).map(x => Row(x))))
+    assertEqual(
+      "values (1, 'a'), (2, 'b'), (3, 'c') as tbl(a, b)",
+      LocalRelation.fromExternalRows(
+        Seq('a.int, 'b.string),
+        Seq((1, "a"), (2, "b"), (3, "c")).map(x => Row(x._1, x._2))).as("tbl"))
+    intercept("values (a, 'a'), (b, 'b')",
+      "All expressions in an inline table must be constants.")
+    intercept("values (1, 'a'), (2, 'b') as tbl(a, b, c)",
+      "Number of aliases must match the number of fields in an inline table.")
+    intercept[ArrayIndexOutOfBoundsException](parsePlan("values (1, 'a'), (2, 'b', 5Y)"))
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
new file mode 100644
index 0000000..297b193
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+class TableIdentifierParserSuite extends SparkFunSuite {
+  import CatalystSqlParser._
+
+  test("table identifier") {
+    // Regular names.
+    assert(TableIdentifier("q") === parseTableIdentifier("q"))
+    assert(TableIdentifier("q", Option("d")) === parseTableIdentifier("d.q"))
+
+    // Illegal names.
+    intercept[ParseException](parseTableIdentifier(""))
+    intercept[ParseException](parseTableIdentifier("d.q.g"))
+
+    // SQL Keywords.
+    val keywords = Seq("select", "from", "where", "left", "right")
+    keywords.foreach { keyword =>
+      intercept[ParseException](parseTableIdentifier(keyword))
+      assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`"))
+      assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`"))
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ErrorParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ErrorParserSuite.scala
deleted file mode 100644
index 1963fc3..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ErrorParserSuite.scala
+++ /dev/null
@@ -1,67 +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.parser.ng
-
-import org.apache.spark.SparkFunSuite
-
-/**
- * Test various parser errors.
- */
-class ErrorParserSuite extends SparkFunSuite {
-  def intercept(sql: String, line: Int, startPosition: Int, messages: String*): Unit = {
-    val e = intercept[ParseException](CatalystSqlParser.parsePlan(sql))
-
-    // Check position.
-    assert(e.line.isDefined)
-    assert(e.line.get === line)
-    assert(e.startPosition.isDefined)
-    assert(e.startPosition.get === startPosition)
-
-    // Check messages.
-    val error = e.getMessage
-    messages.foreach { message =>
-      assert(error.contains(message))
-    }
-  }
-
-  test("no viable input") {
-    intercept("select from tbl", 1, 7, "no viable alternative at input", "-------^^^")
-    intercept("select\nfrom tbl", 2, 0, "no viable alternative at input", "^^^")
-    intercept("select ((r + 1) ", 1, 16, "no viable alternative at input", "----------------^^^")
-  }
-
-  test("extraneous input") {
-    intercept("select 1 1", 1, 9, "extraneous input '1' expecting", "---------^^^")
-    intercept("select *\nfrom r as q t", 2, 12, "extraneous input", "------------^^^")
-  }
-
-  test("mismatched input") {
-    intercept("select * from r order by q from t", 1, 27,
-      "mismatched input",
-      "---------------------------^^^")
-    intercept("select *\nfrom r\norder by q\nfrom t", 4, 0, "mismatched input", "^^^")
-  }
-
-  test("semantic errors") {
-    intercept("select *\nfrom r\norder by q\ncluster by q", 3, 0,
-      "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported",
-      "^^^")
-    intercept("select * from r where a in (select * from t)", 1, 24,
-      "IN with a Sub-query is currently not supported",
-      "------------------------^^^")
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala
deleted file mode 100644
index 32311a5..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala
+++ /dev/null
@@ -1,497 +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.parser.ng
-
-import java.sql.{Date, Timestamp}
-
-import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.PlanTest
-import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
-
-/**
- * Test basic expression parsing. If a type of expression is supported it should be tested here.
- *
- * Please note that some of the expressions test don't have to be sound expressions, only their
- * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
- * CheckAnalysis classes.
- */
-class ExpressionParserSuite extends PlanTest {
-  import CatalystSqlParser._
-  import org.apache.spark.sql.catalyst.dsl.expressions._
-  import org.apache.spark.sql.catalyst.dsl.plans._
-
-  def assertEqual(sqlCommand: String, e: Expression): Unit = {
-    compareExpressions(parseExpression(sqlCommand), e)
-  }
-
-  def intercept(sqlCommand: String, messages: String*): Unit = {
-    val e = intercept[ParseException](parseExpression(sqlCommand))
-    messages.foreach { message =>
-      assert(e.message.contains(message))
-    }
-  }
-
-  test("star expressions") {
-    // Global Star
-    assertEqual("*", UnresolvedStar(None))
-
-    // Targeted Star
-    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
-  }
-
-  // NamedExpression (Alias/Multialias)
-  test("named expressions") {
-    // No Alias
-    val r0 = 'a
-    assertEqual("a", r0)
-
-    // Single Alias.
-    val r1 = 'a as "b"
-    assertEqual("a as b", r1)
-    assertEqual("a b", r1)
-
-    // Multi-Alias
-    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
-    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
-
-    // Numeric literals without a space between the literal qualifier and the alias, should not be
-    // interpreted as such. An unresolved reference should be returned instead.
-    // TODO add the JIRA-ticket number.
-    assertEqual("1SL", Symbol("1SL"))
-
-    // Aliased star is allowed.
-    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
-  }
-
-  test("binary logical expressions") {
-    // And
-    assertEqual("a and b", 'a && 'b)
-
-    // Or
-    assertEqual("a or b", 'a || 'b)
-
-    // Combination And/Or check precedence
-    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
-    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
-
-    // Multiple AND/OR get converted into a balanced tree
-    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
-    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
-  }
-
-  test("long binary logical expressions") {
-    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
-      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
-      val e = parseExpression(sql)
-      assert(e.collect { case _: EqualTo => true }.size === 1000)
-      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
-    }
-    testVeryBinaryExpression(" AND ", classOf[And])
-    testVeryBinaryExpression(" OR ", classOf[Or])
-  }
-
-  test("not expressions") {
-    assertEqual("not a", !'a)
-    assertEqual("!a", !'a)
-    assertEqual("not true > true", Not(GreaterThan(true, true)))
-  }
-
-  test("exists expression") {
-    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
-  }
-
-  test("comparison expressions") {
-    assertEqual("a = b", 'a === 'b)
-    assertEqual("a == b", 'a === 'b)
-    assertEqual("a <=> b", 'a <=> 'b)
-    assertEqual("a <> b", 'a =!= 'b)
-    assertEqual("a != b", 'a =!= 'b)
-    assertEqual("a < b", 'a < 'b)
-    assertEqual("a <= b", 'a <= 'b)
-    assertEqual("a > b", 'a > 'b)
-    assertEqual("a >= b", 'a >= 'b)
-  }
-
-  test("between expressions") {
-    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
-    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
-  }
-
-  test("in expressions") {
-    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
-    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
-  }
-
-  test("in sub-query") {
-    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
-  }
-
-  test("like expressions") {
-    assertEqual("a like 'pattern%'", 'a like "pattern%")
-    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
-    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
-    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
-    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
-    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
-  }
-
-  test("is null expressions") {
-    assertEqual("a is null", 'a.isNull)
-    assertEqual("a is not null", 'a.isNotNull)
-    assertEqual("a = b is null", ('a === 'b).isNull)
-    assertEqual("a = b is not null", ('a === 'b).isNotNull)
-  }
-
-  test("binary arithmetic expressions") {
-    // Simple operations
-    assertEqual("a * b", 'a * 'b)
-    assertEqual("a / b", 'a / 'b)
-    assertEqual("a DIV b", ('a / 'b).cast(LongType))
-    assertEqual("a % b", 'a % 'b)
-    assertEqual("a + b", 'a + 'b)
-    assertEqual("a - b", 'a - 'b)
-    assertEqual("a & b", 'a & 'b)
-    assertEqual("a ^ b", 'a ^ 'b)
-    assertEqual("a | b", 'a | 'b)
-
-    // Check precedences
-    assertEqual(
-      "a * t | b ^ c & d - e + f % g DIV h / i * k",
-      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
-  }
-
-  test("unary arithmetic expressions") {
-    assertEqual("+a", 'a)
-    assertEqual("-a", -'a)
-    assertEqual("~a", ~'a)
-    assertEqual("-+~~a", -(~(~'a)))
-  }
-
-  test("cast expressions") {
-    // Note that DataType parsing is tested elsewhere.
-    assertEqual("cast(a as int)", 'a.cast(IntegerType))
-    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
-    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
-    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
-  }
-
-  test("function expressions") {
-    assertEqual("foo()", 'foo.function())
-    assertEqual("foo.bar()", Symbol("foo.bar").function())
-    assertEqual("foo(*)", 'foo.function(star()))
-    assertEqual("count(*)", 'count.function(1))
-    assertEqual("foo(a, b)", 'foo.function('a, 'b))
-    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
-    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
-    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
-    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
-  }
-
-  test("window function expressions") {
-    val func = 'foo.function(star())
-    def windowed(
-        partitioning: Seq[Expression] = Seq.empty,
-        ordering: Seq[SortOrder] = Seq.empty,
-        frame: WindowFrame = UnspecifiedFrame): Expression = {
-      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
-    }
-
-    // Basic window testing.
-    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
-    assertEqual("foo(*) over ()", windowed())
-    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
-    assertEqual("foo(*) over (distribute by a, b)", windowed(Seq('a, 'b)))
-    assertEqual("foo(*) over (cluster by a, b)", windowed(Seq('a, 'b)))
-    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
-    assertEqual("foo(*) over (sort by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
-    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
-    assertEqual("foo(*) over (distribute by a, b sort by c)", windowed(Seq('a, 'b), Seq('c.asc)))
-
-    // Test use of expressions in window functions.
-    assertEqual(
-      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
-      WindowExpression('sum.function('product + 1),
-        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
-    assertEqual(
-      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
-      WindowExpression('sum.function('product + 1),
-        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
-
-    // Range/Row
-    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
-    val boundaries = Seq(
-      ("10 preceding", ValuePreceding(10), CurrentRow),
-      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
-      ("unbounded preceding", UnboundedPreceding, CurrentRow),
-      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
-      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
-      ("between unbounded preceding and unbounded following",
-        UnboundedPreceding, UnboundedFollowing),
-      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
-      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
-      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
-    )
-    frameTypes.foreach {
-      case (frameTypeSql, frameType) =>
-        boundaries.foreach {
-          case (boundarySql, begin, end) =>
-            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
-            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
-            assertEqual(query, expr)
-        }
-    }
-
-    // We cannot use non integer constants.
-    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
-      "Frame bound value must be a constant integer.")
-
-    // We cannot use an arbitrary expression.
-    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
-      "Frame bound value must be a constant integer.")
-  }
-
-  test("row constructor") {
-    // Note that '(a)' will be interpreted as a nested expression.
-    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
-    assertEqual("(a, b, c)", CreateStruct(Seq('a, 'b, 'c)))
-  }
-
-  test("scalar sub-query") {
-    assertEqual(
-      "(select max(val) from tbl) > current",
-      ScalarSubquery(table("tbl").select('max.function('val))) > 'current)
-    assertEqual(
-      "a = (select b from s)",
-      'a === ScalarSubquery(table("s").select('b)))
-  }
-
-  test("case when") {
-    assertEqual("case a when 1 then b when 2 then c else d end",
-      CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd)))
-    assertEqual("case when a = 1 then b when a = 2 then c else d end",
-      CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd))
-  }
-
-  test("dereference") {
-    assertEqual("a.b", UnresolvedAttribute("a.b"))
-    assertEqual("`select`.b", UnresolvedAttribute("select.b"))
-    assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis.
-    assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b"))
-  }
-
-  test("reference") {
-    // Regular
-    assertEqual("a", 'a)
-
-    // Starting with a digit.
-    assertEqual("1a", Symbol("1a"))
-
-    // Quoted using a keyword.
-    assertEqual("`select`", 'select)
-
-    // Unquoted using an unreserved keyword.
-    assertEqual("columns", 'columns)
-  }
-
-  test("subscript") {
-    assertEqual("a[b]", 'a.getItem('b))
-    assertEqual("a[1 + 1]", 'a.getItem(Literal(1) + 1))
-    assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem('b))
-  }
-
-  test("parenthesis") {
-    assertEqual("(a)", 'a)
-    assertEqual("r * (a + b)", 'r * ('a + 'b))
-  }
-
-  test("type constructors") {
-    // Dates.
-    assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11")))
-    intercept[IllegalArgumentException] {
-      parseExpression("DAtE 'mar 11 2016'")
-    }
-
-    // Timestamps.
-    assertEqual("tImEstAmp '2016-03-11 20:54:00.000'",
-      Literal(Timestamp.valueOf("2016-03-11 20:54:00.000")))
-    intercept[IllegalArgumentException] {
-      parseExpression("timestamP '2016-33-11 20:54:00.000'")
-    }
-
-    // Unsupported datatype.
-    intercept("GEO '(10,-6)'", "Literals of type 'GEO' are currently not supported.")
-  }
-
-  test("literals") {
-    // NULL
-    assertEqual("null", Literal(null))
-
-    // Boolean
-    assertEqual("trUe", Literal(true))
-    assertEqual("False", Literal(false))
-
-    // Integral should have the narrowest possible type
-    assertEqual("787324", Literal(787324))
-    assertEqual("7873247234798249234", Literal(7873247234798249234L))
-    assertEqual("78732472347982492793712334",
-      Literal(BigDecimal("78732472347982492793712334").underlying()))
-
-    // Decimal
-    assertEqual("7873247234798249279371.2334",
-      Literal(BigDecimal("7873247234798249279371.2334").underlying()))
-
-    // Scientific Decimal
-    assertEqual("9.0e1", 90d)
-    assertEqual(".9e+2", 90d)
-    assertEqual("0.9e+2", 90d)
-    assertEqual("900e-1", 90d)
-    assertEqual("900.0E-1", 90d)
-    assertEqual("9.e+1", 90d)
-    intercept(".e3")
-
-    // Tiny Int Literal
-    assertEqual("10Y", Literal(10.toByte))
-    intercept("-1000Y")
-
-    // Small Int Literal
-    assertEqual("10S", Literal(10.toShort))
-    intercept("40000S")
-
-    // Long Int Literal
-    assertEqual("10L", Literal(10L))
-    intercept("78732472347982492793712334L")
-
-    // Double Literal
-    assertEqual("10.0D", Literal(10.0D))
-    // TODO we need to figure out if we should throw an exception here!
-    assertEqual("1E309", Literal(Double.PositiveInfinity))
-  }
-
-  test("strings") {
-    // Single Strings.
-    assertEqual("\"hello\"", "hello")
-    assertEqual("'hello'", "hello")
-
-    // Multi-Strings.
-    assertEqual("\"hello\" 'world'", "helloworld")
-    assertEqual("'hello' \" \" 'world'", "hello world")
-
-    // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
-    // regular '%'; to get the correct result you need to add another escaped '\'.
-    // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
-    assertEqual("'pattern%'", "pattern%")
-    assertEqual("'no-pattern\\%'", "no-pattern\\%")
-    assertEqual("'pattern\\\\%'", "pattern\\%")
-    assertEqual("'pattern\\\\\\%'", "pattern\\\\%")
-
-    // Escaped characters.
-    // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
-    assertEqual("'\\0'", "\u0000") // ASCII NUL (X'00')
-    assertEqual("'\\''", "\'")     // Single quote
-    assertEqual("'\\\"'", "\"")    // Double quote
-    assertEqual("'\\b'", "\b")     // Backspace
-    assertEqual("'\\n'", "\n")     // Newline
-    assertEqual("'\\r'", "\r")     // Carriage return
-    assertEqual("'\\t'", "\t")     // Tab character
-    assertEqual("'\\Z'", "\u001A") // ASCII 26 - CTRL + Z (EOF on windows)
-
-    // Octals
-    assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!")
-
-    // Unicode
-    assertEqual("'\\u0087\\u0111\\u0114\\u0108\\u0100\\u0032\\u0058\\u0041'", "World :)")
-  }
-
-  test("intervals") {
-    def intervalLiteral(u: String, s: String): Literal = {
-      Literal(CalendarInterval.fromSingleUnitString(u, s))
-    }
-
-    // Empty interval statement
-    intercept("interval", "at least one time unit should be given for interval literal")
-
-    // Single Intervals.
-    val units = Seq(
-      "year",
-      "month",
-      "week",
-      "day",
-      "hour",
-      "minute",
-      "second",
-      "millisecond",
-      "microsecond")
-    val forms = Seq("", "s")
-    val values = Seq("0", "10", "-7", "21")
-    units.foreach { unit =>
-      forms.foreach { form =>
-         values.foreach { value =>
-           val expected = intervalLiteral(unit, value)
-           assertEqual(s"interval $value $unit$form", expected)
-           assertEqual(s"interval '$value' $unit$form", expected)
-         }
-      }
-    }
-
-    // Hive nanosecond notation.
-    assertEqual("interval 13.123456789 seconds", intervalLiteral("second", "13.123456789"))
-    assertEqual("interval -13.123456789 second", intervalLiteral("second", "-13.123456789"))
-
-    // Non Existing unit
-    intercept("interval 10 nanoseconds", "No interval can be constructed")
-
-    // Year-Month intervals.
-    val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0")
-    yearMonthValues.foreach { value =>
-      val result = Literal(CalendarInterval.fromYearMonthString(value))
-      assertEqual(s"interval '$value' year to month", result)
-    }
-
-    // Day-Time intervals.
-    val datTimeValues = Seq(
-      "99 11:22:33.123456789",
-      "-99 11:22:33.123456789",
-      "10 9:8:7.123456789",
-      "1 0:0:0",
-      "-1 0:0:0",
-      "1 0:0:1")
-    datTimeValues.foreach { value =>
-      val result = Literal(CalendarInterval.fromDayTimeString(value))
-      assertEqual(s"interval '$value' day to second", result)
-    }
-
-    // Unknown FROM TO intervals
-    intercept("interval 10 month to second", "Intervals FROM month TO second are not supported.")
-
-    // Composed intervals.
-    assertEqual(
-      "interval 3 months 22 seconds 1 millisecond",
-      Literal(new CalendarInterval(3, 22001000L)))
-    assertEqual(
-      "interval 3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second",
-      Literal(new CalendarInterval(14,
-        22 * CalendarInterval.MICROS_PER_DAY + 2 * CalendarInterval.MICROS_PER_SECOND)))
-  }
-
-  test("composed expressions") {
-    assertEqual("1 + r.r As q", (Literal(1) + UnresolvedAttribute("r.r")).as("q"))
-    assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar)))
-    intercept("1 - f('o', o(bar)) hello * world", "mismatched input '*'")
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala
deleted file mode 100644
index 4206d22..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala
+++ /dev/null
@@ -1,429 +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.parser.ng
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.types.{BooleanType, IntegerType}
-
-class PlanParserSuite extends PlanTest {
-  import CatalystSqlParser._
-  import org.apache.spark.sql.catalyst.dsl.expressions._
-  import org.apache.spark.sql.catalyst.dsl.plans._
-
-  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
-    comparePlans(parsePlan(sqlCommand), plan)
-  }
-
-  def intercept(sqlCommand: String, messages: String*): Unit = {
-    val e = intercept[ParseException](parsePlan(sqlCommand))
-    messages.foreach { message =>
-      assert(e.message.contains(message))
-    }
-  }
-
-  test("case insensitive") {
-    val plan = table("a").select(star())
-    assertEqual("sELEct * FroM a", plan)
-    assertEqual("select * fRoM a", plan)
-    assertEqual("SELECT * FROM a", plan)
-  }
-
-  test("show functions") {
-    assertEqual("show functions", ShowFunctions(None, None))
-    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
-    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
-    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
-    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
-  }
-
-  test("describe function") {
-    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
-    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
-    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
-    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
-  }
-
-  test("set operations") {
-    val a = table("a").select(star())
-    val b = table("b").select(star())
-
-    assertEqual("select * from a union select * from b", Distinct(a.union(b)))
-    assertEqual("select * from a union distinct select * from b", Distinct(a.union(b)))
-    assertEqual("select * from a union all select * from b", a.union(b))
-    assertEqual("select * from a except select * from b", a.except(b))
-    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
-    assertEqual("select * from a except distinct select * from b", a.except(b))
-    assertEqual("select * from a intersect select * from b", a.intersect(b))
-    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
-    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
-  }
-
-  test("common table expressions") {
-    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
-      val ctes = namedPlans.map {
-        case (name, cte) =>
-          name -> SubqueryAlias(name, cte)
-      }.toMap
-      With(plan, ctes)
-    }
-    assertEqual(
-      "with cte1 as (select * from a) select * from cte1",
-      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
-    assertEqual(
-      "with cte1 (select 1) select * from cte1",
-      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
-    assertEqual(
-      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
-      cte(table("cte2").select(star()),
-        "cte1" -> OneRowRelation.select(1),
-        "cte2" -> table("cte1").select(star())))
-    intercept(
-      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
-      "Name 'cte1' is used for multiple common table expressions")
-  }
-
-  test("simple select query") {
-    assertEqual("select 1", OneRowRelation.select(1))
-    assertEqual("select a, b", OneRowRelation.select('a, 'b))
-    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
-    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
-    assertEqual(
-      "select a, b from db.c having x < 1",
-      table("db", "c").select('a, 'b).where(('x < 1).cast(BooleanType)))
-    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
-    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
-  }
-
-  test("reverse select query") {
-    assertEqual("from a", table("a"))
-    assertEqual("from a select b, c", table("a").select('b, 'c))
-    assertEqual(
-      "from db.a select b, c where d < 1", table("db", "a").where('d < 1).select('b, 'c))
-    assertEqual("from a select distinct b, c", Distinct(table("a").select('b, 'c)))
-    assertEqual(
-      "from (from a union all from b) c select *",
-      table("a").union(table("b")).as("c").select(star()))
-  }
-
-  test("transform query spec") {
-    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
-    assertEqual("select transform(a, b) using 'func' from e where f < 10",
-      p.copy(child = p.child.where('f < 10), output = Seq('key.string, 'value.string)))
-    assertEqual("map a, b using 'func' as c, d from e",
-      p.copy(output = Seq('c.string, 'd.string)))
-    assertEqual("reduce a, b using 'func' as (c: int, d decimal(10, 0)) from e",
-      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
-  }
-
-  test("multi select query") {
-    assertEqual(
-      "from a select * select * where s < 10",
-      table("a").select(star()).union(table("a").where('s < 10).select(star())))
-    intercept(
-      "from a select * select * from x where a.s < 10",
-      "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements")
-    assertEqual(
-      "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
-      table("a").select(star()).insertInto("tbl1").union(
-        table("a").where('s < 10).select(star()).insertInto("tbl2")))
-  }
-
-  test("query organization") {
-    // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
-    val baseSql = "select * from t"
-    val basePlan = table("t").select(star())
-
-    val ws = Map("w1" -> WindowSpecDefinition(Seq.empty, Seq.empty, UnspecifiedFrame))
-    val limitWindowClauses = Seq(
-      ("", (p: LogicalPlan) => p),
-      (" limit 10", (p: LogicalPlan) => p.limit(10)),
-      (" window w1 as ()", (p: LogicalPlan) => WithWindowDefinition(ws, p)),
-      (" window w1 as () limit 10", (p: LogicalPlan) => WithWindowDefinition(ws, p).limit(10))
-    )
-
-    val orderSortDistrClusterClauses = Seq(
-      ("", basePlan),
-      (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)),
-      (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)),
-      (" distribute by a, b", basePlan.distribute('a, 'b)),
-      (" distribute by a sort by b", basePlan.distribute('a).sortBy('b.asc)),
-      (" cluster by a, b", basePlan.distribute('a, 'b).sortBy('a.asc, 'b.asc))
-    )
-
-    orderSortDistrClusterClauses.foreach {
-      case (s1, p1) =>
-        limitWindowClauses.foreach {
-          case (s2, pf2) =>
-            assertEqual(baseSql + s1 + s2, pf2(p1))
-        }
-    }
-
-    val msg = "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported"
-    intercept(s"$baseSql order by a sort by a", msg)
-    intercept(s"$baseSql cluster by a distribute by a", msg)
-    intercept(s"$baseSql order by a cluster by a", msg)
-    intercept(s"$baseSql order by a distribute by a", msg)
-  }
-
-  test("insert into") {
-    val sql = "select * from t"
-    val plan = table("t").select(star())
-    def insert(
-        partition: Map[String, Option[String]],
-        overwrite: Boolean = false,
-        ifNotExists: Boolean = false): LogicalPlan =
-      InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
-
-    // Single inserts
-    assertEqual(s"insert overwrite table s $sql",
-      insert(Map.empty, overwrite = true))
-    assertEqual(s"insert overwrite table s if not exists $sql",
-      insert(Map.empty, overwrite = true, ifNotExists = true))
-    assertEqual(s"insert into s $sql",
-      insert(Map.empty))
-    assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql",
-      insert(Map("c" -> Option("d"), "e" -> Option("1"))))
-    assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql",
-      insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true))
-
-    // Multi insert
-    val plan2 = table("t").where('x > 5).select(star())
-    assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5",
-      InsertIntoTable(
-        table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).union(
-        InsertIntoTable(
-          table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false)))
-  }
-
-  test("aggregation") {
-    val sql = "select a, b, sum(c) as c from d group by a, b"
-
-    // Normal
-    assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c")))
-
-    // Cube
-    assertEqual(s"$sql with cube",
-      table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
-
-    // Rollup
-    assertEqual(s"$sql with rollup",
-      table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
-
-    // Grouping Sets
-    assertEqual(s"$sql grouping sets((a, b), (a), ())",
-      GroupingSets(Seq(0, 1, 3), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c"))))
-    intercept(s"$sql grouping sets((a, b), (c), ())",
-      "c doesn't show up in the GROUP BY list")
-  }
-
-  test("limit") {
-    val sql = "select * from t"
-    val plan = table("t").select(star())
-    assertEqual(s"$sql limit 10", plan.limit(10))
-    assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
-  }
-
-  test("window spec") {
-    // Note that WindowSpecs are testing in the ExpressionParserSuite
-    val sql = "select * from t"
-    val plan = table("t").select(star())
-    val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc),
-      SpecifiedWindowFrame(RowFrame, ValuePreceding(1), ValueFollowing(1)))
-
-    // Test window resolution.
-    val ws1 = Map("w1" -> spec, "w2" -> spec, "w3" -> spec)
-    assertEqual(
-      s"""$sql
-         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
-         |       w2 as w1,
-         |       w3 as w1""".stripMargin,
-      WithWindowDefinition(ws1, plan))
-
-    // Fail with no reference.
-    intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")
-
-    // Fail when resolved reference is not a window spec.
-    intercept(
-      s"""$sql
-         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
-         |       w2 as w1,
-         |       w3 as w2""".stripMargin,
-      "Window reference 'w2' is not a window specification"
-    )
-  }
-
-  test("lateral view") {
-    // Single lateral view
-    assertEqual(
-      "select * from t lateral view explode(x) expl as x",
-      table("t")
-        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
-        .select(star()))
-
-    // Multiple lateral views
-    assertEqual(
-      """select *
-        |from t
-        |lateral view explode(x) expl
-        |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin,
-      table("t")
-        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq.empty)
-        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z"))
-        .select(star()))
-
-    // Multi-Insert lateral views.
-    val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
-    assertEqual(
-      """from t1
-        |lateral view explode(x) expl as x
-        |insert into t2
-        |select *
-        |lateral view json_tuple(x, y) jtup q, z
-        |insert into t3
-        |select *
-        |where s < 10
-      """.stripMargin,
-      Union(from
-        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z"))
-        .select(star())
-        .insertInto("t2"),
-        from.where('s < 10).select(star()).insertInto("t3")))
-
-    // Unsupported generator.
-    intercept(
-      "select * from t lateral view posexplode(x) posexpl as x, y",
-      "Generator function 'posexplode' is not supported")
-  }
-
-  test("joins") {
-    // Test single joins.
-    val testUnconditionalJoin = (sql: String, jt: JoinType) => {
-      assertEqual(
-        s"select * from t as tt $sql u",
-        table("t").as("tt").join(table("u"), jt, None).select(star()))
-    }
-    val testConditionalJoin = (sql: String, jt: JoinType) => {
-      assertEqual(
-        s"select * from t $sql u as uu on a = b",
-        table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star()))
-    }
-    val testNaturalJoin = (sql: String, jt: JoinType) => {
-      assertEqual(
-        s"select * from t tt natural $sql u as uu",
-        table("t").as("tt").join(table("u").as("uu"), NaturalJoin(jt), None).select(star()))
-    }
-    val testUsingJoin = (sql: String, jt: JoinType) => {
-      assertEqual(
-        s"select * from t $sql u using(a, b)",
-        table("t").join(table("u"), UsingJoin(jt, Seq('a.attr, 'b.attr)), None).select(star()))
-    }
-    val testAll = Seq(testUnconditionalJoin, testConditionalJoin, testNaturalJoin, testUsingJoin)
-
-    def test(sql: String, jt: JoinType, tests: Seq[(String, JoinType) => Unit]): Unit = {
-      tests.foreach(_(sql, jt))
-    }
-    test("cross join", Inner, Seq(testUnconditionalJoin))
-    test(",", Inner, Seq(testUnconditionalJoin))
-    test("join", Inner, testAll)
-    test("inner join", Inner, testAll)
-    test("left join", LeftOuter, testAll)
-    test("left outer join", LeftOuter, testAll)
-    test("right join", RightOuter, testAll)
-    test("right outer join", RightOuter, testAll)
-    test("full join", FullOuter, testAll)
-    test("full outer join", FullOuter, testAll)
-
-    // Test multiple consecutive joins
-    assertEqual(
-      "select * from a join b join c right join d",
-      table("a").join(table("b")).join(table("c")).join(table("d"), RightOuter).select(star()))
-  }
-
-  test("sampled relations") {
-    val sql = "select * from t"
-    assertEqual(s"$sql tablesample(100 rows)",
-      table("t").limit(100).select(star()))
-    assertEqual(s"$sql tablesample(43 percent) as x",
-      Sample(0, .43d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
-    assertEqual(s"$sql tablesample(bucket 4 out of 10) as x",
-      Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
-    intercept(s"$sql tablesample(bucket 4 out of 10 on x) as x",
-      "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported")
-    intercept(s"$sql tablesample(bucket 11 out of 10) as x",
-      s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]")
-  }
-
-  test("sub-query") {
-    val plan = table("t0").select('id)
-    assertEqual("select id from (t0)", plan)
-    assertEqual("select id from ((((((t0))))))", plan)
-    assertEqual(
-      "(select * from t1) union distinct (select * from t2)",
-      Distinct(table("t1").select(star()).union(table("t2").select(star()))))
-    assertEqual(
-      "select * from ((select * from t1) union (select * from t2)) t",
-      Distinct(
-        table("t1").select(star()).union(table("t2").select(star()))).as("t").select(star()))
-    assertEqual(
-      """select  id
-        |from (((select id from t0)
-        |       union all
-        |       (select  id from t0))
-        |      union all
-        |      (select id from t0)) as u_1
-      """.stripMargin,
-      plan.union(plan).union(plan).as("u_1").select('id))
-  }
-
-  test("scalar sub-query") {
-    assertEqual(
-      "select (select max(b) from s) ss from t",
-      table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss")))
-    assertEqual(
-      "select * from t where a = (select b from s)",
-      table("t").where('a === ScalarSubquery(table("s").select('b))).select(star()))
-    assertEqual(
-      "select g from t group by g having a > (select b from s)",
-      table("t")
-        .groupBy('g)('g)
-        .where(('a > ScalarSubquery(table("s").select('b))).cast(BooleanType)))
-  }
-
-  test("table reference") {
-    assertEqual("table t", table("t"))
-    assertEqual("table d.t", table("d", "t"))
-  }
-
-  test("inline table") {
-    assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
-      Seq('col1.int),
-      Seq(1, 2, 3, 4).map(x => Row(x))))
-    assertEqual(
-      "values (1, 'a'), (2, 'b'), (3, 'c') as tbl(a, b)",
-      LocalRelation.fromExternalRows(
-        Seq('a.int, 'b.string),
-        Seq((1, "a"), (2, "b"), (3, "c")).map(x => Row(x._1, x._2))).as("tbl"))
-    intercept("values (a, 'a'), (b, 'b')",
-      "All expressions in an inline table must be constants.")
-    intercept("values (1, 'a'), (2, 'b') as tbl(a, b, c)",
-      "Number of aliases must match the number of fields in an inline table.")
-    intercept[ArrayIndexOutOfBoundsException](parsePlan("values (1, 'a'), (2, 'b', 5Y)"))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/TableIdentifierParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/TableIdentifierParserSuite.scala
deleted file mode 100644
index 0874322..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/TableIdentifierParserSuite.scala
+++ /dev/null
@@ -1,42 +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.parser.ng
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.TableIdentifier
-
-class TableIdentifierParserSuite extends SparkFunSuite {
-  import CatalystSqlParser._
-
-  test("table identifier") {
-    // Regular names.
-    assert(TableIdentifier("q") === parseTableIdentifier("q"))
-    assert(TableIdentifier("q", Option("d")) === parseTableIdentifier("d.q"))
-
-    // Illegal names.
-    intercept[ParseException](parseTableIdentifier(""))
-    intercept[ParseException](parseTableIdentifier("d.q.g"))
-
-    // SQL Keywords.
-    val keywords = Seq("select", "from", "where", "left", "right")
-    keywords.foreach { keyword =>
-      intercept[ParseException](parseTableIdentifier(keyword))
-      assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`"))
-      assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`"))
-    }
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[4/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala
deleted file mode 100644
index c188c5b..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/CatalystQl.scala
+++ /dev/null
@@ -1,933 +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.parser
-
-import java.sql.Date
-
-import scala.collection.mutable.ArrayBuffer
-import scala.util.matching.Regex
-
-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.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-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(val conf: ParserConf = SimpleParserConf()) extends ParserInterface {
-  import ParserUtils._
-
-  /**
-   * 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)
-
-  /**
-   * 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 withLateralView = lateralViewClause.map { lv =>
-            nodeToGenerate(lv.children.head, outer = false, relations)
-          }.getOrElse(relations)
-
-          val withWhere = whereClause.map { whereNode =>
-            val Seq(whereExpr) = whereNode.children
-            Filter(nodeToExpr(whereExpr), withLateralView)
-          }.getOrElse(withLateralView)
-
-          val select = (selectClause orElse selectDistinctClause)
-            .getOrElse(sys.error("No select clause."))
-
-          val transformation = nodeToTransformation(select.children.head, 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, withWhere)
-                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, withWhere, 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,
-                    withWhere)
-                case _ => sys.error("Expect WITH ROLLUP")
-              }),
-              cubeGroupByClause.map(e => e match {
-                case Token("TOK_CUBE_GROUPBY", children) =>
-                  Aggregate(
-                    Seq(Cube(children.map(nodeToExpr))),
-                    selectExpressions,
-                    withWhere)
-                case _ => sys.error("Expect WITH CUBE")
-              }),
-              Some(Project(selectExpressions, withWhere))).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)(
-              isTableSample = true)
-          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)(
-              isTableSample = true)
-          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, joinCondition) = getJoinInfo(joinToken, other, node)
-
-        Join(nodeToRelation(relation1),
-          nodeToRelation(relation2),
-          joinType,
-          joinCondition)
-      case _ =>
-        noParseRule("Relation", node)
-    }
-  }
-
-  protected def getJoinInfo(
-     joinToken: String,
-     joinConditionToken: Seq[ASTNode],
-     node: ASTNode): (JoinType, Option[Expression]) = {
-    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)
-    }
-
-    joinConditionToken match {
-      case Token("TOK_USING", columnList :: Nil) :: Nil =>
-        val colNames = columnList.children.collect {
-          case Token(name, Nil) => UnresolvedAttribute(name)
-        }
-        (UsingJoin(joinType, colNames), None)
-      /* Join expression specified using ON clause */
-      case _ => (joinType, joinConditionToken.headOption.map(nodeToExpr))
-    }
-  }
-
-  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)
-  }
-
-  /**
-   * Flattens the left deep tree with the specified pattern into a list.
-   */
-  private def flattenLeftDeepTree(node: ASTNode, pattern: Regex): Seq[ASTNode] = {
-    val collected = ArrayBuffer[ASTNode]()
-    var rest = node
-    while (rest match {
-      case Token(pattern(), l :: r :: Nil) =>
-        collected += r
-        rest = l
-        true
-      case _ => false
-    }) {
-      // do nothing
-    }
-    collected += rest
-    // keep them in the same order as in SQL
-    collected.reverse
-  }
-
-  /**
-   * Creates a balanced tree that has similar number of nodes on left and right.
-   *
-   * This help to reduce the depth of the tree to prevent StackOverflow in analyzer/optimizer.
-   */
-  private def balancedTree(
-      expr: Seq[Expression],
-      f: (Expression, Expression) => Expression): Expression = expr.length match {
-    case 1 => expr.head
-    case 2 => f(expr.head, expr(1))
-    case l => f(balancedTree(expr.slice(0, l / 2), f), balancedTree(expr.slice(l / 2, l), f))
-  }
-
-  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) =>
-      balancedTree(flattenLeftDeepTree(node, AND).map(nodeToExpr), And)
-    case Token(OR(), left :: right:: Nil) =>
-      balancedTree(flattenLeftDeepTree(node, OR).map(nodeToExpr), Or)
-    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(cleanIdentifier(a.toLowerCase))
-    }
-
-    Generate(
-      generator,
-      join = true,
-      outer = outer,
-      Some(cleanIdentifier(alias.toLowerCase)),
-      attributes,
-      child)
-  }
-
-  protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node)
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParser.scala
index 21deb82..0b570c9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParser.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.parser
 import scala.language.implicitConversions
 import scala.util.matching.Regex
 import scala.util.parsing.combinator.syntactical.StandardTokenParsers
+import scala.util.parsing.input.CharArrayReader._
 
 import org.apache.spark.sql.types._
 
@@ -117,3 +118,69 @@ private[sql] object DataTypeParser {
 
 /** The exception thrown from the [[DataTypeParser]]. */
 private[sql] class DataTypeException(message: String) extends Exception(message)
+
+class SqlLexical extends scala.util.parsing.combinator.lexical.StdLexical {
+  case class DecimalLit(chars: String) extends Token {
+    override def toString: String = chars
+  }
+
+  /* This is a work around to support the lazy setting */
+  def initialize(keywords: Seq[String]): Unit = {
+    reserved.clear()
+    reserved ++= keywords
+  }
+
+  /* Normal the keyword string */
+  def normalizeKeyword(str: String): String = str.toLowerCase
+
+  delimiters += (
+    "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
+    ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~", "<=>"
+    )
+
+  protected override def processIdent(name: String) = {
+    val token = normalizeKeyword(name)
+    if (reserved contains token) Keyword(token) else Identifier(name)
+  }
+
+  override lazy val token: Parser[Token] =
+    ( rep1(digit) ~ scientificNotation ^^ { case i ~ s => DecimalLit(i.mkString + s) }
+      | '.' ~> (rep1(digit) ~ scientificNotation) ^^
+      { case i ~ s => DecimalLit("0." + i.mkString + s) }
+      | rep1(digit) ~ ('.' ~> digit.*) ~ scientificNotation ^^
+      { case i1 ~ i2 ~ s => DecimalLit(i1.mkString + "." + i2.mkString + s) }
+      | digit.* ~ identChar ~ (identChar | digit).* ^^
+      { case first ~ middle ~ rest => processIdent((first ++ (middle :: rest)).mkString) }
+      | rep1(digit) ~ ('.' ~> digit.*).? ^^ {
+      case i ~ None => NumericLit(i.mkString)
+      case i ~ Some(d) => DecimalLit(i.mkString + "." + d.mkString)
+    }
+      | '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^
+      { case chars => StringLit(chars mkString "") }
+      | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^
+      { case chars => StringLit(chars mkString "") }
+      | '`' ~> chrExcept('`', '\n', EofCh).* <~ '`' ^^
+      { case chars => Identifier(chars mkString "") }
+      | EofCh ^^^ EOF
+      | '\'' ~> failure("unclosed string literal")
+      | '"' ~> failure("unclosed string literal")
+      | delim
+      | failure("illegal character")
+      )
+
+  override def identChar: Parser[Elem] = letter | elem('_')
+
+  private lazy val scientificNotation: Parser[String] =
+    (elem('e') | elem('E')) ~> (elem('+') | elem('-')).? ~ rep1(digit) ^^ {
+      case s ~ rest => "e" + s.mkString + rest.mkString
+    }
+
+  override def whitespace: Parser[Any] =
+    ( whitespaceChar
+      | '/' ~ '*' ~ comment
+      | '/' ~ '/' ~ chrExcept(EofCh, '\n').*
+      | '#' ~ chrExcept(EofCh, '\n').*
+      | '-' ~ '-' ~ chrExcept(EofCh, '\n').*
+      | '/' ~ '*' ~ failure("unclosed comment")
+      ).*
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index 51cfc50..d013252 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -16,91 +16,106 @@
  */
 package org.apache.spark.sql.catalyst.parser
 
-import scala.annotation.tailrec
-
-import org.antlr.runtime._
-import org.antlr.runtime.tree.CommonTree
+import org.antlr.v4.runtime._
+import org.antlr.v4.runtime.atn.PredictionMode
+import org.antlr.v4.runtime.misc.ParseCancellationException
 
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.Origin
+import org.apache.spark.sql.types.DataType
 
 /**
- * The ParseDriver takes a SQL command and turns this into an AST.
- *
- * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver
+ * Base SQL parsing infrastructure.
  */
-object ParseDriver extends Logging {
-  /** Create an LogicalPlan ASTNode from a SQL command. */
-  def parsePlan(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser =>
-    parser.statement().getTree
-  }
+abstract class AbstractSqlParser extends ParserInterface with Logging {
 
-  /** Create an Expression ASTNode from a SQL command. */
-  def parseExpression(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser =>
-    parser.singleNamedExpression().getTree
+  /** Creates/Resolves DataType for a given SQL string. */
+  def parseDataType(sqlText: String): DataType = parse(sqlText) { parser =>
+    // TODO add this to the parser interface.
+    astBuilder.visitSingleDataType(parser.singleDataType())
   }
 
-  /** Create an TableIdentifier ASTNode from a SQL command. */
-  def parseTableName(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser =>
-    parser.singleTableName().getTree
+  /** Creates Expression for a given SQL string. */
+  override def parseExpression(sqlText: String): Expression = parse(sqlText) { parser =>
+    astBuilder.visitSingleExpression(parser.singleExpression())
   }
 
-  private def parse(
-      command: String,
-      conf: ParserConf)(
-      toTree: SparkSqlParser => CommonTree): ASTNode = {
-    logInfo(s"Parsing command: $command")
+  /** Creates TableIdentifier for a given SQL string. */
+  override def parseTableIdentifier(sqlText: String): TableIdentifier = parse(sqlText) { parser =>
+    astBuilder.visitSingleTableIdentifier(parser.singleTableIdentifier())
+  }
 
-    // Setup error collection.
-    val reporter = new ParseErrorReporter()
+  /** Creates LogicalPlan for a given SQL string. */
+  override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
+    astBuilder.visitSingleStatement(parser.singleStatement()) match {
+      case plan: LogicalPlan => plan
+      case _ => nativeCommand(sqlText)
+    }
+  }
 
-    // Create lexer.
-    val lexer = new SparkSqlLexer(new ANTLRNoCaseStringStream(command))
-    val tokens = new TokenRewriteStream(lexer)
-    lexer.configure(conf, reporter)
+  /** Get the builder (visitor) which converts a ParseTree into a AST. */
+  protected def astBuilder: AstBuilder
 
-    // Create the parser.
-    val parser = new SparkSqlParser(tokens)
-    parser.configure(conf, reporter)
+  /** Create a native command, or fail when this is not supported. */
+  protected def nativeCommand(sqlText: String): LogicalPlan = {
+    val position = Origin(None, None)
+    throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position)
+  }
 
-    try {
-      val result = toTree(parser)
+  protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    logInfo(s"Parsing command: $command")
 
-      // Check errors.
-      reporter.checkForErrors()
+    val lexer = new SqlBaseLexer(new ANTLRNoCaseStringStream(command))
+    lexer.removeErrorListeners()
+    lexer.addErrorListener(ParseErrorListener)
 
-      // Return the AST node from the result.
-      logInfo(s"Parse completed.")
+    val tokenStream = new CommonTokenStream(lexer)
+    val parser = new SqlBaseParser(tokenStream)
+    parser.addParseListener(PostProcessor)
+    parser.removeErrorListeners()
+    parser.addErrorListener(ParseErrorListener)
 
-      // Find the non null token tree in the result.
-      @tailrec
-      def nonNullToken(tree: CommonTree): CommonTree = {
-        if (tree.token != null || tree.getChildCount == 0) tree
-        else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree])
+    try {
+      try {
+        // first, try parsing with potentially faster SLL mode
+        parser.getInterpreter.setPredictionMode(PredictionMode.SLL)
+        toResult(parser)
       }
-      val tree = nonNullToken(result)
-
-      // Make sure all boundaries are set.
-      tree.setUnknownTokenBoundaries()
-
-      // Construct the immutable AST.
-      def createASTNode(tree: CommonTree): ASTNode = {
-        val children = (0 until tree.getChildCount).map { i =>
-          createASTNode(tree.getChild(i).asInstanceOf[CommonTree])
-        }.toList
-        ASTNode(tree.token, tree.getTokenStartIndex, tree.getTokenStopIndex, children, tokens)
+      catch {
+        case e: ParseCancellationException =>
+          // if we fail, parse with LL mode
+          tokenStream.reset() // rewind input stream
+          parser.reset()
+
+          // Try Again.
+          parser.getInterpreter.setPredictionMode(PredictionMode.LL)
+          toResult(parser)
       }
-      createASTNode(tree)
     }
     catch {
-      case e: RecognitionException =>
-        logInfo(s"Parse failed.")
-        reporter.throwError(e)
+      case e: ParseException if e.command.isDefined =>
+        throw e
+      case e: ParseException =>
+        throw e.withCommand(command)
+      case e: AnalysisException =>
+        val position = Origin(e.line, e.startPosition)
+        throw new ParseException(Option(command), e.message, position, position)
     }
   }
 }
 
 /**
+ * Concrete SQL parser for Catalyst-only SQL statements.
+ */
+object CatalystSqlParser extends AbstractSqlParser {
+  val astBuilder = new AstBuilder
+}
+
+/**
  * This string stream provides the lexer with upper case characters only. This greatly simplifies
  * lexing the stream, while we can maintain the original command.
  *
@@ -120,58 +135,104 @@ object ParseDriver extends Logging {
  * have the ANTLRNoCaseStringStream implementation.
  */
 
-private[parser] class ANTLRNoCaseStringStream(input: String) extends ANTLRStringStream(input) {
+private[parser] class ANTLRNoCaseStringStream(input: String) extends ANTLRInputStream(input) {
   override def LA(i: Int): Int = {
     val la = super.LA(i)
-    if (la == 0 || la == CharStream.EOF) la
+    if (la == 0 || la == IntStream.EOF) la
     else Character.toUpperCase(la)
   }
 }
 
 /**
- * Utility used by the Parser and the Lexer for error collection and reporting.
+ * The ParseErrorListener converts parse errors into AnalysisExceptions.
  */
-private[parser] class ParseErrorReporter {
-  val errors = scala.collection.mutable.Buffer.empty[ParseError]
-
-  def report(br: BaseRecognizer, re: RecognitionException, tokenNames: Array[String]): Unit = {
-    errors += ParseError(br, re, tokenNames)
+case object ParseErrorListener extends BaseErrorListener {
+  override def syntaxError(
+      recognizer: Recognizer[_, _],
+      offendingSymbol: scala.Any,
+      line: Int,
+      charPositionInLine: Int,
+      msg: String,
+      e: RecognitionException): Unit = {
+    val position = Origin(Some(line), Some(charPositionInLine))
+    throw new ParseException(None, msg, position, position)
   }
+}
 
-  def checkForErrors(): Unit = {
-    if (errors.nonEmpty) {
-      val first = errors.head
-      val e = first.re
-      throwError(e.line, e.charPositionInLine, first.buildMessage().toString, errors.tail)
-    }
+/**
+ * A [[ParseException]] is an [[AnalysisException]] that is thrown during the parse process. It
+ * contains fields and an extended error message that make reporting and diagnosing errors easier.
+ */
+class ParseException(
+    val command: Option[String],
+    message: String,
+    val start: Origin,
+    val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) {
+
+  def this(message: String, ctx: ParserRuleContext) = {
+    this(Option(ParserUtils.command(ctx)),
+      message,
+      ParserUtils.position(ctx.getStart),
+      ParserUtils.position(ctx.getStop))
   }
 
-  def throwError(e: RecognitionException): Nothing = {
-    throwError(e.line, e.charPositionInLine, e.toString, errors)
+  override def getMessage: String = {
+    val builder = new StringBuilder
+    builder ++= "\n" ++= message
+    start match {
+      case Origin(Some(l), Some(p)) =>
+        builder ++= s"(line $l, pos $p)\n"
+        command.foreach { cmd =>
+          val (above, below) = cmd.split("\n").splitAt(l)
+          builder ++= "\n== SQL ==\n"
+          above.foreach(builder ++= _ += '\n')
+          builder ++= (0 until p).map(_ => "-").mkString("") ++= "^^^\n"
+          below.foreach(builder ++= _ += '\n')
+        }
+      case _ =>
+        command.foreach { cmd =>
+          builder ++= "\n== SQL ==\n" ++= cmd
+        }
+    }
+    builder.toString
   }
 
-  private def throwError(
-      line: Int,
-      startPosition: Int,
-      msg: String,
-      errors: Seq[ParseError]): Nothing = {
-    val b = new StringBuilder
-    b.append(msg).append("\n")
-    errors.foreach(error => error.buildMessage(b).append("\n"))
-    throw new AnalysisException(b.toString, Option(line), Option(startPosition))
+  def withCommand(cmd: String): ParseException = {
+    new ParseException(Option(cmd), message, start, stop)
   }
 }
 
 /**
- * Error collected during the parsing process.
- *
- * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseError
+ * The post-processor validates & cleans-up the parse tree during the parse process.
  */
-private[parser] case class ParseError(
-    br: BaseRecognizer,
-    re: RecognitionException,
-    tokenNames: Array[String]) {
-  def buildMessage(s: StringBuilder = new StringBuilder): StringBuilder = {
-    s.append(br.getErrorHeader(re)).append(" ").append(br.getErrorMessage(re, tokenNames))
+case object PostProcessor extends SqlBaseBaseListener {
+
+  /** Remove the back ticks from an Identifier. */
+  override def exitQuotedIdentifier(ctx: SqlBaseParser.QuotedIdentifierContext): Unit = {
+    replaceTokenByIdentifier(ctx, 1) { token =>
+      // Remove the double back ticks in the string.
+      token.setText(token.getText.replace("``", "`"))
+      token
+    }
+  }
+
+  /** Treat non-reserved keywords as Identifiers. */
+  override def exitNonReserved(ctx: SqlBaseParser.NonReservedContext): Unit = {
+    replaceTokenByIdentifier(ctx, 0)(identity)
+  }
+
+  private def replaceTokenByIdentifier(
+      ctx: ParserRuleContext,
+      stripMargins: Int)(
+      f: CommonToken => CommonToken = identity): Unit = {
+    val parent = ctx.getParent
+    parent.removeLastChild()
+    val token = ctx.getChild(0).getPayload.asInstanceOf[Token]
+    parent.addChild(f(new CommonToken(
+      new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream),
+      SqlBaseParser.IDENTIFIER,
+      token.getChannel,
+      token.getStartIndex + stripMargins,
+      token.getStopIndex - stripMargins)))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala
deleted file mode 100644
index ce449b1..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala
+++ /dev/null
@@ -1,26 +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.parser
-
-trait ParserConf {
-  def supportQuotedId: Boolean
-  def supportSQL11ReservedKeywords: Boolean
-}
-
-case class SimpleParserConf(
-    supportQuotedId: Boolean = true,
-    supportSQL11ReservedKeywords: Boolean = false) extends ParserConf

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
index 0c2e481..90b76dc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
@@ -14,166 +14,105 @@
  * 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.AnalysisException
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.types._
+import org.antlr.v4.runtime.{CharStream, ParserRuleContext, Token}
+import org.antlr.v4.runtime.misc.Interval
+import org.antlr.v4.runtime.tree.TerminalNode
 
+import org.apache.spark.sql.catalyst.parser.ParseUtils.unescapeSQLString
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
 
 /**
- * A collection of utility methods and patterns for parsing query texts.
+ * A collection of utility methods for use during the parsing process.
  */
-// TODO: merge with ParseUtils
 object ParserUtils {
-
-  object Token {
-    // Match on (text, children)
-    def unapply(node: ASTNode): Some[(String, List[ASTNode])] = {
-      CurrentOrigin.setPosition(node.line, node.positionInLine)
-      node.pattern
-    }
+  /** Get the command which created the token. */
+  def command(ctx: ParserRuleContext): String = {
+    command(ctx.getStart.getInputStream)
   }
 
-  private val escapedIdentifier = "`(.+)`".r
-  private val doubleQuotedString = "\"([^\"]+)\"".r
-  private val singleQuotedString = "'([^']+)'".r
-
-  // Token patterns
-  val COUNT = "(?i)COUNT".r
-  val SUM = "(?i)SUM".r
-  val AND = "(?i)AND".r
-  val OR = "(?i)OR".r
-  val NOT = "(?i)NOT".r
-  val TRUE = "(?i)TRUE".r
-  val FALSE = "(?i)FALSE".r
-  val LIKE = "(?i)LIKE".r
-  val RLIKE = "(?i)RLIKE".r
-  val REGEXP = "(?i)REGEXP".r
-  val IN = "(?i)IN".r
-  val DIV = "(?i)DIV".r
-  val BETWEEN = "(?i)BETWEEN".r
-  val WHEN = "(?i)WHEN".r
-  val CASE = "(?i)CASE".r
-  val INTEGRAL = "[+-]?\\d+".r
-  val DECIMAL = "[+-]?((\\d+(\\.\\d*)?)|(\\.\\d+))".r
-
-  /**
-   * Strip quotes, if any, from the string.
-   */
-  def unquoteString(str: String): String = {
-    str match {
-      case singleQuotedString(s) => s
-      case doubleQuotedString(s) => s
-      case other => other
-    }
+  /** Get the command which created the token. */
+  def command(stream: CharStream): String = {
+    stream.getText(Interval.of(0, stream.size()))
   }
 
-  /**
-   * Strip backticks, if any, from the string.
-   */
-  def cleanIdentifier(ident: String): String = {
-    ident match {
-      case escapedIdentifier(i) => i
-      case plainIdent => plainIdent
-    }
+  /** Get the code that creates the given node. */
+  def source(ctx: ParserRuleContext): String = {
+    val stream = ctx.getStart.getInputStream
+    stream.getText(Interval.of(ctx.getStart.getStartIndex, ctx.getStop.getStopIndex))
   }
 
-  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
-    }
+  /** Get all the text which comes after the given rule. */
+  def remainder(ctx: ParserRuleContext): String = remainder(ctx.getStop)
 
-    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
+  /** Get all the text which comes after the given token. */
+  def remainder(token: Token): String = {
+    val stream = token.getInputStream
+    val interval = Interval.of(token.getStopIndex + 1, stream.size())
+    stream.getText(interval)
   }
 
-  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")}"))
-  }
+  /** Convert a string token into a string. */
+  def string(token: Token): String = unescapeSQLString(token.getText)
 
-  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")
-    }
-  }
+  /** Convert a string node into a string. */
+  def string(node: TerminalNode): String = unescapeSQLString(node.getText)
 
-  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'")
-    }
+  /** Get the origin (line and position) of the token. */
+  def position(token: Token): Origin = {
+    Origin(Option(token.getLine), Option(token.getCharPositionInLine))
   }
 
-  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)
+  /** Assert if a condition holds. If it doesn't throw a parse exception. */
+  def assert(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = {
+    if (!f) {
+      throw new ParseException(message, ctx)
+    }
   }
 
   /**
-   * Throw an exception because we cannot parse the given node for some unexpected reason.
+   * Register the origin of the context. Any TreeNode created in the closure will be assigned the
+   * registered origin. This method restores the previously set origin after completion of the
+   * closure.
    */
-  def parseFailed(msg: String, node: ASTNode): Nothing = {
-    throw new AnalysisException(s"$msg: '${node.source}")
+  def withOrigin[T](ctx: ParserRuleContext)(f: => T): T = {
+    val current = CurrentOrigin.get
+    CurrentOrigin.set(position(ctx.getStart))
+    try {
+      f
+    } finally {
+      CurrentOrigin.set(current)
+    }
   }
 
-  /**
-   * Throw an exception because there are no rules to parse the 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}")
-  }
+  /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */
+  implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal {
+    /**
+     * Create a plan using the block of code when the given context exists. Otherwise return the
+     * original plan.
+     */
+    def optional(ctx: AnyRef)(f: => LogicalPlan): LogicalPlan = {
+      if (ctx != null) {
+        f
+      } else {
+        plan
+      }
+    }
 
+    /**
+     * Map a [[LogicalPlan]] to another [[LogicalPlan]] if the passed context exists using the
+     * passed function. The original plan is returned when the context does not exist.
+     */
+    def optionalMap[C <: ParserRuleContext](
+        ctx: C)(
+        f: (C, LogicalPlan) => LogicalPlan): LogicalPlan = {
+      if (ctx != null) {
+        f(ctx, plan)
+      } else {
+        plan
+      }
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[5/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
new file mode 100644
index 0000000..c350f30
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -0,0 +1,1460 @@
+/*
+ * 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, Timestamp}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.{ParserRuleContext, Token}
+import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.util.random.RandomSampler
+
+/**
+ * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
+ * TableIdentifier.
+ */
+class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
+  import ParserUtils._
+
+  protected def typedVisit[T](ctx: ParseTree): T = {
+    ctx.accept(this).asInstanceOf[T]
+  }
+
+  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
+    visit(ctx.statement).asInstanceOf[LogicalPlan]
+  }
+
+  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
+    visitNamedExpression(ctx.namedExpression)
+  }
+
+  override def visitSingleTableIdentifier(
+      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
+    visitTableIdentifier(ctx.tableIdentifier)
+  }
+
+  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
+    visit(ctx.dataType).asInstanceOf[DataType]
+  }
+
+  /* ********************************************************************************************
+   * Plan parsing
+   * ******************************************************************************************** */
+  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
+
+  /**
+   * Make sure we do not try to create a plan for a native command.
+   */
+  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
+
+  /**
+   * Create a plan for a SHOW FUNCTIONS command.
+   */
+  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
+    import ctx._
+    if (qualifiedName != null) {
+      val names = qualifiedName().identifier().asScala.map(_.getText).toList
+      names match {
+        case db :: name :: Nil =>
+          ShowFunctions(Some(db), Some(name))
+        case name :: Nil =>
+          ShowFunctions(None, Some(name))
+        case _ =>
+          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
+      }
+    } else if (pattern != null) {
+      ShowFunctions(None, Some(string(pattern)))
+    } else {
+      ShowFunctions(None, None)
+    }
+  }
+
+  /**
+   * Create a plan for a DESCRIBE FUNCTION command.
+   */
+  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
+    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
+    DescribeFunction(functionName, ctx.EXTENDED != null)
+  }
+
+  /**
+   * Create a top-level plan with Common Table Expressions.
+   */
+  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
+    val query = plan(ctx.queryNoWith)
+
+    // Apply CTEs
+    query.optional(ctx.ctes) {
+      val ctes = ctx.ctes.namedQuery.asScala.map {
+        case nCtx =>
+          val namedQuery = visitNamedQuery(nCtx)
+          (namedQuery.alias, namedQuery)
+      }
+
+      // Check for duplicate names.
+      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
+        case (name, _) =>
+          throw new ParseException(
+            s"Name '$name' is used for multiple common table expressions", ctx)
+      }
+
+      With(query, ctes.toMap)
+    }
+  }
+
+  /**
+   * Create a named logical plan.
+   *
+   * This is only used for Common Table Expressions.
+   */
+  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
+    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
+  }
+
+  /**
+   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
+   * queries have the following SQL form:
+   * {{{
+   *   [WITH cte...]?
+   *   FROM src
+   *   [INSERT INTO tbl1 SELECT *]+
+   * }}}
+   * For example:
+   * {{{
+   *   FROM db.tbl1 A
+   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
+   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
+   * }}}
+   * This (Hive) feature cannot be combined with set-operators.
+   */
+  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
+    val from = visitFromClause(ctx.fromClause)
+
+    // Build the insert clauses.
+    val inserts = ctx.multiInsertQueryBody.asScala.map {
+      body =>
+        assert(body.querySpecification.fromClause == null,
+          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
+          body)
+
+        withQuerySpecification(body.querySpecification, from).
+          // Add organization statements.
+          optionalMap(body.queryOrganization)(withQueryResultClauses).
+          // Add insert.
+          optionalMap(body.insertInto())(withInsertInto)
+    }
+
+    // If there are multiple INSERTS just UNION them together into one query.
+    inserts match {
+      case Seq(query) => query
+      case queries => Union(queries)
+    }
+  }
+
+  /**
+   * Create a logical plan for a regular (single-insert) query.
+   */
+  override def visitSingleInsertQuery(
+      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
+    plan(ctx.queryTerm).
+      // Add organization statements.
+      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
+      // Add insert.
+      optionalMap(ctx.insertInto())(withInsertInto)
+  }
+
+  /**
+   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
+   */
+  private def withInsertInto(
+      ctx: InsertIntoContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
+    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
+
+    InsertIntoTable(
+      UnresolvedRelation(tableIdent, None),
+      partitionKeys,
+      query,
+      ctx.OVERWRITE != null,
+      ctx.EXISTS != null)
+  }
+
+  /**
+   * Create a partition specification map.
+   */
+  override def visitPartitionSpec(
+      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
+    ctx.partitionVal.asScala.map { pVal =>
+      val name = pVal.identifier.getText.toLowerCase
+      val value = Option(pVal.constant).map(visitStringConstant)
+      name -> value
+    }.toMap
+  }
+
+  /**
+   * Create a partition specification map without optional values.
+   */
+  protected def visitNonOptionalPartitionSpec(
+      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
+    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
+  }
+
+  /**
+   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
+   * main purpose is to prevent slight differences due to back to back conversions i.e.:
+   * String -> Literal -> String.
+   */
+  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
+    ctx match {
+      case s: StringLiteralContext => createString(s)
+      case o => o.getText
+    }
+  }
+
+  /**
+   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
+   * clauses determine the shape (ordering/partitioning/rows) of the query result.
+   */
+  private def withQueryResultClauses(
+      ctx: QueryOrganizationContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    import ctx._
+
+    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
+    val withOrder = if (
+      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
+      // ORDER BY ...
+      Sort(order.asScala.map(visitSortItem), global = true, query)
+    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
+      // SORT BY ...
+      Sort(sort.asScala.map(visitSortItem), global = false, query)
+    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
+      // DISTRIBUTE BY ...
+      RepartitionByExpression(expressionList(distributeBy), query)
+    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
+      // SORT BY ... DISTRIBUTE BY ...
+      Sort(
+        sort.asScala.map(visitSortItem),
+        global = false,
+        RepartitionByExpression(expressionList(distributeBy), query))
+    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
+      // CLUSTER BY ...
+      val expressions = expressionList(clusterBy)
+      Sort(
+        expressions.map(SortOrder(_, Ascending)),
+        global = false,
+        RepartitionByExpression(expressions, query))
+    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
+      // [EMPTY]
+      query
+    } else {
+      throw new ParseException(
+        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
+    }
+
+    // WINDOWS
+    val withWindow = withOrder.optionalMap(windows)(withWindows)
+
+    // LIMIT
+    withWindow.optional(limit) {
+      Limit(typedVisit(limit), withWindow)
+    }
+  }
+
+  /**
+   * Create a logical plan using a query specification.
+   */
+  override def visitQuerySpecification(
+      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
+    val from = OneRowRelation.optional(ctx.fromClause) {
+      visitFromClause(ctx.fromClause)
+    }
+    withQuerySpecification(ctx, from)
+  }
+
+  /**
+   * Add a query specification to a logical plan. The query specification is the core of the logical
+   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
+   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
+   *
+   * Note that query hints are ignored (both by the parser and the builder).
+   */
+  private def withQuerySpecification(
+      ctx: QuerySpecificationContext,
+      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    import ctx._
+
+    // WHERE
+    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
+      Filter(expression(ctx), plan)
+    }
+
+    // Expressions.
+    val expressions = Option(namedExpressionSeq).toSeq
+      .flatMap(_.namedExpression.asScala)
+      .map(typedVisit[Expression])
+
+    // Create either a transform or a regular query.
+    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
+    specType match {
+      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
+        // Transform
+
+        // Add where.
+        val withFilter = relation.optionalMap(where)(filter)
+
+        // Create the attributes.
+        val (attributes, schemaLess) = if (colTypeList != null) {
+          // Typed return columns.
+          (createStructType(colTypeList).toAttributes, false)
+        } else if (identifierSeq != null) {
+          // Untyped return columns.
+          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
+            AttributeReference(name, StringType, nullable = true)()
+          }
+          (attrs, false)
+        } else {
+          (Seq(AttributeReference("key", StringType)(),
+            AttributeReference("value", StringType)()), true)
+        }
+
+        // Create the transform.
+        ScriptTransformation(
+          expressions,
+          string(script),
+          attributes,
+          withFilter,
+          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
+
+      case SqlBaseParser.SELECT =>
+        // Regular select
+
+        // Add lateral views.
+        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
+
+        // Add where.
+        val withFilter = withLateralView.optionalMap(where)(filter)
+
+        // Add aggregation or a project.
+        val namedExpressions = expressions.map {
+          case e: NamedExpression => e
+          case e: Expression => UnresolvedAlias(e)
+        }
+        val withProject = if (aggregation != null) {
+          withAggregation(aggregation, namedExpressions, withFilter)
+        } else if (namedExpressions.nonEmpty) {
+          Project(namedExpressions, withFilter)
+        } else {
+          withFilter
+        }
+
+        // Having
+        val withHaving = withProject.optional(having) {
+          // 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(expression(having), BooleanType), withProject)
+        }
+
+        // Distinct
+        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
+          Distinct(withHaving)
+        } else {
+          withHaving
+        }
+
+        // Window
+        withDistinct.optionalMap(windows)(withWindows)
+    }
+  }
+
+  /**
+   * Create a (Hive based) [[ScriptInputOutputSchema]].
+   */
+  protected def withScriptIOSchema(
+      inRowFormat: RowFormatContext,
+      recordWriter: Token,
+      outRowFormat: RowFormatContext,
+      recordReader: Token,
+      schemaLess: Boolean): ScriptInputOutputSchema = null
+
+  /**
+   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
+   * separated) relations here, these get converted into a single plan by condition-less inner join.
+   */
+  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
+    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
+    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
+  }
+
+  /**
+   * Connect two queries by a Set operator.
+   *
+   * Supported Set operators are:
+   * - UNION [DISTINCT]
+   * - UNION ALL
+   * - EXCEPT [DISTINCT]
+   * - INTERSECT [DISTINCT]
+   */
+  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
+    val left = plan(ctx.left)
+    val right = plan(ctx.right)
+    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
+    ctx.operator.getType match {
+      case SqlBaseParser.UNION if all =>
+        Union(left, right)
+      case SqlBaseParser.UNION =>
+        Distinct(Union(left, right))
+      case SqlBaseParser.INTERSECT if all =>
+        throw new ParseException("INTERSECT ALL is not supported.", ctx)
+      case SqlBaseParser.INTERSECT =>
+        Intersect(left, right)
+      case SqlBaseParser.EXCEPT if all =>
+        throw new ParseException("EXCEPT ALL is not supported.", ctx)
+      case SqlBaseParser.EXCEPT =>
+        Except(left, right)
+    }
+  }
+
+  /**
+   * Add a [[WithWindowDefinition]] operator to a logical plan.
+   */
+  private def withWindows(
+      ctx: WindowsContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // Collect all window specifications defined in the WINDOW clause.
+    val baseWindowMap = ctx.namedWindow.asScala.map {
+      wCtx =>
+        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
+    }.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 windowMapView = baseWindowMap.mapValues {
+      case WindowSpecReference(name) =>
+        baseWindowMap.get(name) match {
+          case Some(spec: WindowSpecDefinition) =>
+            spec
+          case Some(ref) =>
+            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
+          case None =>
+            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
+        }
+      case spec: WindowSpecDefinition => spec
+    }
+
+    // Note that mapValues creates a view instead of materialized map. We force materialization by
+    // mapping over identity.
+    WithWindowDefinition(windowMapView.map(identity), query)
+  }
+
+  /**
+   * Add an [[Aggregate]] to a logical plan.
+   */
+  private def withAggregation(
+      ctx: AggregationContext,
+      selectExpressions: Seq[NamedExpression],
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    import ctx._
+    val groupByExpressions = expressionList(groupingExpressions)
+
+    if (GROUPING != null) {
+      // GROUP BY .... GROUPING SETS (...)
+      val expressionMap = groupByExpressions.zipWithIndex.toMap
+      val numExpressions = expressionMap.size
+      val mask = (1 << numExpressions) - 1
+      val masks = ctx.groupingSet.asScala.map {
+        _.expression.asScala.foldLeft(mask) {
+          case (bitmap, eCtx) =>
+            // Find the index of the expression.
+            val e = typedVisit[Expression](eCtx)
+            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
+              throw new ParseException(
+                s"$e doesn't show up in the GROUP BY list", ctx))
+            // 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 << (numExpressions - 1 - index))
+        }
+      }
+      GroupingSets(masks, groupByExpressions, query, selectExpressions)
+    } else {
+      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
+      val mappedGroupByExpressions = if (CUBE != null) {
+        Seq(Cube(groupByExpressions))
+      } else if (ROLLUP != null) {
+        Seq(Rollup(groupByExpressions))
+      } else {
+        groupByExpressions
+      }
+      Aggregate(mappedGroupByExpressions, selectExpressions, query)
+    }
+  }
+
+  /**
+   * Add a [[Generate]] (Lateral View) to a logical plan.
+   */
+  private def withGenerate(
+      query: LogicalPlan,
+      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
+    val expressions = expressionList(ctx.expression)
+
+    // Create the generator.
+    val generator = ctx.qualifiedName.getText.toLowerCase match {
+      case "explode" if expressions.size == 1 =>
+        Explode(expressions.head)
+      case "json_tuple" =>
+        JsonTuple(expressions)
+      case other =>
+        withGenerator(other, expressions, ctx)
+    }
+
+    Generate(
+      generator,
+      join = true,
+      outer = ctx.OUTER != null,
+      Some(ctx.tblName.getText.toLowerCase),
+      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
+      query)
+  }
+
+  /**
+   * Create a [[Generator]]. Override this method in order to support custom Generators.
+   */
+  protected def withGenerator(
+      name: String,
+      expressions: Seq[Expression],
+      ctx: LateralViewContext): Generator = {
+    throw new ParseException(s"Generator function '$name' is not supported", ctx)
+  }
+
+  /**
+   * Create a joins between two or more logical plans.
+   */
+  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
+    /** Build a join between two plans. */
+    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
+      val baseJoinType = ctx.joinType match {
+        case null => Inner
+        case jt if jt.FULL != null => FullOuter
+        case jt if jt.SEMI != null => LeftSemi
+        case jt if jt.LEFT != null => LeftOuter
+        case jt if jt.RIGHT != null => RightOuter
+        case _ => Inner
+      }
+
+      // Resolve the join type and join condition
+      val (joinType, condition) = Option(ctx.joinCriteria) match {
+        case Some(c) if c.USING != null =>
+          val columns = c.identifier.asScala.map { column =>
+            UnresolvedAttribute.quoted(column.getText)
+          }
+          (UsingJoin(baseJoinType, columns), None)
+        case Some(c) if c.booleanExpression != null =>
+          (baseJoinType, Option(expression(c.booleanExpression)))
+        case None if ctx.NATURAL != null =>
+          (NaturalJoin(baseJoinType), None)
+        case None =>
+          (baseJoinType, None)
+      }
+      Join(left, right, joinType, condition)
+    }
+
+    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
+    // first join clause is at the top. However fields of previously referenced tables can be used
+    // in following join clauses. The tree needs to be reversed in order to make this work.
+    var result = plan(ctx.left)
+    var current = ctx
+    while (current != null) {
+      current.right match {
+        case right: JoinRelationContext =>
+          result = join(current, result, plan(right.left))
+          current = right
+        case right =>
+          result = join(current, result, plan(right))
+          current = null
+      }
+    }
+    result
+  }
+
+  /**
+   * Add a [[Sample]] to a logical plan.
+   *
+   * This currently supports the following sampling methods:
+   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
+   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
+   * are defined as a number between 0 and 100.
+   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
+   */
+  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // Create a sampled plan if we need one.
+    def sample(fraction: Double): Sample = {
+      // 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.
+      val eps = RandomSampler.roundingEpsilon
+      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
+        s"Sampling fraction ($fraction) must be on interval [0, 1]",
+        ctx)
+      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
+    }
+
+    ctx.sampleType.getType match {
+      case SqlBaseParser.ROWS =>
+        Limit(expression(ctx.expression), query)
+
+      case SqlBaseParser.PERCENTLIT =>
+        val fraction = ctx.percentage.getText.toDouble
+        sample(fraction / 100.0d)
+
+      case SqlBaseParser.BUCKET if ctx.ON != null =>
+        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
+
+      case SqlBaseParser.BUCKET =>
+        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
+    }
+  }
+
+  /**
+   * Create a logical plan for a sub-query.
+   */
+  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
+    plan(ctx.queryNoWith)
+  }
+
+  /**
+   * Create an un-aliased table reference. This is typically used for top-level table references,
+   * for example:
+   * {{{
+   *   INSERT INTO db.tbl2
+   *   TABLE db.tbl1
+   * }}}
+   */
+  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
+    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
+  }
+
+  /**
+   * Create an aliased table reference. This is typically used in FROM clauses.
+   */
+  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
+    val table = UnresolvedRelation(
+      visitTableIdentifier(ctx.tableIdentifier),
+      Option(ctx.identifier).map(_.getText))
+    table.optionalMap(ctx.sample)(withSample)
+  }
+
+  /**
+   * Create an inline table (a virtual table in Hive parlance).
+   */
+  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
+    // Get the backing expressions.
+    val expressions = ctx.expression.asScala.map { eCtx =>
+      val e = expression(eCtx)
+      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
+      e
+    }
+
+    // Validate and evaluate the rows.
+    val (structType, structConstructor) = expressions.head.dataType match {
+      case st: StructType =>
+        (st, (e: Expression) => e)
+      case dt =>
+        val st = CreateStruct(Seq(expressions.head)).dataType
+        (st, (e: Expression) => CreateStruct(Seq(e)))
+    }
+    val rows = expressions.map {
+      case expression =>
+        val safe = Cast(structConstructor(expression), structType)
+        safe.eval().asInstanceOf[InternalRow]
+    }
+
+    // Construct attributes.
+    val baseAttributes = structType.toAttributes.map(_.withNullability(true))
+    val attributes = if (ctx.identifierList != null) {
+      val aliases = visitIdentifierList(ctx.identifierList)
+      assert(aliases.size == baseAttributes.size,
+        "Number of aliases must match the number of fields in an inline table.", ctx)
+      baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
+    } else {
+      baseAttributes
+    }
+
+    // Create plan and add an alias if a name has been defined.
+    LocalRelation(attributes, rows).optionalMap(ctx.identifier)(aliasPlan)
+  }
+
+  /**
+   * Create an alias (SubqueryAlias) for a join relation. This is practically the same as
+   * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
+   * hooks.
+   */
+  override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
+    plan(ctx.relation).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan)
+  }
+
+  /**
+   * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as
+   * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different
+   * hooks.
+   */
+  override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
+    plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan)
+  }
+
+  /**
+   * Create an alias (SubqueryAlias) for a LogicalPlan.
+   */
+  private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = {
+    SubqueryAlias(alias.getText, plan)
+  }
+
+  /**
+   * Create a Sequence of Strings for a parenthesis enclosed alias list.
+   */
+  override def visitIdentifierList(ctx: IdentifierListContext): Seq[String] = withOrigin(ctx) {
+    visitIdentifierSeq(ctx.identifierSeq)
+  }
+
+  /**
+   * Create a Sequence of Strings for an identifier list.
+   */
+  override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) {
+    ctx.identifier.asScala.map(_.getText)
+  }
+
+  /* ********************************************************************************************
+   * Table Identifier parsing
+   * ******************************************************************************************** */
+  /**
+   * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern.
+   */
+  override def visitTableIdentifier(
+      ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) {
+    TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
+  }
+
+  /* ********************************************************************************************
+   * Expression parsing
+   * ******************************************************************************************** */
+  /**
+   * Create an expression from the given context. This method just passes the context on to the
+   * vistor and only takes care of typing (We assume that the visitor returns an Expression here).
+   */
+  protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx)
+
+  /**
+   * Create sequence of expressions from the given sequence of contexts.
+   */
+  private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = {
+    trees.asScala.map(expression)
+  }
+
+  /**
+   * Invert a boolean expression if it has a valid NOT clause.
+   */
+  private def invertIfNotDefined(expression: Expression, not: TerminalNode): Expression = {
+    if (not != null) {
+      Not(expression)
+    } else {
+      expression
+    }
+  }
+
+  /**
+   * Create a star (i.e. all) expression; this selects all elements (in the specified object).
+   * Both un-targeted (global) and targeted aliases are supported.
+   */
+  override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) {
+    UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText)))
+  }
+
+  /**
+   * Create an aliased expression if an alias is specified. Both single and multi-aliases are
+   * supported.
+   */
+  override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
+    val e = expression(ctx.expression)
+    if (ctx.identifier != null) {
+      Alias(e, ctx.identifier.getText)()
+    } else if (ctx.identifierList != null) {
+      MultiAlias(e, visitIdentifierList(ctx.identifierList))
+    } else {
+      e
+    }
+  }
+
+  /**
+   * Combine a number of boolean expressions into a balanced expression tree. These expressions are
+   * either combined by a logical [[And]] or a logical [[Or]].
+   *
+   * A balanced binary tree is created because regular left recursive trees cause considerable
+   * performance degradations and can cause stack overflows.
+   */
+  override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) {
+    val expressionType = ctx.operator.getType
+    val expressionCombiner = expressionType match {
+      case SqlBaseParser.AND => And.apply _
+      case SqlBaseParser.OR => Or.apply _
+    }
+
+    // Collect all similar left hand contexts.
+    val contexts = ArrayBuffer(ctx.right)
+    var current = ctx.left
+    def collectContexts: Boolean = current match {
+      case lbc: LogicalBinaryContext if lbc.operator.getType == expressionType =>
+        contexts += lbc.right
+        current = lbc.left
+        true
+      case _ =>
+        contexts += current
+        false
+    }
+    while (collectContexts) {
+      // No body - all updates take place in the collectContexts.
+    }
+
+    // Reverse the contexts to have them in the same sequence as in the SQL statement & turn them
+    // into expressions.
+    val expressions = contexts.reverse.map(expression)
+
+    // Create a balanced tree.
+    def reduceToExpressionTree(low: Int, high: Int): Expression = high - low match {
+      case 0 =>
+        expressions(low)
+      case 1 =>
+        expressionCombiner(expressions(low), expressions(high))
+      case x =>
+        val mid = low + x / 2
+        expressionCombiner(
+          reduceToExpressionTree(low, mid),
+          reduceToExpressionTree(mid + 1, high))
+    }
+    reduceToExpressionTree(0, expressions.size - 1)
+  }
+
+  /**
+   * Invert a boolean expression.
+   */
+  override def visitLogicalNot(ctx: LogicalNotContext): Expression = withOrigin(ctx) {
+    Not(expression(ctx.booleanExpression()))
+  }
+
+  /**
+   * Create a filtering correlated sub-query. This is not supported yet.
+   */
+  override def visitExists(ctx: ExistsContext): Expression = {
+    throw new ParseException("EXISTS clauses are not supported.", ctx)
+  }
+
+  /**
+   * Create a comparison expression. This compares two expressions. The following comparison
+   * operators are supported:
+   * - Equal: '=' or '=='
+   * - Null-safe Equal: '<=>'
+   * - Not Equal: '<>' or '!='
+   * - Less than: '<'
+   * - Less then or Equal: '<='
+   * - Greater than: '>'
+   * - Greater then or Equal: '>='
+   */
+  override def visitComparison(ctx: ComparisonContext): Expression = withOrigin(ctx) {
+    val left = expression(ctx.left)
+    val right = expression(ctx.right)
+    val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
+    operator.getSymbol.getType match {
+      case SqlBaseParser.EQ =>
+        EqualTo(left, right)
+      case SqlBaseParser.NSEQ =>
+        EqualNullSafe(left, right)
+      case SqlBaseParser.NEQ | SqlBaseParser.NEQJ =>
+        Not(EqualTo(left, right))
+      case SqlBaseParser.LT =>
+        LessThan(left, right)
+      case SqlBaseParser.LTE =>
+        LessThanOrEqual(left, right)
+      case SqlBaseParser.GT =>
+        GreaterThan(left, right)
+      case SqlBaseParser.GTE =>
+        GreaterThanOrEqual(left, right)
+    }
+  }
+
+  /**
+   * Create a BETWEEN expression. This tests if an expression lies with in the bounds set by two
+   * other expressions. The inverse can also be created.
+   */
+  override def visitBetween(ctx: BetweenContext): Expression = withOrigin(ctx) {
+    val value = expression(ctx.value)
+    val between = And(
+      GreaterThanOrEqual(value, expression(ctx.lower)),
+      LessThanOrEqual(value, expression(ctx.upper)))
+    invertIfNotDefined(between, ctx.NOT)
+  }
+
+  /**
+   * Create an IN expression. This tests if the value of the left hand side expression is
+   * contained by the sequence of expressions on the right hand side.
+   */
+  override def visitInList(ctx: InListContext): Expression = withOrigin(ctx) {
+    val in = In(expression(ctx.value), ctx.expression().asScala.map(expression))
+    invertIfNotDefined(in, ctx.NOT)
+  }
+
+  /**
+   * Create an IN expression, where the the right hand side is a query. This is unsupported.
+   */
+  override def visitInSubquery(ctx: InSubqueryContext): Expression = {
+    throw new ParseException("IN with a Sub-query is currently not supported.", ctx)
+  }
+
+  /**
+   * Create a (R)LIKE/REGEXP expression.
+   */
+  override def visitLike(ctx: LikeContext): Expression = {
+    val left = expression(ctx.value)
+    val right = expression(ctx.pattern)
+    val like = ctx.like.getType match {
+      case SqlBaseParser.LIKE =>
+        Like(left, right)
+      case SqlBaseParser.RLIKE =>
+        RLike(left, right)
+    }
+    invertIfNotDefined(like, ctx.NOT)
+  }
+
+  /**
+   * Create an IS (NOT) NULL expression.
+   */
+  override def visitNullPredicate(ctx: NullPredicateContext): Expression = withOrigin(ctx) {
+    val value = expression(ctx.value)
+    if (ctx.NOT != null) {
+      IsNotNull(value)
+    } else {
+      IsNull(value)
+    }
+  }
+
+  /**
+   * Create a binary arithmetic expression. The following arithmetic operators are supported:
+   * - Mulitplication: '*'
+   * - Division: '/'
+   * - Hive Long Division: 'DIV'
+   * - Modulo: '%'
+   * - Addition: '+'
+   * - Subtraction: '-'
+   * - Binary AND: '&'
+   * - Binary XOR
+   * - Binary OR: '|'
+   */
+  override def visitArithmeticBinary(ctx: ArithmeticBinaryContext): Expression = withOrigin(ctx) {
+    val left = expression(ctx.left)
+    val right = expression(ctx.right)
+    ctx.operator.getType match {
+      case SqlBaseParser.ASTERISK =>
+        Multiply(left, right)
+      case SqlBaseParser.SLASH =>
+        Divide(left, right)
+      case SqlBaseParser.PERCENT =>
+        Remainder(left, right)
+      case SqlBaseParser.DIV =>
+        Cast(Divide(left, right), LongType)
+      case SqlBaseParser.PLUS =>
+        Add(left, right)
+      case SqlBaseParser.MINUS =>
+        Subtract(left, right)
+      case SqlBaseParser.AMPERSAND =>
+        BitwiseAnd(left, right)
+      case SqlBaseParser.HAT =>
+        BitwiseXor(left, right)
+      case SqlBaseParser.PIPE =>
+        BitwiseOr(left, right)
+    }
+  }
+
+  /**
+   * Create a unary arithmetic expression. The following arithmetic operators are supported:
+   * - Plus: '+'
+   * - Minus: '-'
+   * - Bitwise Not: '~'
+   */
+  override def visitArithmeticUnary(ctx: ArithmeticUnaryContext): Expression = withOrigin(ctx) {
+    val value = expression(ctx.valueExpression)
+    ctx.operator.getType match {
+      case SqlBaseParser.PLUS =>
+        value
+      case SqlBaseParser.MINUS =>
+        UnaryMinus(value)
+      case SqlBaseParser.TILDE =>
+        BitwiseNot(value)
+    }
+  }
+
+  /**
+   * Create a [[Cast]] expression.
+   */
+  override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) {
+    Cast(expression(ctx.expression), typedVisit(ctx.dataType))
+  }
+
+  /**
+   * Create a (windowed) Function expression.
+   */
+  override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) {
+    // Create the function call.
+    val name = ctx.qualifiedName.getText
+    val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null)
+    val arguments = ctx.expression().asScala.map(expression) match {
+      case Seq(UnresolvedStar(None)) if name.toLowerCase == "count" && !isDistinct =>
+        // Transform COUNT(*) into COUNT(1). Move this to analysis?
+        Seq(Literal(1))
+      case expressions =>
+        expressions
+    }
+    val function = UnresolvedFunction(name, arguments, isDistinct)
+
+    // Check if the function is evaluated in a windowed context.
+    ctx.windowSpec match {
+      case spec: WindowRefContext =>
+        UnresolvedWindowExpression(function, visitWindowRef(spec))
+      case spec: WindowDefContext =>
+        WindowExpression(function, visitWindowDef(spec))
+      case _ => function
+    }
+  }
+
+  /**
+   * Create a reference to a window frame, i.e. [[WindowSpecReference]].
+   */
+  override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) {
+    WindowSpecReference(ctx.identifier.getText)
+  }
+
+  /**
+   * Create a window definition, i.e. [[WindowSpecDefinition]].
+   */
+  override def visitWindowDef(ctx: WindowDefContext): WindowSpecDefinition = withOrigin(ctx) {
+    // CLUSTER BY ... | PARTITION BY ... ORDER BY ...
+    val partition = ctx.partition.asScala.map(expression)
+    val order = ctx.sortItem.asScala.map(visitSortItem)
+
+    // RANGE/ROWS BETWEEN ...
+    val frameSpecOption = Option(ctx.windowFrame).map { frame =>
+      val frameType = frame.frameType.getType match {
+        case SqlBaseParser.RANGE => RangeFrame
+        case SqlBaseParser.ROWS => RowFrame
+      }
+
+      SpecifiedWindowFrame(
+        frameType,
+        visitFrameBound(frame.start),
+        Option(frame.end).map(visitFrameBound).getOrElse(CurrentRow))
+    }
+
+    WindowSpecDefinition(
+      partition,
+      order,
+      frameSpecOption.getOrElse(UnspecifiedFrame))
+  }
+
+  /**
+   * Create or resolve a [[FrameBoundary]]. Simple math expressions are allowed for Value
+   * Preceding/Following boundaries. These expressions must be constant (foldable) and return an
+   * integer value.
+   */
+  override def visitFrameBound(ctx: FrameBoundContext): FrameBoundary = withOrigin(ctx) {
+    // We currently only allow foldable integers.
+    def value: Int = {
+      val e = expression(ctx.expression)
+      assert(e.resolved && e.foldable && e.dataType == IntegerType,
+        "Frame bound value must be a constant integer.",
+        ctx)
+      e.eval().asInstanceOf[Int]
+    }
+
+    // Create the FrameBoundary
+    ctx.boundType.getType match {
+      case SqlBaseParser.PRECEDING if ctx.UNBOUNDED != null =>
+        UnboundedPreceding
+      case SqlBaseParser.PRECEDING =>
+        ValuePreceding(value)
+      case SqlBaseParser.CURRENT =>
+        CurrentRow
+      case SqlBaseParser.FOLLOWING if ctx.UNBOUNDED != null =>
+        UnboundedFollowing
+      case SqlBaseParser.FOLLOWING =>
+        ValueFollowing(value)
+    }
+  }
+
+  /**
+   * Create a [[CreateStruct]] expression.
+   */
+  override def visitRowConstructor(ctx: RowConstructorContext): Expression = withOrigin(ctx) {
+    CreateStruct(ctx.expression.asScala.map(expression))
+  }
+
+  /**
+   * Create a [[ScalarSubquery]] expression.
+   */
+  override def visitSubqueryExpression(
+      ctx: SubqueryExpressionContext): Expression = withOrigin(ctx) {
+    ScalarSubquery(plan(ctx.query))
+  }
+
+  /**
+   * Create a value based [[CaseWhen]] expression. This has the following SQL form:
+   * {{{
+   *   CASE [expression]
+   *    WHEN [value] THEN [expression]
+   *    ...
+   *    ELSE [expression]
+   *   END
+   * }}}
+   */
+  override def visitSimpleCase(ctx: SimpleCaseContext): Expression = withOrigin(ctx) {
+    val e = expression(ctx.valueExpression)
+    val branches = ctx.whenClause.asScala.map { wCtx =>
+      (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result))
+    }
+    CaseWhen(branches, Option(ctx.elseExpression).map(expression))
+  }
+
+  /**
+   * Create a condition based [[CaseWhen]] expression. This has the following SQL syntax:
+   * {{{
+   *   CASE
+   *    WHEN [predicate] THEN [expression]
+   *    ...
+   *    ELSE [expression]
+   *   END
+   * }}}
+   *
+   * @param ctx the parse tree
+   *    */
+  override def visitSearchedCase(ctx: SearchedCaseContext): Expression = withOrigin(ctx) {
+    val branches = ctx.whenClause.asScala.map { wCtx =>
+      (expression(wCtx.condition), expression(wCtx.result))
+    }
+    CaseWhen(branches, Option(ctx.elseExpression).map(expression))
+  }
+
+  /**
+   * Create a dereference expression. The return type depends on the type of the parent, this can
+   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
+   * [[UnresolvedExtractValue]] if the parent is some expression.
+   */
+  override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
+    val attr = ctx.fieldName.getText
+    expression(ctx.base) match {
+      case UnresolvedAttribute(nameParts) =>
+        UnresolvedAttribute(nameParts :+ attr)
+      case e =>
+        UnresolvedExtractValue(e, Literal(attr))
+    }
+  }
+
+  /**
+   * Create an [[UnresolvedAttribute]] expression.
+   */
+  override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
+    UnresolvedAttribute.quoted(ctx.getText)
+  }
+
+  /**
+   * Create an [[UnresolvedExtractValue]] expression, this is used for subscript access to an array.
+   */
+  override def visitSubscript(ctx: SubscriptContext): Expression = withOrigin(ctx) {
+    UnresolvedExtractValue(expression(ctx.value), expression(ctx.index))
+  }
+
+  /**
+   * Create an expression for an expression between parentheses. This is need because the ANTLR
+   * visitor cannot automatically convert the nested context into an expression.
+   */
+  override def visitParenthesizedExpression(
+     ctx: ParenthesizedExpressionContext): Expression = withOrigin(ctx) {
+    expression(ctx.expression)
+  }
+
+  /**
+   * Create a [[SortOrder]] expression.
+   */
+  override def visitSortItem(ctx: SortItemContext): SortOrder = withOrigin(ctx) {
+    if (ctx.DESC != null) {
+      SortOrder(expression(ctx.expression), Descending)
+    } else {
+      SortOrder(expression(ctx.expression), Ascending)
+    }
+  }
+
+  /**
+   * Create a typed Literal expression. A typed literal has the following SQL syntax:
+   * {{{
+   *   [TYPE] '[VALUE]'
+   * }}}
+   * Currently Date and Timestamp typed literals are supported.
+   *
+   * TODO what the added value of this over casting?
+   */
+  override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) {
+    val value = string(ctx.STRING)
+    ctx.identifier.getText.toUpperCase match {
+      case "DATE" =>
+        Literal(Date.valueOf(value))
+      case "TIMESTAMP" =>
+        Literal(Timestamp.valueOf(value))
+      case other =>
+        throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx)
+    }
+  }
+
+  /**
+   * Create a NULL literal expression.
+   */
+  override def visitNullLiteral(ctx: NullLiteralContext): Literal = withOrigin(ctx) {
+    Literal(null)
+  }
+
+  /**
+   * Create a Boolean literal expression.
+   */
+  override def visitBooleanLiteral(ctx: BooleanLiteralContext): Literal = withOrigin(ctx) {
+    if (ctx.getText.toBoolean) {
+      Literal.TrueLiteral
+    } else {
+      Literal.FalseLiteral
+    }
+  }
+
+  /**
+   * Create an integral literal expression. The code selects the most narrow integral type
+   * possible, either a BigDecimal, a Long or an Integer is returned.
+   */
+  override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) {
+    BigDecimal(ctx.getText) match {
+      case v if v.isValidInt =>
+        Literal(v.intValue())
+      case v if v.isValidLong =>
+        Literal(v.longValue())
+      case v => Literal(v.underlying())
+    }
+  }
+
+  /**
+   * Create a double literal for a number denoted in scientifc notation.
+   */
+  override def visitScientificDecimalLiteral(
+      ctx: ScientificDecimalLiteralContext): Literal = withOrigin(ctx) {
+    Literal(ctx.getText.toDouble)
+  }
+
+  /**
+   * Create a decimal literal for a regular decimal number.
+   */
+  override def visitDecimalLiteral(ctx: DecimalLiteralContext): Literal = withOrigin(ctx) {
+    Literal(BigDecimal(ctx.getText).underlying())
+  }
+
+  /** Create a numeric literal expression. */
+  private def numericLiteral(ctx: NumberContext)(f: String => Any): Literal = withOrigin(ctx) {
+    val raw = ctx.getText
+    try {
+      Literal(f(raw.substring(0, raw.length - 1)))
+    } catch {
+      case e: NumberFormatException =>
+        throw new ParseException(e.getMessage, ctx)
+    }
+  }
+
+  /**
+   * Create a Byte Literal expression.
+   */
+  override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = numericLiteral(ctx) {
+    _.toByte
+  }
+
+  /**
+   * Create a Short Literal expression.
+   */
+  override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = numericLiteral(ctx) {
+    _.toShort
+  }
+
+  /**
+   * Create a Long Literal expression.
+   */
+  override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = numericLiteral(ctx) {
+    _.toLong
+  }
+
+  /**
+   * Create a Double Literal expression.
+   */
+  override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = numericLiteral(ctx) {
+    _.toDouble
+  }
+
+  /**
+   * Create a String literal expression.
+   */
+  override def visitStringLiteral(ctx: StringLiteralContext): Literal = withOrigin(ctx) {
+    Literal(createString(ctx))
+  }
+
+  /**
+   * Create a String from a string literal context. This supports multiple consecutive string
+   * literals, these are concatenated, for example this expression "'hello' 'world'" will be
+   * converted into "helloworld".
+   *
+   * Special characters can be escaped by using Hive/C-style escaping.
+   */
+  private def createString(ctx: StringLiteralContext): String = {
+    ctx.STRING().asScala.map(string).mkString
+  }
+
+  /**
+   * Create a [[CalendarInterval]] literal expression. An interval expression can contain multiple
+   * unit value pairs, for instance: interval 2 months 2 days.
+   */
+  override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) {
+    val intervals = ctx.intervalField.asScala.map(visitIntervalField)
+    assert(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx)
+    Literal(intervals.reduce(_.add(_)))
+  }
+
+  /**
+   * Create a [[CalendarInterval]] for a unit value pair. Two unit configuration types are
+   * supported:
+   * - Single unit.
+   * - From-To unit (only 'YEAR TO MONTH' and 'DAY TO SECOND' are supported).
+   */
+  override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) {
+    import ctx._
+    val s = value.getText
+    try {
+      val interval = (unit.getText.toLowerCase, Option(to).map(_.getText.toLowerCase)) match {
+        case (u, None) if u.endsWith("s") =>
+          // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/...
+          CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s)
+        case (u, None) =>
+          CalendarInterval.fromSingleUnitString(u, s)
+        case ("year", Some("month")) =>
+          CalendarInterval.fromYearMonthString(s)
+        case ("day", Some("second")) =>
+          CalendarInterval.fromDayTimeString(s)
+        case (from, Some(t)) =>
+          throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx)
+      }
+      assert(interval != null, "No interval can be constructed", ctx)
+      interval
+    } catch {
+      // Handle Exceptions thrown by CalendarInterval
+      case e: IllegalArgumentException =>
+        val pe = new ParseException(e.getMessage, ctx)
+        pe.setStackTrace(e.getStackTrace)
+        throw pe
+    }
+  }
+
+  /* ********************************************************************************************
+   * DataType parsing
+   * ******************************************************************************************** */
+  /**
+   * Resolve/create a primitive type.
+   */
+  override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) {
+    (ctx.identifier.getText.toLowerCase, ctx.INTEGER_VALUE().asScala.toList) match {
+      case ("boolean", Nil) => BooleanType
+      case ("tinyint" | "byte", Nil) => ByteType
+      case ("smallint" | "short", Nil) => ShortType
+      case ("int" | "integer", Nil) => IntegerType
+      case ("bigint" | "long", Nil) => LongType
+      case ("float", Nil) => FloatType
+      case ("double", Nil) => DoubleType
+      case ("date", Nil) => DateType
+      case ("timestamp", Nil) => TimestampType
+      case ("char" | "varchar" | "string", Nil) => StringType
+      case ("char" | "varchar", _ :: Nil) => StringType
+      case ("binary", Nil) => BinaryType
+      case ("decimal", Nil) => DecimalType.USER_DEFAULT
+      case ("decimal", precision :: Nil) => DecimalType(precision.getText.toInt, 0)
+      case ("decimal", precision :: scale :: Nil) =>
+        DecimalType(precision.getText.toInt, scale.getText.toInt)
+      case (dt, params) =>
+        throw new ParseException(
+          s"DataType $dt${params.mkString("(", ",", ")")} is not supported.", ctx)
+    }
+  }
+
+  /**
+   * Create a complex DataType. Arrays, Maps and Structures are supported.
+   */
+  override def visitComplexDataType(ctx: ComplexDataTypeContext): DataType = withOrigin(ctx) {
+    ctx.complex.getType match {
+      case SqlBaseParser.ARRAY =>
+        ArrayType(typedVisit(ctx.dataType(0)))
+      case SqlBaseParser.MAP =>
+        MapType(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1)))
+      case SqlBaseParser.STRUCT =>
+        createStructType(ctx.colTypeList())
+    }
+  }
+
+  /**
+    * Create a [[StructType]] from a sequence of [[StructField]]s.
+    */
+  protected def createStructType(ctx: ColTypeListContext): StructType = {
+    StructType(Option(ctx).toSeq.flatMap(visitColTypeList))
+  }
+
+  /**
+   * Create a [[StructType]] from a number of column definitions.
+   */
+  override def visitColTypeList(ctx: ColTypeListContext): Seq[StructField] = withOrigin(ctx) {
+    ctx.colType().asScala.map(visitColType)
+  }
+
+  /**
+   * Create a [[StructField]] from a column definition.
+   */
+  override def visitColType(ctx: ColTypeContext): StructField = withOrigin(ctx) {
+    import ctx._
+
+    // Add the comment to the metadata.
+    val builder = new MetadataBuilder
+    if (STRING != null) {
+      builder.putString("comment", string(STRING))
+    }
+
+    StructField(identifier.getText, typedVisit(dataType), nullable = true, builder.build())
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[8/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
[SPARK-14211][SQL] Remove ANTLR3 based parser

### What changes were proposed in this pull request?

This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`.

### How was this patch tested?

Existing unit tests.

cc rxin andrewor14 yhuai

Author: Herman van Hovell <hv...@questtec.nl>

Closes #12071 from hvanhovell/SPARK-14211.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a9b93e07
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a9b93e07
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a9b93e07

Branch: refs/heads/master
Commit: a9b93e07391faede77dde4c0b3c21c9b3f97f8eb
Parents: 26445c2
Author: Herman van Hovell <hv...@questtec.nl>
Authored: Thu Mar 31 09:25:09 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Thu Mar 31 09:25:09 2016 -0700

----------------------------------------------------------------------
 dev/deps/spark-deps-hadoop-2.2                  |    4 +-
 dev/deps/spark-deps-hadoop-2.3                  |    4 +-
 dev/deps/spark-deps-hadoop-2.4                  |    4 +-
 dev/deps/spark-deps-hadoop-2.6                  |    4 +-
 dev/deps/spark-deps-hadoop-2.7                  |    4 +-
 pom.xml                                         |    6 -
 project/SparkBuild.scala                        |   54 +-
 project/plugins.sbt                             |    3 -
 python/pyspark/sql/utils.py                     |    2 +-
 sql/catalyst/pom.xml                            |   22 -
 .../sql/catalyst/parser/ExpressionParser.g      |  400 ---
 .../sql/catalyst/parser/FromClauseParser.g      |  341 ---
 .../sql/catalyst/parser/IdentifiersParser.g     |  184 --
 .../spark/sql/catalyst/parser/KeywordParser.g   |  244 --
 .../sql/catalyst/parser/SelectClauseParser.g    |  235 --
 .../spark/sql/catalyst/parser/SparkSqlLexer.g   |  491 ----
 .../spark/sql/catalyst/parser/SparkSqlParser.g  | 2596 ------------------
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |  943 +++++++
 .../spark/sql/catalyst/parser/ng/SqlBase.g4     |  941 -------
 .../spark/sql/catalyst/parser/ASTNode.scala     |   99 -
 .../parser/AbstractSparkSQLParser.scala         |  145 -
 .../spark/sql/catalyst/parser/AstBuilder.scala  | 1460 ++++++++++
 .../spark/sql/catalyst/parser/CatalystQl.scala  |  933 -------
 .../sql/catalyst/parser/DataTypeParser.scala    |   67 +
 .../spark/sql/catalyst/parser/ParseDriver.scala |  245 +-
 .../spark/sql/catalyst/parser/ParserConf.scala  |   26 -
 .../spark/sql/catalyst/parser/ParserUtils.scala |  209 +-
 .../sql/catalyst/parser/ng/AstBuilder.scala     | 1452 ----------
 .../sql/catalyst/parser/ng/ParseDriver.scala    |  240 --
 .../sql/catalyst/parser/ng/ParserUtils.scala    |  118 -
 .../sql/catalyst/parser/ASTNodeSuite.scala      |   38 -
 .../sql/catalyst/parser/CatalystQlSuite.scala   |  223 --
 .../catalyst/parser/DataTypeParserSuite.scala   |    1 -
 .../sql/catalyst/parser/ErrorParserSuite.scala  |   67 +
 .../catalyst/parser/ExpressionParserSuite.scala |  497 ++++
 .../sql/catalyst/parser/PlanParserSuite.scala   |  429 +++
 .../parser/TableIdentifierParserSuite.scala     |   42 +
 .../catalyst/parser/ng/ErrorParserSuite.scala   |   67 -
 .../parser/ng/ExpressionParserSuite.scala       |  497 ----
 .../catalyst/parser/ng/PlanParserSuite.scala    |  429 ---
 .../parser/ng/TableIdentifierParserSuite.scala  |   42 -
 .../apache/spark/sql/execution/SparkQl.scala    |  387 ---
 .../spark/sql/execution/SparkSqlParser.scala    |    6 +-
 .../command/AlterTableCommandParser.scala       |  431 ---
 .../org/apache/spark/sql/internal/SQLConf.scala |   20 +-
 .../spark/sql/execution/command/DDLSuite.scala  |   13 +-
 sql/hive/pom.xml                                |   19 -
 .../spark/sql/hive/HiveMetastoreCatalog.scala   |   27 +-
 .../org/apache/spark/sql/hive/HiveQl.scala      |  749 -----
 .../sql/hive/execution/HiveSqlParser.scala      |   31 +-
 .../spark/sql/hive/ErrorPositionSuite.scala     |    4 +-
 .../org/apache/spark/sql/hive/HiveQlSuite.scala |    4 +-
 .../apache/spark/sql/hive/StatisticsSuite.scala |    5 +-
 53 files changed, 3816 insertions(+), 11688 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/dev/deps/spark-deps-hadoop-2.2
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index 7c2f88b..0c4e43b 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -2,7 +2,8 @@ JavaEWAH-0.3.2.jar
 RoaringBitmap-0.5.11.jar
 ST4-4.0.4.jar
 activation-1.1.jar
-antlr-runtime-3.5.2.jar
+antlr-2.7.7.jar
+antlr-runtime-3.4.jar
 antlr4-runtime-4.5.2-1.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
@@ -173,6 +174,7 @@ spire_2.11-0.7.4.jar
 stax-api-1.0-2.jar
 stax-api-1.0.1.jar
 stream-2.7.0.jar
+stringtemplate-3.2.1.jar
 super-csv-2.2.0.jar
 univocity-parsers-1.5.6.jar
 unused-1.0.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/dev/deps/spark-deps-hadoop-2.3
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index f4d6000..a0d62a1 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -2,7 +2,8 @@ JavaEWAH-0.3.2.jar
 RoaringBitmap-0.5.11.jar
 ST4-4.0.4.jar
 activation-1.1.1.jar
-antlr-runtime-3.5.2.jar
+antlr-2.7.7.jar
+antlr-runtime-3.4.jar
 antlr4-runtime-4.5.2-1.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
@@ -164,6 +165,7 @@ spire_2.11-0.7.4.jar
 stax-api-1.0-2.jar
 stax-api-1.0.1.jar
 stream-2.7.0.jar
+stringtemplate-3.2.1.jar
 super-csv-2.2.0.jar
 univocity-parsers-1.5.6.jar
 unused-1.0.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/dev/deps/spark-deps-hadoop-2.4
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index 7c5e2c3..cc6e403 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -2,7 +2,8 @@ JavaEWAH-0.3.2.jar
 RoaringBitmap-0.5.11.jar
 ST4-4.0.4.jar
 activation-1.1.1.jar
-antlr-runtime-3.5.2.jar
+antlr-2.7.7.jar
+antlr-runtime-3.4.jar
 antlr4-runtime-4.5.2-1.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
@@ -165,6 +166,7 @@ spire_2.11-0.7.4.jar
 stax-api-1.0-2.jar
 stax-api-1.0.1.jar
 stream-2.7.0.jar
+stringtemplate-3.2.1.jar
 super-csv-2.2.0.jar
 univocity-parsers-1.5.6.jar
 unused-1.0.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/dev/deps/spark-deps-hadoop-2.6
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index 03d9a51..5c93db5 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -2,7 +2,8 @@ JavaEWAH-0.3.2.jar
 RoaringBitmap-0.5.11.jar
 ST4-4.0.4.jar
 activation-1.1.1.jar
-antlr-runtime-3.5.2.jar
+antlr-2.7.7.jar
+antlr-runtime-3.4.jar
 antlr4-runtime-4.5.2-1.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
@@ -171,6 +172,7 @@ spire_2.11-0.7.4.jar
 stax-api-1.0-2.jar
 stax-api-1.0.1.jar
 stream-2.7.0.jar
+stringtemplate-3.2.1.jar
 super-csv-2.2.0.jar
 univocity-parsers-1.5.6.jar
 unused-1.0.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/dev/deps/spark-deps-hadoop-2.7
----------------------------------------------------------------------
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index 5765071..860fd79 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -2,7 +2,8 @@ JavaEWAH-0.3.2.jar
 RoaringBitmap-0.5.11.jar
 ST4-4.0.4.jar
 activation-1.1.1.jar
-antlr-runtime-3.5.2.jar
+antlr-2.7.7.jar
+antlr-runtime-3.4.jar
 antlr4-runtime-4.5.2-1.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
@@ -172,6 +173,7 @@ spire_2.11-0.7.4.jar
 stax-api-1.0-2.jar
 stax-api-1.0.1.jar
 stream-2.7.0.jar
+stringtemplate-3.2.1.jar
 super-csv-2.2.0.jar
 univocity-parsers-1.5.6.jar
 unused-1.0.0.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1513a18..9dab0bc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -177,7 +177,6 @@
     <jodd.version>3.5.2</jodd.version>
     <jsr305.version>1.3.9</jsr305.version>
     <libthrift.version>0.9.2</libthrift.version>
-    <antlr.version>3.5.2</antlr.version>
     <antlr4.version>4.5.2-1</antlr4.version>
 
     <test.java.home>${java.home}</test.java.home>
@@ -1757,11 +1756,6 @@
       </dependency>
       <dependency>
         <groupId>org.antlr</groupId>
-        <artifactId>antlr-runtime</artifactId>
-        <version>${antlr.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.antlr</groupId>
         <artifactId>antlr4-runtime</artifactId>
         <version>${antlr4.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 39a9e16..5d62b68 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -403,59 +403,9 @@ object OldDeps {
 
 object Catalyst {
   lazy val settings = antlr4Settings ++ Seq(
-    antlr4PackageName in Antlr4 := Some("org.apache.spark.sql.catalyst.parser.ng"),
+    antlr4PackageName in Antlr4 := Some("org.apache.spark.sql.catalyst.parser"),
     antlr4GenListener in Antlr4 := true,
-    antlr4GenVisitor in Antlr4 := true,
-    // ANTLR code-generation step.
-    //
-    // This has been heavily inspired by com.github.stefri.sbt-antlr (0.5.3). It fixes a number of
-    // build errors in the current plugin.
-    // Create Parser from ANTLR grammar files.
-    sourceGenerators in Compile += Def.task {
-      val log = streams.value.log
-
-      val grammarFileNames = Seq(
-        "SparkSqlLexer.g",
-        "SparkSqlParser.g")
-      val sourceDir = (sourceDirectory in Compile).value / "antlr3"
-      val targetDir = (sourceManaged in Compile).value / "antlr3"
-
-      // Create default ANTLR Tool.
-      val antlr = new org.antlr.Tool
-
-      // Setup input and output directories.
-      antlr.setInputDirectory(sourceDir.getPath)
-      antlr.setOutputDirectory(targetDir.getPath)
-      antlr.setForceRelativeOutput(true)
-      antlr.setMake(true)
-
-      // Add grammar files.
-      grammarFileNames.flatMap(gFileName => (sourceDir ** gFileName).get).foreach { gFilePath =>
-        val relGFilePath = (gFilePath relativeTo sourceDir).get.getPath
-        log.info("ANTLR: Grammar file '%s' detected.".format(relGFilePath))
-        antlr.addGrammarFile(relGFilePath)
-        // We will set library directory multiple times here. However, only the
-        // last one has effect. Because the grammar files are located under the same directory,
-        // We assume there is only one library directory.
-        antlr.setLibDirectory(gFilePath.getParent)
-      }
-
-      // Generate the parser.
-      antlr.process()
-      val errorState = org.antlr.tool.ErrorManager.getErrorState
-      if (errorState.errors > 0) {
-        sys.error("ANTLR: Caught %d build errors.".format(errorState.errors))
-      } else if (errorState.warnings > 0) {
-        sys.error("ANTLR: Caught %d build warnings.".format(errorState.warnings))
-      }
-
-      // Return all generated java files.
-      (targetDir ** "*.java").get.toSeq
-    }.taskValue,
-    // Include ANTLR tokens files.
-    resourceGenerators in Compile += Def.task {
-      ((sourceManaged in Compile).value ** "*.tokens").get.toSeq
-    }.taskValue
+    antlr4GenVisitor in Antlr4 := true
   )
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/project/plugins.sbt
----------------------------------------------------------------------
diff --git a/project/plugins.sbt b/project/plugins.sbt
index d9ed796..4929ba3 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -22,9 +22,6 @@ libraryDependencies += "org.ow2.asm"  % "asm" % "5.0.3"
 
 libraryDependencies += "org.ow2.asm"  % "asm-commons" % "5.0.3"
 
-libraryDependencies += "org.antlr" % "antlr" % "3.5.2"
-
-
 // TODO I am not sure we want such a dep.
 resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases"
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/python/pyspark/sql/utils.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py
index b89ea8c..7ea0e0d 100644
--- a/python/pyspark/sql/utils.py
+++ b/python/pyspark/sql/utils.py
@@ -55,7 +55,7 @@ def capture_sql_exception(f):
                                              e.java_exception.getStackTrace()))
             if s.startswith('org.apache.spark.sql.AnalysisException: '):
                 raise AnalysisException(s.split(': ', 1)[1], stackTrace)
-            if s.startswith('org.apache.spark.sql.catalyst.parser.ng.ParseException: '):
+            if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '):
                 raise ParseException(s.split(': ', 1)[1], stackTrace)
             if s.startswith('java.lang.IllegalArgumentException: '):
                 raise IllegalArgumentException(s.split(': ', 1)[1], stackTrace)

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/pom.xml
----------------------------------------------------------------------
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 9bfe495..1748fa2 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -73,10 +73,6 @@
     </dependency>
     <dependency>
       <groupId>org.antlr</groupId>
-      <artifactId>antlr-runtime</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.antlr</groupId>
       <artifactId>antlr4-runtime</artifactId>
     </dependency>
     <dependency>
@@ -117,24 +113,6 @@
       </plugin>
       <plugin>
         <groupId>org.antlr</groupId>
-        <artifactId>antlr3-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>antlr</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <sourceDirectory>../catalyst/src/main/antlr3</sourceDirectory>
-          <includes>
-            <include>**/SparkSqlLexer.g</include>
-            <include>**/SparkSqlParser.g</include>
-          </includes>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.antlr</groupId>
         <artifactId>antlr4-maven-plugin</artifactId>
         <executions>
           <execution>

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g
deleted file mode 100644
index 13a6a2d..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g
+++ /dev/null
@@ -1,400 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar.
-*/
-
-parser grammar ExpressionParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-// fun(par1, par2, par3)
-function
-@init { gParent.pushMsg("function specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    functionName
-    LPAREN
-      (
-        (STAR) => (star=STAR)
-        | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)?
-      )
-    RPAREN (KW_OVER ws=window_specification)?
-           -> {$star != null}? ^(TOK_FUNCTIONSTAR functionName $ws?)
-           -> {$dist == null}? ^(TOK_FUNCTION functionName (selectExpression+)? $ws?)
-                            -> ^(TOK_FUNCTIONDI functionName (selectExpression+)? $ws?)
-    ;
-
-functionName
-@init { gParent.pushMsg("function name", state); }
-@after { gParent.popMsg(state); }
-    : // Keyword IF is also a function name
-    (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) => (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE)
-    |
-    (functionIdentifier) => functionIdentifier
-    |
-    {!useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsCastFunctionName -> Identifier[$sql11ReservedKeywordsUsedAsCastFunctionName.text]
-    ;
-
-castExpression
-@init { gParent.pushMsg("cast expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CAST
-    LPAREN
-          expression
-          KW_AS
-          primitiveType
-    RPAREN -> ^(TOK_FUNCTION primitiveType expression)
-    ;
-
-caseExpression
-@init { gParent.pushMsg("case expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CASE expression
-    (KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END -> ^(TOK_FUNCTION KW_CASE expression*)
-    ;
-
-whenExpression
-@init { gParent.pushMsg("case expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CASE
-     ( KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END -> ^(TOK_FUNCTION KW_WHEN expression*)
-    ;
-
-constant
-@init { gParent.pushMsg("constant", state); }
-@after { gParent.popMsg(state); }
-    :
-    Number
-    | dateLiteral
-    | timestampLiteral
-    | intervalLiteral
-    | StringLiteral
-    | stringLiteralSequence
-    | BigintLiteral
-    | SmallintLiteral
-    | TinyintLiteral
-    | DoubleLiteral
-    | booleanValue
-    ;
-
-stringLiteralSequence
-    :
-    StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+)
-    ;
-
-dateLiteral
-    :
-    KW_DATE StringLiteral ->
-    {
-      // Create DateLiteral token, but with the text of the string value
-      // This makes the dateLiteral more consistent with the other type literals.
-      adaptor.create(TOK_DATELITERAL, $StringLiteral.text)
-    }
-    |
-    KW_CURRENT_DATE -> ^(TOK_FUNCTION KW_CURRENT_DATE)
-    ;
-
-timestampLiteral
-    :
-    KW_TIMESTAMP StringLiteral ->
-    {
-      adaptor.create(TOK_TIMESTAMPLITERAL, $StringLiteral.text)
-    }
-    |
-    KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP)
-    ;
-
-intervalLiteral
-    :
-    (KW_INTERVAL intervalConstant KW_YEAR KW_TO KW_MONTH) => KW_INTERVAL intervalConstant KW_YEAR KW_TO KW_MONTH
-      -> ^(TOK_INTERVAL_YEAR_MONTH_LITERAL intervalConstant)
-    | (KW_INTERVAL intervalConstant KW_DAY KW_TO KW_SECOND) => KW_INTERVAL intervalConstant KW_DAY KW_TO KW_SECOND
-      -> ^(TOK_INTERVAL_DAY_TIME_LITERAL intervalConstant)
-    | KW_INTERVAL
-      ((intervalConstant KW_YEAR)=> year=intervalConstant KW_YEAR)?
-      ((intervalConstant KW_MONTH)=> month=intervalConstant KW_MONTH)?
-      ((intervalConstant KW_WEEK)=> week=intervalConstant KW_WEEK)?
-      ((intervalConstant KW_DAY)=> day=intervalConstant KW_DAY)?
-      ((intervalConstant KW_HOUR)=> hour=intervalConstant KW_HOUR)?
-      ((intervalConstant KW_MINUTE)=> minute=intervalConstant KW_MINUTE)?
-      ((intervalConstant KW_SECOND)=> second=intervalConstant KW_SECOND)?
-      ((intervalConstant KW_MILLISECOND)=> millisecond=intervalConstant KW_MILLISECOND)?
-      ((intervalConstant KW_MICROSECOND)=> microsecond=intervalConstant KW_MICROSECOND)?
-      -> ^(TOK_INTERVAL
-          ^(TOK_INTERVAL_YEAR_LITERAL $year?)
-          ^(TOK_INTERVAL_MONTH_LITERAL $month?)
-          ^(TOK_INTERVAL_WEEK_LITERAL $week?)
-          ^(TOK_INTERVAL_DAY_LITERAL $day?)
-          ^(TOK_INTERVAL_HOUR_LITERAL $hour?)
-          ^(TOK_INTERVAL_MINUTE_LITERAL $minute?)
-          ^(TOK_INTERVAL_SECOND_LITERAL $second?)
-          ^(TOK_INTERVAL_MILLISECOND_LITERAL $millisecond?)
-          ^(TOK_INTERVAL_MICROSECOND_LITERAL $microsecond?))
-    ;
-
-intervalConstant
-    :
-    sign=(MINUS|PLUS)? value=Number -> {
-      adaptor.create(Number, ($sign != null ? $sign.getText() : "") + $value.getText())
-    }
-    | StringLiteral
-    ;
-
-expression
-@init { gParent.pushMsg("expression specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    precedenceOrExpression
-    ;
-
-atomExpression
-    :
-    (KW_NULL) => KW_NULL -> TOK_NULL
-    | (constant) => constant
-    | castExpression
-    | caseExpression
-    | whenExpression
-    | (functionName LPAREN) => function
-    | tableOrColumn
-    | (LPAREN KW_SELECT) => subQueryExpression
-      -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP) subQueryExpression)
-    | LPAREN! expression RPAREN!
-    ;
-
-
-precedenceFieldExpression
-    :
-    atomExpression ((LSQUARE^ expression RSQUARE!) | (DOT^ identifier))*
-    ;
-
-precedenceUnaryOperator
-    :
-    PLUS | MINUS | TILDE
-    ;
-
-nullCondition
-    :
-    KW_NULL -> ^(TOK_ISNULL)
-    | KW_NOT KW_NULL -> ^(TOK_ISNOTNULL)
-    ;
-
-precedenceUnaryPrefixExpression
-    :
-    (precedenceUnaryOperator+)=> precedenceUnaryOperator^ precedenceUnaryPrefixExpression
-    | precedenceFieldExpression
-    ;
-
-precedenceUnarySuffixExpression
-    :
-    (
-    (LPAREN precedenceUnaryPrefixExpression RPAREN) => LPAREN precedenceUnaryPrefixExpression (a=KW_IS nullCondition)? RPAREN
-    |
-    precedenceUnaryPrefixExpression (a=KW_IS nullCondition)?
-    )
-    -> {$a != null}? ^(TOK_FUNCTION nullCondition precedenceUnaryPrefixExpression)
-    -> precedenceUnaryPrefixExpression
-    ;
-
-
-precedenceBitwiseXorOperator
-    :
-    BITWISEXOR
-    ;
-
-precedenceBitwiseXorExpression
-    :
-    precedenceUnarySuffixExpression (precedenceBitwiseXorOperator^ precedenceUnarySuffixExpression)*
-    ;
-
-
-precedenceStarOperator
-    :
-    STAR | DIVIDE | MOD | DIV
-    ;
-
-precedenceStarExpression
-    :
-    precedenceBitwiseXorExpression (precedenceStarOperator^ precedenceBitwiseXorExpression)*
-    ;
-
-
-precedencePlusOperator
-    :
-    PLUS | MINUS
-    ;
-
-precedencePlusExpression
-    :
-    precedenceStarExpression (precedencePlusOperator^ precedenceStarExpression)*
-    ;
-
-
-precedenceAmpersandOperator
-    :
-    AMPERSAND
-    ;
-
-precedenceAmpersandExpression
-    :
-    precedencePlusExpression (precedenceAmpersandOperator^ precedencePlusExpression)*
-    ;
-
-
-precedenceBitwiseOrOperator
-    :
-    BITWISEOR
-    ;
-
-precedenceBitwiseOrExpression
-    :
-    precedenceAmpersandExpression (precedenceBitwiseOrOperator^ precedenceAmpersandExpression)*
-    ;
-
-
-// Equal operators supporting NOT prefix
-precedenceEqualNegatableOperator
-    :
-    KW_LIKE | KW_RLIKE | KW_REGEXP
-    ;
-
-precedenceEqualOperator
-    :
-    precedenceEqualNegatableOperator | EQUAL | EQUAL_NS | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
-    ;
-
-subQueryExpression
-    :
-    LPAREN! selectStatement[true] RPAREN!
-    ;
-
-precedenceEqualExpression
-    :
-    (LPAREN precedenceBitwiseOrExpression COMMA) => precedenceEqualExpressionMutiple
-    |
-    precedenceEqualExpressionSingle
-    ;
-
-precedenceEqualExpressionSingle
-    :
-    (left=precedenceBitwiseOrExpression -> $left)
-    (
-       (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression)
-       -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpressionSingle $notExpr))
-    | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
-       -> ^(precedenceEqualOperator $precedenceEqualExpressionSingle $equalExpr)
-    | (KW_NOT KW_IN LPAREN KW_SELECT)=>  (KW_NOT KW_IN subQueryExpression)
-       -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle))
-    | (KW_NOT KW_IN expressions)
-       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions))
-    | (KW_IN LPAREN KW_SELECT)=>  (KW_IN subQueryExpression)
-       -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle)
-    | (KW_IN expressions)
-       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions)
-    | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
-       -> ^(TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max)
-    | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
-       -> ^(TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max)
-    )*
-    | (KW_EXISTS LPAREN KW_SELECT)=> (KW_EXISTS subQueryExpression) -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_EXISTS) subQueryExpression)
-    ;
-
-expressions
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> expression+
-    ;
-
-//we transform the (col0, col1) in ((v00,v01),(v10,v11)) into struct(col0, col1) in (struct(v00,v01),struct(v10,v11))
-precedenceEqualExpressionMutiple
-    :
-    (LPAREN precedenceBitwiseOrExpression (COMMA precedenceBitwiseOrExpression)+ RPAREN -> ^(TOK_FUNCTION Identifier["struct"] precedenceBitwiseOrExpression+))
-    ( (KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
-       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)
-    | (KW_NOT KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
-       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)))
-    ;
-
-expressionsToStruct
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_FUNCTION Identifier["struct"] expression+)
-    ;
-
-precedenceNotOperator
-    :
-    KW_NOT
-    ;
-
-precedenceNotExpression
-    :
-    (precedenceNotOperator^)* precedenceEqualExpression
-    ;
-
-
-precedenceAndOperator
-    :
-    KW_AND
-    ;
-
-precedenceAndExpression
-    :
-    precedenceNotExpression (precedenceAndOperator^ precedenceNotExpression)*
-    ;
-
-
-precedenceOrOperator
-    :
-    KW_OR
-    ;
-
-precedenceOrExpression
-    :
-    precedenceAndExpression (precedenceOrOperator^ precedenceAndExpression)*
-    ;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g
deleted file mode 100644
index 1bf461c..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g
+++ /dev/null
@@ -1,341 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/FromClauseParser.g grammar.
-*/
-parser grammar FromClauseParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//-----------------------------------------------------------------------------------
-
-tableAllColumns
-    : STAR
-        -> ^(TOK_ALLCOLREF)
-    | tableName DOT STAR
-        -> ^(TOK_ALLCOLREF tableName)
-    ;
-
-// (table|column)
-tableOrColumn
-@init { gParent.pushMsg("table or column identifier", state); }
-@after { gParent.popMsg(state); }
-    :
-    identifier -> ^(TOK_TABLE_OR_COL identifier)
-    ;
-
-expressionList
-@init { gParent.pushMsg("expression list", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression (COMMA expression)* -> ^(TOK_EXPLIST expression+)
-    ;
-
-aliasList
-@init { gParent.pushMsg("alias list", state); }
-@after { gParent.popMsg(state); }
-    :
-    identifier (COMMA identifier)* -> ^(TOK_ALIASLIST identifier+)
-    ;
-
-//----------------------- Rules for parsing fromClause ------------------------------
-// from [col1, col2, col3] table1, [col4, col5] table2
-fromClause
-@init { gParent.pushMsg("from clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_FROM joinSource -> ^(TOK_FROM joinSource)
-    ;
-
-joinSource
-@init { gParent.pushMsg("join source", state); }
-@after { gParent.popMsg(state); }
-    : fromSource ( joinToken^ fromSource ( joinCond {$joinToken.start.getType() != COMMA}? )? )*
-    | uniqueJoinToken^ uniqueJoinSource (COMMA! uniqueJoinSource)+
-    ;
-
-joinCond
-@init { gParent.pushMsg("join expression list", state); }
-@after { gParent.popMsg(state); }
-    : KW_ON! expression
-    | KW_USING LPAREN columnNameList RPAREN -> ^(TOK_USING columnNameList)
-    ;
-
-uniqueJoinSource
-@init { gParent.pushMsg("unique join source", state); }
-@after { gParent.popMsg(state); }
-    : KW_PRESERVE? fromSource uniqueJoinExpr
-    ;
-
-uniqueJoinExpr
-@init { gParent.pushMsg("unique join expression list", state); }
-@after { gParent.popMsg(state); }
-    : LPAREN e1+=expression (COMMA e1+=expression)* RPAREN
-      -> ^(TOK_EXPLIST $e1*)
-    ;
-
-uniqueJoinToken
-@init { gParent.pushMsg("unique join", state); }
-@after { gParent.popMsg(state); }
-    : KW_UNIQUEJOIN -> TOK_UNIQUEJOIN;
-
-joinToken
-@init { gParent.pushMsg("join type specifier", state); }
-@after { gParent.popMsg(state); }
-    :
-      KW_JOIN                                 -> TOK_JOIN
-    | KW_INNER KW_JOIN                        -> TOK_JOIN
-    | KW_NATURAL KW_JOIN                      -> TOK_NATURALJOIN
-    | KW_NATURAL KW_INNER KW_JOIN             -> TOK_NATURALJOIN
-    | COMMA                                   -> TOK_JOIN
-    | KW_CROSS KW_JOIN                        -> TOK_CROSSJOIN
-    | KW_LEFT  (KW_OUTER)? KW_JOIN            -> TOK_LEFTOUTERJOIN
-    | KW_RIGHT (KW_OUTER)? KW_JOIN            -> TOK_RIGHTOUTERJOIN
-    | KW_FULL  (KW_OUTER)? KW_JOIN            -> TOK_FULLOUTERJOIN
-    | KW_NATURAL KW_LEFT  (KW_OUTER)? KW_JOIN -> TOK_NATURALLEFTOUTERJOIN
-    | KW_NATURAL KW_RIGHT (KW_OUTER)? KW_JOIN -> TOK_NATURALRIGHTOUTERJOIN
-    | KW_NATURAL KW_FULL  (KW_OUTER)? KW_JOIN -> TOK_NATURALFULLOUTERJOIN
-    | KW_LEFT KW_SEMI KW_JOIN                 -> TOK_LEFTSEMIJOIN
-    | KW_ANTI KW_JOIN                         -> TOK_ANTIJOIN
-    ;
-
-lateralView
-@init {gParent.pushMsg("lateral view", state); }
-@after {gParent.popMsg(state); }
-	:
-	(KW_LATERAL KW_VIEW KW_OUTER) => KW_LATERAL KW_VIEW KW_OUTER function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
-	-> ^(TOK_LATERAL_VIEW_OUTER ^(TOK_SELECT ^(TOK_SELEXPR function identifier* tableAlias)))
-	|
-	KW_LATERAL KW_VIEW function tableAlias (KW_AS identifier ((COMMA)=> COMMA identifier)*)?
-	-> ^(TOK_LATERAL_VIEW ^(TOK_SELECT ^(TOK_SELEXPR function identifier* tableAlias)))
-	;
-
-tableAlias
-@init {gParent.pushMsg("table alias", state); }
-@after {gParent.popMsg(state); }
-    :
-    identifier -> ^(TOK_TABALIAS identifier)
-    ;
-
-fromSource
-@init { gParent.pushMsg("from source", state); }
-@after { gParent.popMsg(state); }
-    :
-    (LPAREN KW_VALUES) => fromSource0
-    | fromSource0
-    | (LPAREN joinSource) => LPAREN joinSource RPAREN -> joinSource
-    ;
-
-
-fromSource0
-@init { gParent.pushMsg("from source 0", state); }
-@after { gParent.popMsg(state); }
-    :
-    ((Identifier LPAREN)=> partitionedTableFunction | tableSource | subQuerySource | virtualTableSource) (lateralView^)*
-    ;
-
-tableBucketSample
-@init { gParent.pushMsg("table bucket sample specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_TABLESAMPLE LPAREN KW_BUCKET (numerator=Number) KW_OUT KW_OF (denominator=Number) (KW_ON expr+=expression (COMMA expr+=expression)*)? RPAREN -> ^(TOK_TABLEBUCKETSAMPLE $numerator $denominator $expr*)
-    ;
-
-splitSample
-@init { gParent.pushMsg("table split sample specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_TABLESAMPLE LPAREN  (numerator=Number) (percent=KW_PERCENT|KW_ROWS) RPAREN
-    -> {percent != null}? ^(TOK_TABLESPLITSAMPLE TOK_PERCENT $numerator)
-    -> ^(TOK_TABLESPLITSAMPLE TOK_ROWCOUNT $numerator)
-    |
-    KW_TABLESAMPLE LPAREN  (numerator=ByteLengthLiteral) RPAREN
-    -> ^(TOK_TABLESPLITSAMPLE TOK_LENGTH $numerator)
-    ;
-
-tableSample
-@init { gParent.pushMsg("table sample specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    tableBucketSample |
-    splitSample
-    ;
-
-tableSource
-@init { gParent.pushMsg("table source", state); }
-@after { gParent.popMsg(state); }
-    : tabname=tableName 
-    ((tableProperties) => props=tableProperties)?
-    ((tableSample) => ts=tableSample)? 
-    ((KW_AS) => (KW_AS alias=Identifier) 
-    |
-    (Identifier) => (alias=Identifier))?
-    -> ^(TOK_TABREF $tabname $props? $ts? $alias?)
-    ;
-
-tableName
-@init { gParent.pushMsg("table name", state); }
-@after { gParent.popMsg(state); }
-    :
-    id1=identifier (DOT id2=identifier)?
-    -> ^(TOK_TABNAME $id1 $id2?)
-    ;
-
-viewName
-@init { gParent.pushMsg("view name", state); }
-@after { gParent.popMsg(state); }
-    :
-    (db=identifier DOT)? view=identifier
-    -> ^(TOK_TABNAME $db? $view)
-    ;
-
-subQuerySource
-@init { gParent.pushMsg("subquery source", state); }
-@after { gParent.popMsg(state); }
-    :
-    LPAREN queryStatementExpression[false] RPAREN KW_AS? identifier -> ^(TOK_SUBQUERY queryStatementExpression identifier)
-    ;
-
-//---------------------- Rules for parsing PTF clauses -----------------------------
-partitioningSpec
-@init { gParent.pushMsg("partitioningSpec clause", state); }
-@after { gParent.popMsg(state); } 
-   :
-   partitionByClause orderByClause? -> ^(TOK_PARTITIONINGSPEC partitionByClause orderByClause?) |
-   orderByClause -> ^(TOK_PARTITIONINGSPEC orderByClause) |
-   distributeByClause sortByClause? -> ^(TOK_PARTITIONINGSPEC distributeByClause sortByClause?) |
-   sortByClause -> ^(TOK_PARTITIONINGSPEC sortByClause) |
-   clusterByClause -> ^(TOK_PARTITIONINGSPEC clusterByClause)
-   ;
-
-partitionTableFunctionSource
-@init { gParent.pushMsg("partitionTableFunctionSource clause", state); }
-@after { gParent.popMsg(state); } 
-   :
-   subQuerySource |
-   tableSource |
-   partitionedTableFunction
-   ;
-
-partitionedTableFunction
-@init { gParent.pushMsg("ptf clause", state); }
-@after { gParent.popMsg(state); } 
-   :
-   name=Identifier LPAREN KW_ON 
-   ((partitionTableFunctionSource) => (ptfsrc=partitionTableFunctionSource spec=partitioningSpec?))
-   ((Identifier LPAREN expression RPAREN ) => Identifier LPAREN expression RPAREN ( COMMA Identifier LPAREN expression RPAREN)*)?
-   ((RPAREN) => (RPAREN)) ((Identifier) => alias=Identifier)?
-   ->   ^(TOK_PTBLFUNCTION $name $alias? $ptfsrc $spec? expression*)
-   ; 
-
-//----------------------- Rules for parsing whereClause -----------------------------
-// where a=b and ...
-whereClause
-@init { gParent.pushMsg("where clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_WHERE searchCondition -> ^(TOK_WHERE searchCondition)
-    ;
-
-searchCondition
-@init { gParent.pushMsg("search condition", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression
-    ;
-
-//-----------------------------------------------------------------------------------
-
-//-------- Row Constructor ----------------------------------------------------------
-//in support of SELECT * FROM (VALUES(1,2,3),(4,5,6),...) as FOO(a,b,c) and
-// INSERT INTO <table> (col1,col2,...) VALUES(...),(...),...
-// INSERT INTO <table> (col1,col2,...) SELECT * FROM (VALUES(1,2,3),(4,5,6),...) as Foo(a,b,c)
-valueRowConstructor
-@init { gParent.pushMsg("value row constructor", state); }
-@after { gParent.popMsg(state); }
-    :
-    LPAREN precedenceUnaryPrefixExpression (COMMA precedenceUnaryPrefixExpression)* RPAREN -> ^(TOK_VALUE_ROW precedenceUnaryPrefixExpression+)
-    ;
-
-valuesTableConstructor
-@init { gParent.pushMsg("values table constructor", state); }
-@after { gParent.popMsg(state); }
-    :
-    valueRowConstructor (COMMA valueRowConstructor)* -> ^(TOK_VALUES_TABLE valueRowConstructor+)
-    ;
-
-/*
-VALUES(1),(2) means 2 rows, 1 column each.
-VALUES(1,2),(3,4) means 2 rows, 2 columns each.
-VALUES(1,2,3) means 1 row, 3 columns
-*/
-valuesClause
-@init { gParent.pushMsg("values clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_VALUES valuesTableConstructor -> valuesTableConstructor
-    ;
-
-/*
-This represents a clause like this:
-(VALUES(1,2),(2,3)) as VirtTable(col1,col2)
-*/
-virtualTableSource
-@init { gParent.pushMsg("virtual table source", state); }
-@after { gParent.popMsg(state); }
-   :
-   LPAREN valuesClause RPAREN tableNameColList -> ^(TOK_VIRTUAL_TABLE tableNameColList valuesClause)
-   ;
-/*
-e.g. as VirtTable(col1,col2)
-Note that we only want literals as column names
-*/
-tableNameColList
-@init { gParent.pushMsg("from source", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_AS? identifier LPAREN identifier (COMMA identifier)* RPAREN -> ^(TOK_VIRTUAL_TABREF ^(TOK_TABNAME identifier) ^(TOK_COL_NAME identifier+))
-    ;
-
-//-----------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
deleted file mode 100644
index 916eb6a..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
+++ /dev/null
@@ -1,184 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar.
-*/
-parser grammar IdentifiersParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//-----------------------------------------------------------------------------------
-
-// group by a,b
-groupByClause
-@init { gParent.pushMsg("group by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_GROUP KW_BY
-    expression
-    ( COMMA expression)*
-    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
-    (sets=KW_GROUPING KW_SETS 
-    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
-    -> {rollup != null}? ^(TOK_ROLLUP_GROUPBY expression+)
-    -> {cube != null}? ^(TOK_CUBE_GROUPBY expression+)
-    -> {sets != null}? ^(TOK_GROUPING_SETS expression+ groupingSetExpression+)
-    -> ^(TOK_GROUPBY expression+)
-    ;
-
-groupingSetExpression
-@init {gParent.pushMsg("grouping set expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   (LPAREN) => groupingSetExpressionMultiple 
-   |
-   groupingExpressionSingle
-   ;
-
-groupingSetExpressionMultiple
-@init {gParent.pushMsg("grouping set part expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   LPAREN 
-   expression? (COMMA expression)*
-   RPAREN
-   -> ^(TOK_GROUPING_SETS_EXPRESSION expression*)
-   ;
-
-groupingExpressionSingle
-@init { gParent.pushMsg("groupingExpression expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression -> ^(TOK_GROUPING_SETS_EXPRESSION expression)
-    ;
-
-havingClause
-@init { gParent.pushMsg("having clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_HAVING havingCondition -> ^(TOK_HAVING havingCondition)
-    ;
-
-havingCondition
-@init { gParent.pushMsg("having condition", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression
-    ;
-
-expressionsInParenthese
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> expression+
-    ;
-
-expressionsNotInParenthese
-    :
-    expression (COMMA expression)* -> expression+
-    ;
-
-columnRefOrderInParenthese
-    :
-    LPAREN columnRefOrder (COMMA columnRefOrder)* RPAREN -> columnRefOrder+
-    ;
-
-columnRefOrderNotInParenthese
-    :
-    columnRefOrder (COMMA columnRefOrder)* -> columnRefOrder+
-    ;
-    
-// order by a,b
-orderByClause
-@init { gParent.pushMsg("order by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_ORDER KW_BY columnRefOrder ( COMMA columnRefOrder)* -> ^(TOK_ORDERBY columnRefOrder+)
-    ;
-    
-clusterByClause
-@init { gParent.pushMsg("cluster by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CLUSTER KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_CLUSTERBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_CLUSTERBY expressionsNotInParenthese)
-    )
-    ;
-
-partitionByClause
-@init  { gParent.pushMsg("partition by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_PARTITION KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-distributeByClause
-@init { gParent.pushMsg("distribute by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_DISTRIBUTE KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-sortByClause
-@init { gParent.pushMsg("sort by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_SORT KW_BY
-    (
-    (LPAREN) => columnRefOrderInParenthese -> ^(TOK_SORTBY columnRefOrderInParenthese)
-    |
-    columnRefOrderNotInParenthese -> ^(TOK_SORTBY columnRefOrderNotInParenthese)
-    )
-    ;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/KeywordParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/KeywordParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/KeywordParser.g
deleted file mode 100644
index 12cd5f5..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/KeywordParser.g
+++ /dev/null
@@ -1,244 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar.
-*/
-
-parser grammar KeywordParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-booleanValue
-    :
-    KW_TRUE^ | KW_FALSE^
-    ;
-
-booleanValueTok
-   :
-   KW_TRUE -> TOK_TRUE
-   | KW_FALSE -> TOK_FALSE
-   ;
-
-tableOrPartition
-   :
-   tableName partitionSpec? -> ^(TOK_TAB tableName partitionSpec?)
-   ;
-
-partitionSpec
-    :
-    KW_PARTITION
-     LPAREN partitionVal (COMMA  partitionVal )* RPAREN -> ^(TOK_PARTSPEC partitionVal +)
-    ;
-
-partitionVal
-    :
-    identifier (EQUAL constant)? -> ^(TOK_PARTVAL identifier constant?)
-    ;
-
-dropPartitionSpec
-    :
-    KW_PARTITION
-     LPAREN dropPartitionVal (COMMA  dropPartitionVal )* RPAREN -> ^(TOK_PARTSPEC dropPartitionVal +)
-    ;
-
-dropPartitionVal
-    :
-    identifier dropPartitionOperator constant -> ^(TOK_PARTVAL identifier dropPartitionOperator constant)
-    ;
-
-dropPartitionOperator
-    :
-    EQUAL | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
-    ;
-
-sysFuncNames
-    :
-      KW_AND
-    | KW_OR
-    | KW_NOT
-    | KW_LIKE
-    | KW_IF
-    | KW_CASE
-    | KW_WHEN
-    | KW_TINYINT
-    | KW_SMALLINT
-    | KW_INT
-    | KW_BIGINT
-    | KW_FLOAT
-    | KW_DOUBLE
-    | KW_BOOLEAN
-    | KW_STRING
-    | KW_BINARY
-    | KW_ARRAY
-    | KW_MAP
-    | KW_STRUCT
-    | KW_UNIONTYPE
-    | EQUAL
-    | EQUAL_NS
-    | NOTEQUAL
-    | LESSTHANOREQUALTO
-    | LESSTHAN
-    | GREATERTHANOREQUALTO
-    | GREATERTHAN
-    | DIVIDE
-    | PLUS
-    | MINUS
-    | STAR
-    | MOD
-    | DIV
-    | AMPERSAND
-    | TILDE
-    | BITWISEOR
-    | BITWISEXOR
-    | KW_RLIKE
-    | KW_REGEXP
-    | KW_IN
-    | KW_BETWEEN
-    ;
-
-descFuncNames
-    :
-      (sysFuncNames) => sysFuncNames
-    | StringLiteral
-    | functionIdentifier
-    ;
-
-//We are allowed to use From and To in CreateTableUsing command's options (actually seems we can use any string as the option key). But we can't simply add them into nonReserved because by doing that we mess other existing rules. So we create a looseIdentifier and looseNonReserved here.
-looseIdentifier
-    :
-    Identifier
-    | looseNonReserved -> Identifier[$looseNonReserved.text]
-    // If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false,
-    // the sql11keywords in existing q tests will NOT be added back.
-    | {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text]
-    ;
-
-identifier
-    :
-    Identifier
-    | nonReserved -> Identifier[$nonReserved.text]
-    // If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false,
-    // the sql11keywords in existing q tests will NOT be added back.
-    | {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text]
-    ;
-
-functionIdentifier
-@init { gParent.pushMsg("function identifier", state); }
-@after { gParent.popMsg(state); }
-    :
-    identifier (DOT identifier)? -> identifier+
-    ;
-
-principalIdentifier
-@init { gParent.pushMsg("identifier for principal spec", state); }
-@after { gParent.popMsg(state); }
-    : identifier
-    | QuotedIdentifier
-    ;
-
-looseNonReserved
-    : nonReserved | KW_FROM | KW_TO
-    ;
-
-//The new version of nonReserved + sql11ReservedKeywordsUsedAsIdentifier = old version of nonReserved
-//Non reserved keywords are basically the keywords that can be used as identifiers.
-//All the KW_* are automatically not only keywords, but also reserved keywords.
-//That means, they can NOT be used as identifiers.
-//If you would like to use them as identifiers, put them in the nonReserved list below.
-//If you are not sure, please refer to the SQL2011 column in
-//http://www.postgresql.org/docs/9.5/static/sql-keywords-appendix.html
-nonReserved
-    :
-    KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS
-    | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS
-    | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY
-    | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY
-    | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_ELEM_TYPE
-    | KW_ENABLE | KW_ESCAPED | KW_EXCLUSIVE | KW_EXPLAIN | KW_EXPORT | KW_FIELDS | KW_FILE | KW_FILEFORMAT
-    | KW_FIRST | KW_FORMAT | KW_FORMATTED | KW_FUNCTIONS | KW_HOLD_DDLTIME | KW_HOUR | KW_IDXPROPERTIES | KW_IGNORE
-    | KW_INDEX | KW_INDEXES | KW_INPATH | KW_INPUTDRIVER | KW_INPUTFORMAT | KW_ITEMS | KW_JAR
-    | KW_KEYS | KW_KEY_TYPE | KW_LIMIT | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG
-    | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE
-    | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY
-    | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
-    | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE
-    | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED
-    | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED
-    | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED
-    | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET
-    | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR
-    | KW_WORK
-    | KW_TRANSACTION
-    | KW_WRITE
-    | KW_ISOLATION
-    | KW_LEVEL
-    | KW_SNAPSHOT
-    | KW_AUTOCOMMIT
-    | KW_ANTI
-    | KW_WEEK | KW_MILLISECOND | KW_MICROSECOND
-    | KW_CLEAR | KW_LAZY | KW_CACHE | KW_UNCACHE | KW_DFS
-;
-
-//The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers.
-sql11ReservedKeywordsUsedAsCastFunctionName
-    :
-    KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP
-    ;
-
-//The following SQL2011 reserved keywords are used as identifiers in many q tests, they may be added back due to backward compatibility.
-//We are planning to remove the following whole list after several releases.
-//Thus, please do not change the following list unless you know what to do.
-sql11ReservedKeywordsUsedAsIdentifier
-    :
-    KW_ALL | KW_ALTER | KW_ARRAY | KW_AS | KW_AUTHORIZATION | KW_BETWEEN | KW_BIGINT | KW_BINARY | KW_BOOLEAN
-    | KW_BOTH | KW_BY | KW_CREATE | KW_CUBE | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_CURSOR | KW_DATE | KW_DECIMAL | KW_DELETE | KW_DESCRIBE
-    | KW_DOUBLE | KW_DROP | KW_EXISTS | KW_EXTERNAL | KW_FALSE | KW_FETCH | KW_FLOAT | KW_FOR | KW_FULL | KW_GRANT
-    | KW_GROUP | KW_GROUPING | KW_IMPORT | KW_IN | KW_INNER | KW_INSERT | KW_INT | KW_INTERSECT | KW_INTO | KW_IS | KW_LATERAL
-    | KW_LEFT | KW_LIKE | KW_LOCAL | KW_NONE | KW_NULL | KW_OF | KW_ORDER | KW_OUT | KW_OUTER | KW_PARTITION
-    | KW_PERCENT | KW_PROCEDURE | KW_RANGE | KW_READS | KW_REVOKE | KW_RIGHT
-    | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE
-    | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH
-//The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL.
-    | KW_REGEXP | KW_RLIKE
-    ;

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
deleted file mode 100644
index f18b6ec..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
+++ /dev/null
@@ -1,235 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/SelectClauseParser.g grammar.
-*/
-parser grammar SelectClauseParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//----------------------- Rules for parsing selectClause -----------------------------
-// select a,b,c ...
-selectClause
-@init { gParent.pushMsg("select clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_SELECT hintClause? (((KW_ALL | dist=KW_DISTINCT)? selectList)
-                          | (transform=KW_TRANSFORM selectTrfmClause))
-     -> {$transform == null && $dist == null}? ^(TOK_SELECT hintClause? selectList)
-     -> {$transform == null && $dist != null}? ^(TOK_SELECTDI hintClause? selectList)
-     -> ^(TOK_SELECT hintClause? ^(TOK_SELEXPR selectTrfmClause) )
-    |
-    trfmClause  ->^(TOK_SELECT ^(TOK_SELEXPR trfmClause))
-    ;
-
-selectList
-@init { gParent.pushMsg("select list", state); }
-@after { gParent.popMsg(state); }
-    :
-    selectItem ( COMMA  selectItem )* -> selectItem+
-    ;
-
-selectTrfmClause
-@init { gParent.pushMsg("transform clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    LPAREN selectExpressionList RPAREN
-    inSerde=rowFormat inRec=recordWriter
-    KW_USING StringLiteral
-    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
-    outSerde=rowFormat outRec=recordReader
-    -> ^(TOK_TRANSFORM selectExpressionList $inSerde $inRec StringLiteral $outSerde $outRec aliasList? columnNameTypeList?)
-    ;
-
-hintClause
-@init { gParent.pushMsg("hint clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    DIVIDE STAR PLUS hintList STAR DIVIDE -> ^(TOK_HINTLIST hintList)
-    ;
-
-hintList
-@init { gParent.pushMsg("hint list", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintItem (COMMA hintItem)* -> hintItem+
-    ;
-
-hintItem
-@init { gParent.pushMsg("hint item", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintName (LPAREN hintArgs RPAREN)? -> ^(TOK_HINT hintName hintArgs?)
-    ;
-
-hintName
-@init { gParent.pushMsg("hint name", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_MAPJOIN -> TOK_MAPJOIN
-    | KW_STREAMTABLE -> TOK_STREAMTABLE
-    ;
-
-hintArgs
-@init { gParent.pushMsg("hint arguments", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintArgName (COMMA hintArgName)* -> ^(TOK_HINTARGLIST hintArgName+)
-    ;
-
-hintArgName
-@init { gParent.pushMsg("hint argument name", state); }
-@after { gParent.popMsg(state); }
-    :
-    identifier
-    ;
-
-selectItem
-@init { gParent.pushMsg("selection target", state); }
-@after { gParent.popMsg(state); }
-    :
-    (tableAllColumns) => tableAllColumns -> ^(TOK_SELEXPR tableAllColumns)
-    |
-    namedExpression
-    ;
-
-namedExpression
-@init { gParent.pushMsg("select named expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    ( expression
-      ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?
-    ) -> ^(TOK_SELEXPR expression identifier*)
-    ;
-
-trfmClause
-@init { gParent.pushMsg("transform clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    (   KW_MAP    selectExpressionList
-      | KW_REDUCE selectExpressionList )
-    inSerde=rowFormat inRec=recordWriter
-    KW_USING StringLiteral
-    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
-    outSerde=rowFormat outRec=recordReader
-    -> ^(TOK_TRANSFORM selectExpressionList $inSerde $inRec StringLiteral $outSerde $outRec aliasList? columnNameTypeList?)
-    ;
-
-selectExpression
-@init { gParent.pushMsg("select expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    (tableAllColumns) => tableAllColumns
-    |
-    expression
-    ;
-
-selectExpressionList
-@init { gParent.pushMsg("select expression list", state); }
-@after { gParent.popMsg(state); }
-    :
-    selectExpression (COMMA selectExpression)* -> ^(TOK_EXPLIST selectExpression+)
-    ;
-
-//---------------------- Rules for windowing clauses -------------------------------
-window_clause 
-@init { gParent.pushMsg("window_clause", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_WINDOW window_defn (COMMA window_defn)* -> ^(KW_WINDOW window_defn+)
-;  
-
-window_defn 
-@init { gParent.pushMsg("window_defn", state); }
-@after { gParent.popMsg(state); } 
-:
-  Identifier KW_AS window_specification -> ^(TOK_WINDOWDEF Identifier window_specification)
-;  
-
-window_specification 
-@init { gParent.pushMsg("window_specification", state); }
-@after { gParent.popMsg(state); } 
-:
-  (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) -> ^(TOK_WINDOWSPEC Identifier? partitioningSpec? window_frame?)
-;
-
-window_frame :
- window_range_expression |
- window_value_expression
-;
-
-window_range_expression 
-@init { gParent.pushMsg("window_range_expression", state); }
-@after { gParent.popMsg(state); } 
-:
- KW_ROWS sb=window_frame_start_boundary -> ^(TOK_WINDOWRANGE $sb) |
- KW_ROWS KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary -> ^(TOK_WINDOWRANGE $s $end)
-;
-
-window_value_expression 
-@init { gParent.pushMsg("window_value_expression", state); }
-@after { gParent.popMsg(state); } 
-:
- KW_RANGE sb=window_frame_start_boundary -> ^(TOK_WINDOWVALUES $sb) |
- KW_RANGE KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary -> ^(TOK_WINDOWVALUES $s $end)
-;
-
-window_frame_start_boundary 
-@init { gParent.pushMsg("windowframestartboundary", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_UNBOUNDED KW_PRECEDING  -> ^(KW_PRECEDING KW_UNBOUNDED) | 
-  KW_CURRENT KW_ROW  -> ^(KW_CURRENT) |
-  Number KW_PRECEDING -> ^(KW_PRECEDING Number)
-;
-
-window_frame_boundary 
-@init { gParent.pushMsg("windowframeboundary", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING)  -> ^($r KW_UNBOUNDED) | 
-  KW_CURRENT KW_ROW  -> ^(KW_CURRENT) |
-  Number (d=KW_PRECEDING | d=KW_FOLLOWING ) -> ^($d Number)
-;   
-

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g
deleted file mode 100644
index fd1ad59..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g
+++ /dev/null
@@ -1,491 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/HiveLexer.g grammar.
-*/
-lexer grammar SparkSqlLexer;
-
-@lexer::header {
-package org.apache.spark.sql.catalyst.parser;
-
-}
-
-@lexer::members {
-  private ParserConf parserConf;
-  private ParseErrorReporter reporter;
-  
-  public void configure(ParserConf parserConf, ParseErrorReporter reporter) {
-    this.parserConf = parserConf;
-    this.reporter = reporter;
-  }
-  
-  protected boolean allowQuotedId() {
-    if (parserConf == null) {
-      return true;
-    }
-    return parserConf.supportQuotedId();
-  }
-
-  @Override
-  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-    if (reporter != null) {
-      reporter.report(this, e, tokenNames);
-    }
-  }
-}
-
-// Keywords
-
-KW_TRUE : 'TRUE';
-KW_FALSE : 'FALSE';
-KW_ALL : 'ALL';
-KW_NONE: 'NONE';
-KW_AND : 'AND';
-KW_OR : 'OR';
-KW_NOT : 'NOT' | '!';
-KW_LIKE : 'LIKE';
-
-KW_IF : 'IF';
-KW_EXISTS : 'EXISTS';
-
-KW_ASC : 'ASC';
-KW_DESC : 'DESC';
-KW_ORDER : 'ORDER';
-KW_GROUP : 'GROUP';
-KW_BY : 'BY';
-KW_HAVING : 'HAVING';
-KW_WHERE : 'WHERE';
-KW_FROM : 'FROM';
-KW_AS : 'AS';
-KW_SELECT : 'SELECT';
-KW_DISTINCT : 'DISTINCT';
-KW_INSERT : 'INSERT';
-KW_OVERWRITE : 'OVERWRITE';
-KW_OUTER : 'OUTER';
-KW_UNIQUEJOIN : 'UNIQUEJOIN';
-KW_PRESERVE : 'PRESERVE';
-KW_JOIN : 'JOIN';
-KW_LEFT : 'LEFT';
-KW_RIGHT : 'RIGHT';
-KW_FULL : 'FULL';
-KW_ANTI : 'ANTI';
-KW_ON : 'ON';
-KW_PARTITION : 'PARTITION';
-KW_PARTITIONS : 'PARTITIONS';
-KW_TABLE: 'TABLE';
-KW_TABLES: 'TABLES';
-KW_COLUMNS: 'COLUMNS';
-KW_INDEX: 'INDEX';
-KW_INDEXES: 'INDEXES';
-KW_REBUILD: 'REBUILD';
-KW_FUNCTIONS: 'FUNCTIONS';
-KW_SHOW: 'SHOW';
-KW_MSCK: 'MSCK';
-KW_REPAIR: 'REPAIR';
-KW_DIRECTORY: 'DIRECTORY';
-KW_LOCAL: 'LOCAL';
-KW_TRANSFORM : 'TRANSFORM';
-KW_USING: 'USING';
-KW_CLUSTER: 'CLUSTER';
-KW_DISTRIBUTE: 'DISTRIBUTE';
-KW_SORT: 'SORT';
-KW_UNION: 'UNION';
-KW_EXCEPT: 'EXCEPT';
-KW_LOAD: 'LOAD';
-KW_EXPORT: 'EXPORT';
-KW_IMPORT: 'IMPORT';
-KW_REPLICATION: 'REPLICATION';
-KW_METADATA: 'METADATA';
-KW_DATA: 'DATA';
-KW_INPATH: 'INPATH';
-KW_IS: 'IS';
-KW_NULL: 'NULL';
-KW_CREATE: 'CREATE';
-KW_EXTERNAL: 'EXTERNAL';
-KW_ALTER: 'ALTER';
-KW_CHANGE: 'CHANGE';
-KW_COLUMN: 'COLUMN';
-KW_FIRST: 'FIRST';
-KW_AFTER: 'AFTER';
-KW_DESCRIBE: 'DESCRIBE';
-KW_DROP: 'DROP';
-KW_RENAME: 'RENAME';
-KW_TO: 'TO';
-KW_COMMENT: 'COMMENT';
-KW_BOOLEAN: 'BOOLEAN';
-KW_TINYINT: 'TINYINT';
-KW_SMALLINT: 'SMALLINT';
-KW_INT: 'INT';
-KW_BIGINT: 'BIGINT';
-KW_FLOAT: 'FLOAT';
-KW_DOUBLE: 'DOUBLE';
-KW_DATE: 'DATE';
-KW_DATETIME: 'DATETIME';
-KW_TIMESTAMP: 'TIMESTAMP';
-KW_INTERVAL: 'INTERVAL';
-KW_DECIMAL: 'DECIMAL';
-KW_STRING: 'STRING';
-KW_CHAR: 'CHAR';
-KW_VARCHAR: 'VARCHAR';
-KW_ARRAY: 'ARRAY';
-KW_STRUCT: 'STRUCT';
-KW_MAP: 'MAP';
-KW_UNIONTYPE: 'UNIONTYPE';
-KW_REDUCE: 'REDUCE';
-KW_PARTITIONED: 'PARTITIONED';
-KW_CLUSTERED: 'CLUSTERED';
-KW_SORTED: 'SORTED';
-KW_INTO: 'INTO';
-KW_BUCKETS: 'BUCKETS';
-KW_ROW: 'ROW';
-KW_ROWS: 'ROWS';
-KW_FORMAT: 'FORMAT';
-KW_DELIMITED: 'DELIMITED';
-KW_FIELDS: 'FIELDS';
-KW_TERMINATED: 'TERMINATED';
-KW_ESCAPED: 'ESCAPED';
-KW_COLLECTION: 'COLLECTION';
-KW_ITEMS: 'ITEMS';
-KW_KEYS: 'KEYS';
-KW_KEY_TYPE: '$KEY$';
-KW_LINES: 'LINES';
-KW_STORED: 'STORED';
-KW_FILEFORMAT: 'FILEFORMAT';
-KW_INPUTFORMAT: 'INPUTFORMAT';
-KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
-KW_INPUTDRIVER: 'INPUTDRIVER';
-KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
-KW_ENABLE: 'ENABLE';
-KW_DISABLE: 'DISABLE';
-KW_LOCATION: 'LOCATION';
-KW_TABLESAMPLE: 'TABLESAMPLE';
-KW_BUCKET: 'BUCKET';
-KW_OUT: 'OUT';
-KW_OF: 'OF';
-KW_PERCENT: 'PERCENT';
-KW_CAST: 'CAST';
-KW_ADD: 'ADD';
-KW_REPLACE: 'REPLACE';
-KW_RLIKE: 'RLIKE';
-KW_REGEXP: 'REGEXP';
-KW_TEMPORARY: 'TEMPORARY';
-KW_FUNCTION: 'FUNCTION';
-KW_MACRO: 'MACRO';
-KW_FILE: 'FILE';
-KW_JAR: 'JAR';
-KW_EXPLAIN: 'EXPLAIN';
-KW_EXTENDED: 'EXTENDED';
-KW_FORMATTED: 'FORMATTED';
-KW_PRETTY: 'PRETTY';
-KW_DEPENDENCY: 'DEPENDENCY';
-KW_LOGICAL: 'LOGICAL';
-KW_SERDE: 'SERDE';
-KW_WITH: 'WITH';
-KW_DEFERRED: 'DEFERRED';
-KW_SERDEPROPERTIES: 'SERDEPROPERTIES';
-KW_DBPROPERTIES: 'DBPROPERTIES';
-KW_LIMIT: 'LIMIT';
-KW_SET: 'SET';
-KW_UNSET: 'UNSET';
-KW_TBLPROPERTIES: 'TBLPROPERTIES';
-KW_IDXPROPERTIES: 'IDXPROPERTIES';
-KW_VALUE_TYPE: '$VALUE$';
-KW_ELEM_TYPE: '$ELEM$';
-KW_DEFINED: 'DEFINED';
-KW_CASE: 'CASE';
-KW_WHEN: 'WHEN';
-KW_THEN: 'THEN';
-KW_ELSE: 'ELSE';
-KW_END: 'END';
-KW_MAPJOIN: 'MAPJOIN';
-KW_STREAMTABLE: 'STREAMTABLE';
-KW_CLUSTERSTATUS: 'CLUSTERSTATUS';
-KW_UTC: 'UTC';
-KW_UTCTIMESTAMP: 'UTC_TMESTAMP';
-KW_LONG: 'LONG';
-KW_DELETE: 'DELETE';
-KW_PLUS: 'PLUS';
-KW_MINUS: 'MINUS';
-KW_FETCH: 'FETCH';
-KW_INTERSECT: 'INTERSECT';
-KW_VIEW: 'VIEW';
-KW_IN: 'IN';
-KW_DATABASE: 'DATABASE';
-KW_DATABASES: 'DATABASES';
-KW_MATERIALIZED: 'MATERIALIZED';
-KW_SCHEMA: 'SCHEMA';
-KW_SCHEMAS: 'SCHEMAS';
-KW_GRANT: 'GRANT';
-KW_REVOKE: 'REVOKE';
-KW_SSL: 'SSL';
-KW_UNDO: 'UNDO';
-KW_LOCK: 'LOCK';
-KW_LOCKS: 'LOCKS';
-KW_UNLOCK: 'UNLOCK';
-KW_SHARED: 'SHARED';
-KW_EXCLUSIVE: 'EXCLUSIVE';
-KW_PROCEDURE: 'PROCEDURE';
-KW_UNSIGNED: 'UNSIGNED';
-KW_WHILE: 'WHILE';
-KW_READ: 'READ';
-KW_READS: 'READS';
-KW_PURGE: 'PURGE';
-KW_RANGE: 'RANGE';
-KW_ANALYZE: 'ANALYZE';
-KW_BEFORE: 'BEFORE';
-KW_BETWEEN: 'BETWEEN';
-KW_BOTH: 'BOTH';
-KW_BINARY: 'BINARY';
-KW_CROSS: 'CROSS';
-KW_CONTINUE: 'CONTINUE';
-KW_CURSOR: 'CURSOR';
-KW_TRIGGER: 'TRIGGER';
-KW_RECORDREADER: 'RECORDREADER';
-KW_RECORDWRITER: 'RECORDWRITER';
-KW_SEMI: 'SEMI';
-KW_LATERAL: 'LATERAL';
-KW_TOUCH: 'TOUCH';
-KW_ARCHIVE: 'ARCHIVE';
-KW_UNARCHIVE: 'UNARCHIVE';
-KW_COMPUTE: 'COMPUTE';
-KW_STATISTICS: 'STATISTICS';
-KW_USE: 'USE';
-KW_OPTION: 'OPTION';
-KW_CONCATENATE: 'CONCATENATE';
-KW_SHOW_DATABASE: 'SHOW_DATABASE';
-KW_UPDATE: 'UPDATE';
-KW_RESTRICT: 'RESTRICT';
-KW_CASCADE: 'CASCADE';
-KW_SKEWED: 'SKEWED';
-KW_ROLLUP: 'ROLLUP';
-KW_CUBE: 'CUBE';
-KW_DIRECTORIES: 'DIRECTORIES';
-KW_FOR: 'FOR';
-KW_WINDOW: 'WINDOW';
-KW_UNBOUNDED: 'UNBOUNDED';
-KW_PRECEDING: 'PRECEDING';
-KW_FOLLOWING: 'FOLLOWING';
-KW_CURRENT: 'CURRENT';
-KW_CURRENT_DATE: 'CURRENT_DATE';
-KW_CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP';
-KW_LESS: 'LESS';
-KW_MORE: 'MORE';
-KW_OVER: 'OVER';
-KW_GROUPING: 'GROUPING';
-KW_SETS: 'SETS';
-KW_TRUNCATE: 'TRUNCATE';
-KW_NOSCAN: 'NOSCAN';
-KW_PARTIALSCAN: 'PARTIALSCAN';
-KW_USER: 'USER';
-KW_ROLE: 'ROLE';
-KW_ROLES: 'ROLES';
-KW_INNER: 'INNER';
-KW_EXCHANGE: 'EXCHANGE';
-KW_URI: 'URI';
-KW_SERVER : 'SERVER';
-KW_ADMIN: 'ADMIN';
-KW_OWNER: 'OWNER';
-KW_PRINCIPALS: 'PRINCIPALS';
-KW_COMPACT: 'COMPACT';
-KW_COMPACTIONS: 'COMPACTIONS';
-KW_TRANSACTIONS: 'TRANSACTIONS';
-KW_REWRITE : 'REWRITE';
-KW_AUTHORIZATION: 'AUTHORIZATION';
-KW_CONF: 'CONF';
-KW_VALUES: 'VALUES';
-KW_RELOAD: 'RELOAD';
-KW_YEAR: 'YEAR'|'YEARS';
-KW_MONTH: 'MONTH'|'MONTHS';
-KW_DAY: 'DAY'|'DAYS';
-KW_HOUR: 'HOUR'|'HOURS';
-KW_MINUTE: 'MINUTE'|'MINUTES';
-KW_SECOND: 'SECOND'|'SECONDS';
-KW_START: 'START';
-KW_TRANSACTION: 'TRANSACTION';
-KW_COMMIT: 'COMMIT';
-KW_ROLLBACK: 'ROLLBACK';
-KW_WORK: 'WORK';
-KW_ONLY: 'ONLY';
-KW_WRITE: 'WRITE';
-KW_ISOLATION: 'ISOLATION';
-KW_LEVEL: 'LEVEL';
-KW_SNAPSHOT: 'SNAPSHOT';
-KW_AUTOCOMMIT: 'AUTOCOMMIT';
-KW_REFRESH: 'REFRESH';
-KW_OPTIONS: 'OPTIONS';
-KW_WEEK: 'WEEK'|'WEEKS';
-KW_MILLISECOND: 'MILLISECOND'|'MILLISECONDS';
-KW_MICROSECOND: 'MICROSECOND'|'MICROSECONDS';
-KW_CLEAR: 'CLEAR';
-KW_LAZY: 'LAZY';
-KW_CACHE: 'CACHE';
-KW_UNCACHE: 'UNCACHE';
-KW_DFS: 'DFS';
-
-KW_NATURAL: 'NATURAL';
-
-// Operators
-// NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.
-
-DOT : '.'; // generated as a part of Number rule
-COLON : ':' ;
-COMMA : ',' ;
-SEMICOLON : ';' ;
-
-LPAREN : '(' ;
-RPAREN : ')' ;
-LSQUARE : '[' ;
-RSQUARE : ']' ;
-LCURLY : '{';
-RCURLY : '}';
-
-EQUAL : '=' | '==';
-EQUAL_NS : '<=>';
-NOTEQUAL : '<>' | '!=';
-LESSTHANOREQUALTO : '<=';
-LESSTHAN : '<';
-GREATERTHANOREQUALTO : '>=';
-GREATERTHAN : '>';
-
-DIVIDE : '/';
-PLUS : '+';
-MINUS : '-';
-STAR : '*';
-MOD : '%';
-DIV : 'DIV';
-
-AMPERSAND : '&';
-TILDE : '~';
-BITWISEOR : '|';
-BITWISEXOR : '^';
-QUESTION : '?';
-DOLLAR : '$';
-
-// LITERALS
-fragment
-Letter
-    : 'a'..'z' | 'A'..'Z'
-    ;
-
-fragment
-HexDigit
-    : 'a'..'f' | 'A'..'F'
-    ;
-
-fragment
-Digit
-    :
-    '0'..'9'
-    ;
-
-fragment
-Exponent
-    :
-    ('e' | 'E') ( PLUS|MINUS )? (Digit)+
-    ;
-
-fragment
-RegexComponent
-    : 'a'..'z' | 'A'..'Z' | '0'..'9' | '_'
-    | PLUS | STAR | QUESTION | MINUS | DOT
-    | LPAREN | RPAREN | LSQUARE | RSQUARE | LCURLY | RCURLY
-    | BITWISEXOR | BITWISEOR | DOLLAR | '!'
-    ;
-
-StringLiteral
-    :
-    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
-    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
-    )+
-    ;
-
-BigintLiteral
-    :
-    (Digit)+ 'L'
-    ;
-
-SmallintLiteral
-    :
-    (Digit)+ 'S'
-    ;
-
-TinyintLiteral
-    :
-    (Digit)+ 'Y'
-    ;
-
-DoubleLiteral
-    :
-    Number 'D'
-    ;
-
-ByteLengthLiteral
-    :
-    (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
-    ;
-
-Number
-    :
-    ((Digit+ (DOT Digit*)?) | (DOT Digit+)) Exponent?
-    ;
-
-/*
-An Identifier can be:
-- tableName
-- columnName
-- select expr alias
-- lateral view aliases
-- database name
-- view name
-- subquery alias
-- function name
-- ptf argument identifier
-- index name
-- property name for: db,tbl,partition...
-- fileFormat
-- role name
-- privilege name
-- principal name
-- macro name
-- hint name
-- window name
-*/
-Identifier
-    :
-    (Letter | Digit | '_')+
-    | {allowQuotedId()}? QuotedIdentifier  /* though at the language level we allow all Identifiers to be QuotedIdentifiers; 
-                                              at the API level only columns are allowed to be of this form */
-    | '`' RegexComponent+ '`'
-    ;
-
-fragment    
-QuotedIdentifier 
-    :
-    '`'  ( '``' | ~('`') )* '`' { setText(getText().replaceAll("``", "`")); }
-    ;
-
-WS  :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}
-    ;
-
-COMMENT
-  : '--' (~('\n'|'\r'))*
-    { $channel=HIDDEN; }
-  ;
-
-/* Prevent that the lexer swallows unknown characters. */
-ANY
- :.
- ;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[7/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g
deleted file mode 100644
index f0c2368..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g
+++ /dev/null
@@ -1,2596 +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.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/HiveParser.g grammar.
-*/
-parser grammar SparkSqlParser;
-
-options
-{
-tokenVocab=SparkSqlLexer;
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-import SelectClauseParser, FromClauseParser, IdentifiersParser, KeywordParser, ExpressionParser;
-
-tokens {
-TOK_INSERT;
-TOK_QUERY;
-TOK_SELECT;
-TOK_SELECTDI;
-TOK_SELEXPR;
-TOK_FROM;
-TOK_TAB;
-TOK_PARTSPEC;
-TOK_PARTVAL;
-TOK_DIR;
-TOK_TABREF;
-TOK_SUBQUERY;
-TOK_INSERT_INTO;
-TOK_DESTINATION;
-TOK_ALLCOLREF;
-TOK_TABLE_OR_COL;
-TOK_FUNCTION;
-TOK_FUNCTIONDI;
-TOK_FUNCTIONSTAR;
-TOK_WHERE;
-TOK_OP_EQ;
-TOK_OP_NE;
-TOK_OP_LE;
-TOK_OP_LT;
-TOK_OP_GE;
-TOK_OP_GT;
-TOK_OP_DIV;
-TOK_OP_ADD;
-TOK_OP_SUB;
-TOK_OP_MUL;
-TOK_OP_MOD;
-TOK_OP_BITAND;
-TOK_OP_BITNOT;
-TOK_OP_BITOR;
-TOK_OP_BITXOR;
-TOK_OP_AND;
-TOK_OP_OR;
-TOK_OP_NOT;
-TOK_OP_LIKE;
-TOK_TRUE;
-TOK_FALSE;
-TOK_TRANSFORM;
-TOK_SERDE;
-TOK_SERDENAME;
-TOK_SERDEPROPS;
-TOK_EXPLIST;
-TOK_ALIASLIST;
-TOK_GROUPBY;
-TOK_ROLLUP_GROUPBY;
-TOK_CUBE_GROUPBY;
-TOK_GROUPING_SETS;
-TOK_GROUPING_SETS_EXPRESSION;
-TOK_HAVING;
-TOK_ORDERBY;
-TOK_CLUSTERBY;
-TOK_DISTRIBUTEBY;
-TOK_SORTBY;
-TOK_UNIONALL;
-TOK_UNIONDISTINCT;
-TOK_EXCEPT;
-TOK_INTERSECT;
-TOK_JOIN;
-TOK_LEFTOUTERJOIN;
-TOK_RIGHTOUTERJOIN;
-TOK_FULLOUTERJOIN;
-TOK_UNIQUEJOIN;
-TOK_CROSSJOIN;
-TOK_NATURALJOIN;
-TOK_NATURALLEFTOUTERJOIN;
-TOK_NATURALRIGHTOUTERJOIN;
-TOK_NATURALFULLOUTERJOIN;
-TOK_LOAD;
-TOK_EXPORT;
-TOK_IMPORT;
-TOK_REPLICATION;
-TOK_METADATA;
-TOK_NULL;
-TOK_ISNULL;
-TOK_ISNOTNULL;
-TOK_TINYINT;
-TOK_SMALLINT;
-TOK_INT;
-TOK_BIGINT;
-TOK_BOOLEAN;
-TOK_FLOAT;
-TOK_DOUBLE;
-TOK_DATE;
-TOK_DATELITERAL;
-TOK_DATETIME;
-TOK_TIMESTAMP;
-TOK_TIMESTAMPLITERAL;
-TOK_INTERVAL;
-TOK_INTERVAL_YEAR_MONTH;
-TOK_INTERVAL_YEAR_MONTH_LITERAL;
-TOK_INTERVAL_DAY_TIME;
-TOK_INTERVAL_DAY_TIME_LITERAL;
-TOK_INTERVAL_YEAR_LITERAL;
-TOK_INTERVAL_MONTH_LITERAL;
-TOK_INTERVAL_WEEK_LITERAL;
-TOK_INTERVAL_DAY_LITERAL;
-TOK_INTERVAL_HOUR_LITERAL;
-TOK_INTERVAL_MINUTE_LITERAL;
-TOK_INTERVAL_SECOND_LITERAL;
-TOK_INTERVAL_MILLISECOND_LITERAL;
-TOK_INTERVAL_MICROSECOND_LITERAL;
-TOK_STRING;
-TOK_CHAR;
-TOK_VARCHAR;
-TOK_BINARY;
-TOK_DECIMAL;
-TOK_LIST;
-TOK_STRUCT;
-TOK_MAP;
-TOK_UNIONTYPE;
-TOK_COLTYPELIST;
-TOK_CREATEDATABASE;
-TOK_CREATETABLE;
-TOK_CREATETABLEUSING;
-TOK_TRUNCATETABLE;
-TOK_CREATEINDEX;
-TOK_CREATEINDEX_INDEXTBLNAME;
-TOK_DEFERRED_REBUILDINDEX;
-TOK_DROPINDEX;
-TOK_LIKETABLE;
-TOK_DESCTABLE;
-TOK_DESCFUNCTION;
-TOK_ALTERTABLE;
-TOK_ALTERTABLE_RENAME;
-TOK_ALTERTABLE_ADDCOLS;
-TOK_ALTERTABLE_RENAMECOL;
-TOK_ALTERTABLE_RENAMEPART;
-TOK_ALTERTABLE_REPLACECOLS;
-TOK_ALTERTABLE_ADDPARTS;
-TOK_ALTERTABLE_DROPPARTS;
-TOK_ALTERTABLE_PARTCOLTYPE;
-TOK_ALTERTABLE_MERGEFILES;
-TOK_ALTERTABLE_TOUCH;
-TOK_ALTERTABLE_ARCHIVE;
-TOK_ALTERTABLE_UNARCHIVE;
-TOK_ALTERTABLE_SERDEPROPERTIES;
-TOK_ALTERTABLE_SERIALIZER;
-TOK_ALTERTABLE_UPDATECOLSTATS;
-TOK_TABLE_PARTITION;
-TOK_ALTERTABLE_FILEFORMAT;
-TOK_ALTERTABLE_LOCATION;
-TOK_ALTERTABLE_PROPERTIES;
-TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
-TOK_ALTERTABLE_DROPPROPERTIES;
-TOK_ALTERTABLE_SKEWED;
-TOK_ALTERTABLE_EXCHANGEPARTITION;
-TOK_ALTERTABLE_SKEWED_LOCATION;
-TOK_ALTERTABLE_BUCKETS;
-TOK_ALTERTABLE_CLUSTER_SORT;
-TOK_ALTERTABLE_COMPACT;
-TOK_ALTERINDEX_REBUILD;
-TOK_ALTERINDEX_PROPERTIES;
-TOK_MSCK;
-TOK_SHOWDATABASES;
-TOK_SHOWTABLES;
-TOK_SHOWCOLUMNS;
-TOK_SHOWFUNCTIONS;
-TOK_SHOWPARTITIONS;
-TOK_SHOW_CREATEDATABASE;
-TOK_SHOW_CREATETABLE;
-TOK_SHOW_TABLESTATUS;
-TOK_SHOW_TBLPROPERTIES;
-TOK_SHOWLOCKS;
-TOK_SHOWCONF;
-TOK_LOCKTABLE;
-TOK_UNLOCKTABLE;
-TOK_LOCKDB;
-TOK_UNLOCKDB;
-TOK_SWITCHDATABASE;
-TOK_DROPDATABASE;
-TOK_DROPTABLE;
-TOK_DATABASECOMMENT;
-TOK_TABCOLLIST;
-TOK_TABCOL;
-TOK_TABLECOMMENT;
-TOK_TABLEPARTCOLS;
-TOK_TABLEROWFORMAT;
-TOK_TABLEROWFORMATFIELD;
-TOK_TABLEROWFORMATCOLLITEMS;
-TOK_TABLEROWFORMATMAPKEYS;
-TOK_TABLEROWFORMATLINES;
-TOK_TABLEROWFORMATNULL;
-TOK_TABLEFILEFORMAT;
-TOK_FILEFORMAT_GENERIC;
-TOK_OFFLINE;
-TOK_ENABLE;
-TOK_DISABLE;
-TOK_READONLY;
-TOK_NO_DROP;
-TOK_STORAGEHANDLER;
-TOK_NOT_CLUSTERED;
-TOK_NOT_SORTED;
-TOK_TABCOLNAME;
-TOK_TABLELOCATION;
-TOK_PARTITIONLOCATION;
-TOK_TABLEBUCKETSAMPLE;
-TOK_TABLESPLITSAMPLE;
-TOK_PERCENT;
-TOK_LENGTH;
-TOK_ROWCOUNT;
-TOK_TMP_FILE;
-TOK_TABSORTCOLNAMEASC;
-TOK_TABSORTCOLNAMEDESC;
-TOK_STRINGLITERALSEQUENCE;
-TOK_CREATEFUNCTION;
-TOK_DROPFUNCTION;
-TOK_RELOADFUNCTION;
-TOK_CREATEMACRO;
-TOK_DROPMACRO;
-TOK_TEMPORARY;
-TOK_CREATEVIEW;
-TOK_DROPVIEW;
-TOK_ALTERVIEW;
-TOK_ALTERVIEW_PROPERTIES;
-TOK_ALTERVIEW_DROPPROPERTIES;
-TOK_ALTERVIEW_ADDPARTS;
-TOK_ALTERVIEW_DROPPARTS;
-TOK_ALTERVIEW_RENAME;
-TOK_VIEWPARTCOLS;
-TOK_EXPLAIN;
-TOK_EXPLAIN_SQ_REWRITE;
-TOK_TABLESERIALIZER;
-TOK_TABLEPROPERTIES;
-TOK_TABLEPROPLIST;
-TOK_INDEXPROPERTIES;
-TOK_INDEXPROPLIST;
-TOK_TABTYPE;
-TOK_LIMIT;
-TOK_TABLEPROPERTY;
-TOK_IFEXISTS;
-TOK_IFNOTEXISTS;
-TOK_ORREPLACE;
-TOK_HINTLIST;
-TOK_HINT;
-TOK_MAPJOIN;
-TOK_STREAMTABLE;
-TOK_HINTARGLIST;
-TOK_USERSCRIPTCOLNAMES;
-TOK_USERSCRIPTCOLSCHEMA;
-TOK_RECORDREADER;
-TOK_RECORDWRITER;
-TOK_LEFTSEMIJOIN;
-TOK_ANTIJOIN;
-TOK_LATERAL_VIEW;
-TOK_LATERAL_VIEW_OUTER;
-TOK_TABALIAS;
-TOK_ANALYZE;
-TOK_CREATEROLE;
-TOK_DROPROLE;
-TOK_GRANT;
-TOK_REVOKE;
-TOK_SHOW_GRANT;
-TOK_PRIVILEGE_LIST;
-TOK_PRIVILEGE;
-TOK_PRINCIPAL_NAME;
-TOK_USER;
-TOK_GROUP;
-TOK_ROLE;
-TOK_RESOURCE_ALL;
-TOK_GRANT_WITH_OPTION;
-TOK_GRANT_WITH_ADMIN_OPTION;
-TOK_ADMIN_OPTION_FOR;
-TOK_GRANT_OPTION_FOR;
-TOK_PRIV_ALL;
-TOK_PRIV_ALTER_METADATA;
-TOK_PRIV_ALTER_DATA;
-TOK_PRIV_DELETE;
-TOK_PRIV_DROP;
-TOK_PRIV_INDEX;
-TOK_PRIV_INSERT;
-TOK_PRIV_LOCK;
-TOK_PRIV_SELECT;
-TOK_PRIV_SHOW_DATABASE;
-TOK_PRIV_CREATE;
-TOK_PRIV_OBJECT;
-TOK_PRIV_OBJECT_COL;
-TOK_GRANT_ROLE;
-TOK_REVOKE_ROLE;
-TOK_SHOW_ROLE_GRANT;
-TOK_SHOW_ROLES;
-TOK_SHOW_SET_ROLE;
-TOK_SHOW_ROLE_PRINCIPALS;
-TOK_SHOWINDEXES;
-TOK_SHOWDBLOCKS;
-TOK_INDEXCOMMENT;
-TOK_DESCDATABASE;
-TOK_DATABASEPROPERTIES;
-TOK_DATABASELOCATION;
-TOK_DBPROPLIST;
-TOK_ALTERDATABASE_PROPERTIES;
-TOK_ALTERDATABASE_OWNER;
-TOK_TABNAME;
-TOK_TABSRC;
-TOK_RESTRICT;
-TOK_CASCADE;
-TOK_TABLESKEWED;
-TOK_TABCOLVALUE;
-TOK_TABCOLVALUE_PAIR;
-TOK_TABCOLVALUES;
-TOK_SKEWED_LOCATIONS;
-TOK_SKEWED_LOCATION_LIST;
-TOK_SKEWED_LOCATION_MAP;
-TOK_STOREDASDIRS;
-TOK_PARTITIONINGSPEC;
-TOK_PTBLFUNCTION;
-TOK_WINDOWDEF;
-TOK_WINDOWSPEC;
-TOK_WINDOWVALUES;
-TOK_WINDOWRANGE;
-TOK_SUBQUERY_EXPR;
-TOK_SUBQUERY_OP;
-TOK_SUBQUERY_OP_NOTIN;
-TOK_SUBQUERY_OP_NOTEXISTS;
-TOK_DB_TYPE;
-TOK_TABLE_TYPE;
-TOK_CTE;
-TOK_ARCHIVE;
-TOK_FILE;
-TOK_JAR;
-TOK_RESOURCE_URI;
-TOK_RESOURCE_LIST;
-TOK_SHOW_COMPACTIONS;
-TOK_SHOW_TRANSACTIONS;
-TOK_DELETE_FROM;
-TOK_UPDATE_TABLE;
-TOK_SET_COLUMNS_CLAUSE;
-TOK_VALUE_ROW;
-TOK_VALUES_TABLE;
-TOK_VIRTUAL_TABLE;
-TOK_VIRTUAL_TABREF;
-TOK_ANONYMOUS;
-TOK_COL_NAME;
-TOK_URI_TYPE;
-TOK_SERVER_TYPE;
-TOK_START_TRANSACTION;
-TOK_ISOLATION_LEVEL;
-TOK_ISOLATION_SNAPSHOT;
-TOK_TXN_ACCESS_MODE;
-TOK_TXN_READ_ONLY;
-TOK_TXN_READ_WRITE;
-TOK_COMMIT;
-TOK_ROLLBACK;
-TOK_SET_AUTOCOMMIT;
-TOK_REFRESHTABLE;
-TOK_TABLEPROVIDER;
-TOK_TABLEOPTIONS;
-TOK_TABLEOPTION;
-TOK_CACHETABLE;
-TOK_UNCACHETABLE;
-TOK_CLEARCACHE;
-TOK_SETCONFIG;
-TOK_DFS;
-TOK_ADDFILE;
-TOK_ADDJAR;
-TOK_USING;
-}
-
-
-// Package headers
-@header {
-package org.apache.spark.sql.catalyst.parser;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-}
-
-
-@members {
-  Stack msgs = new Stack<String>();
-
-  private static HashMap<String, String> xlateMap;
-  static {
-    //this is used to support auto completion in CLI
-    xlateMap = new HashMap<String, String>();
-
-    // Keywords
-    xlateMap.put("KW_TRUE", "TRUE");
-    xlateMap.put("KW_FALSE", "FALSE");
-    xlateMap.put("KW_ALL", "ALL");
-    xlateMap.put("KW_NONE", "NONE");
-    xlateMap.put("KW_AND", "AND");
-    xlateMap.put("KW_OR", "OR");
-    xlateMap.put("KW_NOT", "NOT");
-    xlateMap.put("KW_LIKE", "LIKE");
-
-    xlateMap.put("KW_ASC", "ASC");
-    xlateMap.put("KW_DESC", "DESC");
-    xlateMap.put("KW_ORDER", "ORDER");
-    xlateMap.put("KW_BY", "BY");
-    xlateMap.put("KW_GROUP", "GROUP");
-    xlateMap.put("KW_WHERE", "WHERE");
-    xlateMap.put("KW_FROM", "FROM");
-    xlateMap.put("KW_AS", "AS");
-    xlateMap.put("KW_SELECT", "SELECT");
-    xlateMap.put("KW_DISTINCT", "DISTINCT");
-    xlateMap.put("KW_INSERT", "INSERT");
-    xlateMap.put("KW_OVERWRITE", "OVERWRITE");
-    xlateMap.put("KW_OUTER", "OUTER");
-    xlateMap.put("KW_JOIN", "JOIN");
-    xlateMap.put("KW_LEFT", "LEFT");
-    xlateMap.put("KW_RIGHT", "RIGHT");
-    xlateMap.put("KW_FULL", "FULL");
-    xlateMap.put("KW_ON", "ON");
-    xlateMap.put("KW_PARTITION", "PARTITION");
-    xlateMap.put("KW_PARTITIONS", "PARTITIONS");
-    xlateMap.put("KW_TABLE", "TABLE");
-    xlateMap.put("KW_TABLES", "TABLES");
-    xlateMap.put("KW_TBLPROPERTIES", "TBLPROPERTIES");
-    xlateMap.put("KW_SHOW", "SHOW");
-    xlateMap.put("KW_MSCK", "MSCK");
-    xlateMap.put("KW_DIRECTORY", "DIRECTORY");
-    xlateMap.put("KW_LOCAL", "LOCAL");
-    xlateMap.put("KW_TRANSFORM", "TRANSFORM");
-    xlateMap.put("KW_USING", "USING");
-    xlateMap.put("KW_CLUSTER", "CLUSTER");
-    xlateMap.put("KW_DISTRIBUTE", "DISTRIBUTE");
-    xlateMap.put("KW_SORT", "SORT");
-    xlateMap.put("KW_UNION", "UNION");
-    xlateMap.put("KW_LOAD", "LOAD");
-    xlateMap.put("KW_DATA", "DATA");
-    xlateMap.put("KW_INPATH", "INPATH");
-    xlateMap.put("KW_IS", "IS");
-    xlateMap.put("KW_NULL", "NULL");
-    xlateMap.put("KW_CREATE", "CREATE");
-    xlateMap.put("KW_EXTERNAL", "EXTERNAL");
-    xlateMap.put("KW_ALTER", "ALTER");
-    xlateMap.put("KW_DESCRIBE", "DESCRIBE");
-    xlateMap.put("KW_DROP", "DROP");
-    xlateMap.put("KW_RENAME", "RENAME");
-    xlateMap.put("KW_TO", "TO");
-    xlateMap.put("KW_COMMENT", "COMMENT");
-    xlateMap.put("KW_BOOLEAN", "BOOLEAN");
-    xlateMap.put("KW_TINYINT", "TINYINT");
-    xlateMap.put("KW_SMALLINT", "SMALLINT");
-    xlateMap.put("KW_INT", "INT");
-    xlateMap.put("KW_BIGINT", "BIGINT");
-    xlateMap.put("KW_FLOAT", "FLOAT");
-    xlateMap.put("KW_DOUBLE", "DOUBLE");
-    xlateMap.put("KW_DATE", "DATE");
-    xlateMap.put("KW_DATETIME", "DATETIME");
-    xlateMap.put("KW_TIMESTAMP", "TIMESTAMP");
-    xlateMap.put("KW_STRING", "STRING");
-    xlateMap.put("KW_BINARY", "BINARY");
-    xlateMap.put("KW_ARRAY", "ARRAY");
-    xlateMap.put("KW_MAP", "MAP");
-    xlateMap.put("KW_REDUCE", "REDUCE");
-    xlateMap.put("KW_PARTITIONED", "PARTITIONED");
-    xlateMap.put("KW_CLUSTERED", "CLUSTERED");
-    xlateMap.put("KW_SORTED", "SORTED");
-    xlateMap.put("KW_INTO", "INTO");
-    xlateMap.put("KW_BUCKETS", "BUCKETS");
-    xlateMap.put("KW_ROW", "ROW");
-    xlateMap.put("KW_FORMAT", "FORMAT");
-    xlateMap.put("KW_DELIMITED", "DELIMITED");
-    xlateMap.put("KW_FIELDS", "FIELDS");
-    xlateMap.put("KW_TERMINATED", "TERMINATED");
-    xlateMap.put("KW_COLLECTION", "COLLECTION");
-    xlateMap.put("KW_ITEMS", "ITEMS");
-    xlateMap.put("KW_KEYS", "KEYS");
-    xlateMap.put("KW_KEY_TYPE", "\$KEY\$");
-    xlateMap.put("KW_LINES", "LINES");
-    xlateMap.put("KW_STORED", "STORED");
-    xlateMap.put("KW_SEQUENCEFILE", "SEQUENCEFILE");
-    xlateMap.put("KW_TEXTFILE", "TEXTFILE");
-    xlateMap.put("KW_INPUTFORMAT", "INPUTFORMAT");
-    xlateMap.put("KW_OUTPUTFORMAT", "OUTPUTFORMAT");
-    xlateMap.put("KW_LOCATION", "LOCATION");
-    xlateMap.put("KW_TABLESAMPLE", "TABLESAMPLE");
-    xlateMap.put("KW_BUCKET", "BUCKET");
-    xlateMap.put("KW_OUT", "OUT");
-    xlateMap.put("KW_OF", "OF");
-    xlateMap.put("KW_CAST", "CAST");
-    xlateMap.put("KW_ADD", "ADD");
-    xlateMap.put("KW_REPLACE", "REPLACE");
-    xlateMap.put("KW_COLUMNS", "COLUMNS");
-    xlateMap.put("KW_RLIKE", "RLIKE");
-    xlateMap.put("KW_REGEXP", "REGEXP");
-    xlateMap.put("KW_TEMPORARY", "TEMPORARY");
-    xlateMap.put("KW_FUNCTION", "FUNCTION");
-    xlateMap.put("KW_EXPLAIN", "EXPLAIN");
-    xlateMap.put("KW_EXTENDED", "EXTENDED");
-    xlateMap.put("KW_SERDE", "SERDE");
-    xlateMap.put("KW_WITH", "WITH");
-    xlateMap.put("KW_SERDEPROPERTIES", "SERDEPROPERTIES");
-    xlateMap.put("KW_LIMIT", "LIMIT");
-    xlateMap.put("KW_SET", "SET");
-    xlateMap.put("KW_PROPERTIES", "TBLPROPERTIES");
-    xlateMap.put("KW_VALUE_TYPE", "\$VALUE\$");
-    xlateMap.put("KW_ELEM_TYPE", "\$ELEM\$");
-    xlateMap.put("KW_DEFINED", "DEFINED");
-    xlateMap.put("KW_SUBQUERY", "SUBQUERY");
-    xlateMap.put("KW_REWRITE", "REWRITE");
-    xlateMap.put("KW_UPDATE", "UPDATE");
-    xlateMap.put("KW_VALUES", "VALUES");
-    xlateMap.put("KW_PURGE", "PURGE");
-    xlateMap.put("KW_WEEK", "WEEK");
-    xlateMap.put("KW_MILLISECOND", "MILLISECOND");
-    xlateMap.put("KW_MICROSECOND", "MICROSECOND");
-    xlateMap.put("KW_CLEAR", "CLEAR");
-    xlateMap.put("KW_LAZY", "LAZY");
-    xlateMap.put("KW_CACHE", "CACHE");
-    xlateMap.put("KW_UNCACHE", "UNCACHE");
-    xlateMap.put("KW_DFS", "DFS");
-
-    // Operators
-    xlateMap.put("DOT", ".");
-    xlateMap.put("COLON", ":");
-    xlateMap.put("COMMA", ",");
-    xlateMap.put("SEMICOLON", ");");
-
-    xlateMap.put("LPAREN", "(");
-    xlateMap.put("RPAREN", ")");
-    xlateMap.put("LSQUARE", "[");
-    xlateMap.put("RSQUARE", "]");
-
-    xlateMap.put("EQUAL", "=");
-    xlateMap.put("NOTEQUAL", "<>");
-    xlateMap.put("EQUAL_NS", "<=>");
-    xlateMap.put("LESSTHANOREQUALTO", "<=");
-    xlateMap.put("LESSTHAN", "<");
-    xlateMap.put("GREATERTHANOREQUALTO", ">=");
-    xlateMap.put("GREATERTHAN", ">");
-
-    xlateMap.put("DIVIDE", "/");
-    xlateMap.put("PLUS", "+");
-    xlateMap.put("MINUS", "-");
-    xlateMap.put("STAR", "*");
-    xlateMap.put("MOD", "\%");
-
-    xlateMap.put("AMPERSAND", "&");
-    xlateMap.put("TILDE", "~");
-    xlateMap.put("BITWISEOR", "|");
-    xlateMap.put("BITWISEXOR", "^");
-    xlateMap.put("CharSetLiteral", "\\'");
-  }
-
-  public static Collection<String> getKeywords() {
-    return xlateMap.values();
-  }
-
-  private static String xlate(String name) {
-
-    String ret = xlateMap.get(name);
-    if (ret == null) {
-      ret = name;
-    }
-
-    return ret;
-  }
-
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-
-  @Override
-  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-    if (reporter != null) {
-      reporter.report(this, e, tokenNames);
-    }
-  }
-
-  @Override
-  public String getErrorHeader(RecognitionException e) {
-    String header = null;
-    if (e.charPositionInLine < 0 && input.LT(-1) != null) {
-      Token t = input.LT(-1);
-      header = "line " + t.getLine() + ":" + t.getCharPositionInLine();
-    } else {
-      header = super.getErrorHeader(e);
-    }
-
-    return header;
-  }
-
-  @Override
-  public String getErrorMessage(RecognitionException e, String[] tokenNames) {
-    String msg = null;
-
-    // Translate the token names to something that the user can understand
-    String[] xlateNames = new String[tokenNames.length];
-    for (int i = 0; i < tokenNames.length; ++i) {
-      xlateNames[i] = SparkSqlParser.xlate(tokenNames[i]);
-    }
-
-    if (e instanceof NoViableAltException) {
-      @SuppressWarnings("unused")
-      NoViableAltException nvae = (NoViableAltException) e;
-      // for development, can add
-      // "decision=<<"+nvae.grammarDecisionDescription+">>"
-      // and "(decision="+nvae.decisionNumber+") and
-      // "state "+nvae.stateNumber
-      msg = "cannot recognize input near"
-              + (input.LT(1) != null ? " " + getTokenErrorDisplay(input.LT(1)) : "")
-              + (input.LT(2) != null ? " " + getTokenErrorDisplay(input.LT(2)) : "")
-              + (input.LT(3) != null ? " " + getTokenErrorDisplay(input.LT(3)) : "");
-    } else if (e instanceof MismatchedTokenException) {
-      MismatchedTokenException mte = (MismatchedTokenException) e;
-      msg = super.getErrorMessage(e, xlateNames) + (input.LT(-1) == null ? "":" near '" + input.LT(-1).getText()) + "'";
-    } else if (e instanceof FailedPredicateException) {
-      FailedPredicateException fpe = (FailedPredicateException) e;
-      msg = "Failed to recognize predicate '" + fpe.token.getText() + "'. Failed rule: '" + fpe.ruleName + "'";
-    } else {
-      msg = super.getErrorMessage(e, xlateNames);
-    }
-
-    if (msgs.size() > 0) {
-      msg = msg + " in " + msgs.peek();
-    }
-    return msg;
-  }
-
-  public void pushMsg(String msg, RecognizerSharedState state) {
-    // ANTLR generated code does not wrap the @init code wit this backtracking check,
-    //  even if the matching @after has it. If we have parser rules with that are doing
-    // some lookahead with syntactic predicates this can cause the push() and pop() calls
-    // to become unbalanced, so make sure both push/pop check the backtracking state.
-    if (state.backtracking == 0) {
-      msgs.push(msg);
-    }
-  }
-
-  public void popMsg(RecognizerSharedState state) {
-    if (state.backtracking == 0) {
-      Object o = msgs.pop();
-    }
-  }
-
-  // counter to generate unique union aliases
-  private int aliasCounter;
-  private String generateUnionAlias() {
-    return "u_" + (++aliasCounter);
-  }
-  private char [] excludedCharForColumnName = {'.', ':'};
-  private boolean containExcludedCharForCreateTableColumnName(String input) {
-    if (input.length() > 0) {
-      if (input.charAt(0) == '`' && input.charAt(input.length() - 1) == '`') {
-        // When column name is backquoted, we don't care about excluded chars.
-        return false;
-      }
-    }
-    for(char c : excludedCharForColumnName) {
-      if(input.indexOf(c)>-1) {
-        return true;
-      }
-    }
-    return false;
-  }
-  private CommonTree throwSetOpException() throws RecognitionException {
-    throw new FailedPredicateException(input, "orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.", "");
-  }
-  private CommonTree throwColumnNameException() throws RecognitionException {
-    throw new FailedPredicateException(input, Arrays.toString(excludedCharForColumnName) + " can not be used in column name in create table statement.", "");
-  }
-
-  private ParserConf parserConf;
-  private ParseErrorReporter reporter;
-
-  public void configure(ParserConf parserConf, ParseErrorReporter reporter) {
-    this.parserConf = parserConf;
-    this.reporter = reporter;
-  }
-
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    if (parserConf == null) {
-      return true;
-    }
-    return !parserConf.supportSQL11ReservedKeywords();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
- reportError(e);
-  throw e;
-}
-}
-
-// starting rule
-statement
-    : explainStatement EOF
-    | execStatement EOF
-    | KW_ADD KW_JAR -> ^(TOK_ADDJAR)
-    | KW_ADD KW_FILE -> ^(TOK_ADDFILE)
-    | KW_DFS -> ^(TOK_DFS)
-    | (KW_SET)=> KW_SET -> ^(TOK_SETCONFIG)
-	;
-
-// Rule for expression parsing
-singleNamedExpression
-    :
-    namedExpression EOF
-    ;
-
-// Rule for table name parsing
-singleTableName
-    :
-    tableName EOF
-    ;
-
-explainStatement
-@init { pushMsg("explain statement", state); }
-@after { popMsg(state); }
-	: KW_EXPLAIN (
-	    explainOption* execStatement -> ^(TOK_EXPLAIN execStatement explainOption*)
-        |
-        KW_REWRITE queryStatementExpression[true] -> ^(TOK_EXPLAIN_SQ_REWRITE queryStatementExpression))
-	;
-
-explainOption
-@init { msgs.push("explain option"); }
-@after { msgs.pop(); }
-    : KW_EXTENDED|KW_FORMATTED|KW_DEPENDENCY|KW_LOGICAL|KW_AUTHORIZATION
-    ;
-
-execStatement
-@init { pushMsg("statement", state); }
-@after { popMsg(state); }
-    : queryStatementExpression[true]
-    | loadStatement
-    | exportStatement
-    | importStatement
-    | ddlStatement
-    | deleteStatement
-    | updateStatement
-    | sqlTransactionStatement
-    | cacheStatement
-    ;
-
-loadStatement
-@init { pushMsg("load statement", state); }
-@after { popMsg(state); }
-    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition)
-    -> ^(TOK_LOAD $path $tab $islocal? $isoverwrite?)
-    ;
-
-replicationClause
-@init { pushMsg("replication clause", state); }
-@after { popMsg(state); }
-    : KW_FOR (isMetadataOnly=KW_METADATA)? KW_REPLICATION LPAREN (replId=StringLiteral) RPAREN
-    -> ^(TOK_REPLICATION $replId $isMetadataOnly?)
-    ;
-
-exportStatement
-@init { pushMsg("export statement", state); }
-@after { popMsg(state); }
-    : KW_EXPORT
-      KW_TABLE (tab=tableOrPartition)
-      KW_TO (path=StringLiteral)
-      replicationClause?
-    -> ^(TOK_EXPORT $tab $path replicationClause?)
-    ;
-
-importStatement
-@init { pushMsg("import statement", state); }
-@after { popMsg(state); }
-       : KW_IMPORT
-         ((ext=KW_EXTERNAL)? KW_TABLE (tab=tableOrPartition))?
-         KW_FROM (path=StringLiteral)
-         tableLocation?
-    -> ^(TOK_IMPORT $path $tab? $ext? tableLocation?)
-    ;
-
-ddlStatement
-@init { pushMsg("ddl statement", state); }
-@after { popMsg(state); }
-    : createDatabaseStatement
-    | switchDatabaseStatement
-    | dropDatabaseStatement
-    | createTableStatement
-    | dropTableStatement
-    | truncateTableStatement
-    | alterStatement
-    | descStatement
-    | refreshStatement
-    | showStatement
-    | metastoreCheck
-    | createViewStatement
-    | dropViewStatement
-    | createFunctionStatement
-    | createMacroStatement
-    | createIndexStatement
-    | dropIndexStatement
-    | dropFunctionStatement
-    | reloadFunctionStatement
-    | dropMacroStatement
-    | analyzeStatement
-    | lockStatement
-    | unlockStatement
-    | lockDatabase
-    | unlockDatabase
-    | createRoleStatement
-    | dropRoleStatement
-    | (grantPrivileges) => grantPrivileges
-    | (revokePrivileges) => revokePrivileges
-    | showGrants
-    | showRoleGrants
-    | showRolePrincipals
-    | showRoles
-    | grantRole
-    | revokeRole
-    | setRole
-    | showCurrentRole
-    ;
-
-ifExists
-@init { pushMsg("if exists clause", state); }
-@after { popMsg(state); }
-    : KW_IF KW_EXISTS
-    -> ^(TOK_IFEXISTS)
-    ;
-
-restrictOrCascade
-@init { pushMsg("restrict or cascade clause", state); }
-@after { popMsg(state); }
-    : KW_RESTRICT
-    -> ^(TOK_RESTRICT)
-    | KW_CASCADE
-    -> ^(TOK_CASCADE)
-    ;
-
-ifNotExists
-@init { pushMsg("if not exists clause", state); }
-@after { popMsg(state); }
-    : KW_IF KW_NOT KW_EXISTS
-    -> ^(TOK_IFNOTEXISTS)
-    ;
-
-storedAsDirs
-@init { pushMsg("stored as directories", state); }
-@after { popMsg(state); }
-    : KW_STORED KW_AS KW_DIRECTORIES
-    -> ^(TOK_STOREDASDIRS)
-    ;
-
-orReplace
-@init { pushMsg("or replace clause", state); }
-@after { popMsg(state); }
-    : KW_OR KW_REPLACE
-    -> ^(TOK_ORREPLACE)
-    ;
-
-createDatabaseStatement
-@init { pushMsg("create database statement", state); }
-@after { popMsg(state); }
-    : KW_CREATE (KW_DATABASE|KW_SCHEMA)
-        ifNotExists?
-        name=identifier
-        databaseComment?
-        dbLocation?
-        (KW_WITH KW_DBPROPERTIES dbprops=dbProperties)?
-    -> ^(TOK_CREATEDATABASE $name ifNotExists? dbLocation? databaseComment? $dbprops?)
-    ;
-
-dbLocation
-@init { pushMsg("database location specification", state); }
-@after { popMsg(state); }
-    :
-      KW_LOCATION locn=StringLiteral -> ^(TOK_DATABASELOCATION $locn)
-    ;
-
-dbProperties
-@init { pushMsg("dbproperties", state); }
-@after { popMsg(state); }
-    :
-      LPAREN dbPropertiesList RPAREN -> ^(TOK_DATABASEPROPERTIES dbPropertiesList)
-    ;
-
-dbPropertiesList
-@init { pushMsg("database properties list", state); }
-@after { popMsg(state); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* -> ^(TOK_DBPROPLIST keyValueProperty+)
-    ;
-
-
-switchDatabaseStatement
-@init { pushMsg("switch database statement", state); }
-@after { popMsg(state); }
-    : KW_USE identifier
-    -> ^(TOK_SWITCHDATABASE identifier)
-    ;
-
-dropDatabaseStatement
-@init { pushMsg("drop database statement", state); }
-@after { popMsg(state); }
-    : KW_DROP (KW_DATABASE|KW_SCHEMA) ifExists? identifier restrictOrCascade?
-    -> ^(TOK_DROPDATABASE identifier ifExists? restrictOrCascade?)
-    ;
-
-databaseComment
-@init { pushMsg("database's comment", state); }
-@after { popMsg(state); }
-    : KW_COMMENT comment=StringLiteral
-    -> ^(TOK_DATABASECOMMENT $comment)
-    ;
-
-createTableStatement
-@init { pushMsg("create table statement", state); }
-@after { popMsg(state); }
-    : KW_CREATE (temp=KW_TEMPORARY)? (ext=KW_EXTERNAL)? KW_TABLE ifNotExists? name=tableName
-      (
-         like=KW_LIKE likeName=tableName
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-      -> ^(TOK_CREATETABLE $name $temp? $ext? ifNotExists?
-         ^(TOK_LIKETABLE $likeName?)
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-         )
-      |
-         (tableProvider) => tableProvider
-         tableOpts?
-         (KW_AS selectStatementWithCTE)?
-      -> ^(TOK_CREATETABLEUSING $name $temp? ifNotExists?
-          tableProvider
-          tableOpts?
-          selectStatementWithCTE?
-          )
-       | (LPAREN columnNameTypeList RPAREN)?
-         (p=tableProvider?)
-         tableOpts?
-         tableComment?
-         tablePartition?
-         tableBuckets?
-         tableSkewed?
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-         (KW_AS selectStatementWithCTE)?
-      -> {p != null}?
-         ^(TOK_CREATETABLEUSING $name $temp? ifNotExists?
-         columnNameTypeList?
-         $p
-         tableOpts?
-         selectStatementWithCTE?
-         )
-      ->
-         ^(TOK_CREATETABLE $name $temp? $ext? ifNotExists?
-         ^(TOK_LIKETABLE $likeName?)
-         columnNameTypeList?
-         tableComment?
-         tablePartition?
-         tableBuckets?
-         tableSkewed?
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-         selectStatementWithCTE?
-         )
-      )
-    ;
-
-truncateTableStatement
-@init { pushMsg("truncate table statement", state); }
-@after { popMsg(state); }
-    : KW_TRUNCATE KW_TABLE tablePartitionPrefix (KW_COLUMNS LPAREN columnNameList RPAREN)? -> ^(TOK_TRUNCATETABLE tablePartitionPrefix columnNameList?);
-
-createIndexStatement
-@init { pushMsg("create index statement", state);}
-@after {popMsg(state);}
-    : KW_CREATE KW_INDEX indexName=identifier
-      KW_ON KW_TABLE tab=tableName LPAREN indexedCols=columnNameList RPAREN
-      KW_AS typeName=StringLiteral
-      autoRebuild?
-      indexPropertiesPrefixed?
-      indexTblName?
-      tableRowFormat?
-      tableFileFormat?
-      tableLocation?
-      tablePropertiesPrefixed?
-      indexComment?
-    ->^(TOK_CREATEINDEX $indexName $typeName $tab $indexedCols
-        autoRebuild?
-        indexPropertiesPrefixed?
-        indexTblName?
-        tableRowFormat?
-        tableFileFormat?
-        tableLocation?
-        tablePropertiesPrefixed?
-        indexComment?)
-    ;
-
-indexComment
-@init { pushMsg("comment on an index", state);}
-@after {popMsg(state);}
-        :
-                KW_COMMENT comment=StringLiteral  -> ^(TOK_INDEXCOMMENT $comment)
-        ;
-
-autoRebuild
-@init { pushMsg("auto rebuild index", state);}
-@after {popMsg(state);}
-    : KW_WITH KW_DEFERRED KW_REBUILD
-    ->^(TOK_DEFERRED_REBUILDINDEX)
-    ;
-
-indexTblName
-@init { pushMsg("index table name", state);}
-@after {popMsg(state);}
-    : KW_IN KW_TABLE indexTbl=tableName
-    ->^(TOK_CREATEINDEX_INDEXTBLNAME $indexTbl)
-    ;
-
-indexPropertiesPrefixed
-@init { pushMsg("table properties with prefix", state); }
-@after { popMsg(state); }
-    :
-        KW_IDXPROPERTIES! indexProperties
-    ;
-
-indexProperties
-@init { pushMsg("index properties", state); }
-@after { popMsg(state); }
-    :
-      LPAREN indexPropertiesList RPAREN -> ^(TOK_INDEXPROPERTIES indexPropertiesList)
-    ;
-
-indexPropertiesList
-@init { pushMsg("index properties list", state); }
-@after { popMsg(state); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* -> ^(TOK_INDEXPROPLIST keyValueProperty+)
-    ;
-
-dropIndexStatement
-@init { pushMsg("drop index statement", state);}
-@after {popMsg(state);}
-    : KW_DROP KW_INDEX ifExists? indexName=identifier KW_ON tab=tableName
-    ->^(TOK_DROPINDEX $indexName $tab ifExists?)
-    ;
-
-dropTableStatement
-@init { pushMsg("drop statement", state); }
-@after { popMsg(state); }
-    : KW_DROP KW_TABLE ifExists? tableName KW_PURGE? replicationClause?
-    -> ^(TOK_DROPTABLE tableName ifExists? KW_PURGE? replicationClause?)
-    ;
-
-alterStatement
-@init { pushMsg("alter statement", state); }
-@after { popMsg(state); }
-    : KW_ALTER KW_TABLE tableName alterTableStatementSuffix -> ^(TOK_ALTERTABLE tableName alterTableStatementSuffix)
-    | KW_ALTER KW_VIEW tableName KW_AS? alterViewStatementSuffix -> ^(TOK_ALTERVIEW tableName alterViewStatementSuffix)
-    | KW_ALTER KW_INDEX alterIndexStatementSuffix -> alterIndexStatementSuffix
-    | KW_ALTER (KW_DATABASE|KW_SCHEMA) alterDatabaseStatementSuffix -> alterDatabaseStatementSuffix
-    ;
-
-alterTableStatementSuffix
-@init { pushMsg("alter table statement", state); }
-@after { popMsg(state); }
-    : (alterStatementSuffixRename[true]) => alterStatementSuffixRename[true]
-    | alterStatementSuffixDropPartitions[true]
-    | alterStatementSuffixAddPartitions[true]
-    | alterStatementSuffixTouch
-    | alterStatementSuffixArchive
-    | alterStatementSuffixUnArchive
-    | alterStatementSuffixProperties
-    | alterStatementSuffixSkewedby
-    | alterStatementSuffixExchangePartition
-    | alterStatementPartitionKeyType
-    | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
-    ;
-
-alterTblPartitionStatementSuffix
-@init {pushMsg("alter table partition statement suffix", state);}
-@after {popMsg(state);}
-  : alterStatementSuffixFileFormat
-  | alterStatementSuffixLocation
-  | alterStatementSuffixMergeFiles
-  | alterStatementSuffixSerdeProperties
-  | alterStatementSuffixRenamePart
-  | alterStatementSuffixBucketNum
-  | alterTblPartitionStatementSuffixSkewedLocation
-  | alterStatementSuffixClusterbySortby
-  | alterStatementSuffixCompact
-  | alterStatementSuffixUpdateStatsCol
-  | alterStatementSuffixRenameCol
-  | alterStatementSuffixAddCol
-  ;
-
-alterStatementPartitionKeyType
-@init {msgs.push("alter partition key type"); }
-@after {msgs.pop();}
-	: KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
-	-> ^(TOK_ALTERTABLE_PARTCOLTYPE columnNameType)
-	;
-
-alterViewStatementSuffix
-@init { pushMsg("alter view statement", state); }
-@after { popMsg(state); }
-    : alterViewSuffixProperties
-    | alterStatementSuffixRename[false]
-    | alterStatementSuffixAddPartitions[false]
-    | alterStatementSuffixDropPartitions[false]
-    | selectStatementWithCTE
-    ;
-
-alterIndexStatementSuffix
-@init { pushMsg("alter index statement", state); }
-@after { popMsg(state); }
-    : indexName=identifier KW_ON tableName partitionSpec?
-    (
-      KW_REBUILD
-      ->^(TOK_ALTERINDEX_REBUILD tableName $indexName partitionSpec?)
-    |
-      KW_SET KW_IDXPROPERTIES
-      indexProperties
-      ->^(TOK_ALTERINDEX_PROPERTIES tableName $indexName indexProperties)
-    )
-    ;
-
-alterDatabaseStatementSuffix
-@init { pushMsg("alter database statement", state); }
-@after { popMsg(state); }
-    : alterDatabaseSuffixProperties
-    | alterDatabaseSuffixSetOwner
-    ;
-
-alterDatabaseSuffixProperties
-@init { pushMsg("alter database properties statement", state); }
-@after { popMsg(state); }
-    : name=identifier KW_SET KW_DBPROPERTIES dbProperties
-    -> ^(TOK_ALTERDATABASE_PROPERTIES $name dbProperties)
-    ;
-
-alterDatabaseSuffixSetOwner
-@init { pushMsg("alter database set owner", state); }
-@after { popMsg(state); }
-    : dbName=identifier KW_SET KW_OWNER principalName
-    -> ^(TOK_ALTERDATABASE_OWNER $dbName principalName)
-    ;
-
-alterStatementSuffixRename[boolean table]
-@init { pushMsg("rename statement", state); }
-@after { popMsg(state); }
-    : KW_RENAME KW_TO tableName
-    -> { table }? ^(TOK_ALTERTABLE_RENAME tableName)
-    ->            ^(TOK_ALTERVIEW_RENAME tableName)
-    ;
-
-alterStatementSuffixAddCol
-@init { pushMsg("add column statement", state); }
-@after { popMsg(state); }
-    : (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN restrictOrCascade?
-    -> {$add != null}? ^(TOK_ALTERTABLE_ADDCOLS columnNameTypeList restrictOrCascade?)
-    ->                 ^(TOK_ALTERTABLE_REPLACECOLS columnNameTypeList restrictOrCascade?)
-    ;
-
-alterStatementSuffixRenameCol
-@init { pushMsg("rename column name", state); }
-@after { popMsg(state); }
-    : KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition? restrictOrCascade?
-    ->^(TOK_ALTERTABLE_RENAMECOL $oldName $newName colType $comment? alterStatementChangeColPosition? restrictOrCascade?)
-    ;
-
-alterStatementSuffixUpdateStatsCol
-@init { pushMsg("update column statistics", state); }
-@after { popMsg(state); }
-    : KW_UPDATE KW_STATISTICS KW_FOR KW_COLUMN? colName=identifier KW_SET tableProperties (KW_COMMENT comment=StringLiteral)?
-    ->^(TOK_ALTERTABLE_UPDATECOLSTATS $colName tableProperties $comment?)
-    ;
-
-alterStatementChangeColPosition
-    : first=KW_FIRST|KW_AFTER afterCol=identifier
-    ->{$first != null}? ^(TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION )
-    -> ^(TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION $afterCol)
-    ;
-
-alterStatementSuffixAddPartitions[boolean table]
-@init { pushMsg("add partition statement", state); }
-@after { popMsg(state); }
-    : KW_ADD ifNotExists? alterStatementSuffixAddPartitionsElement+
-    -> { table }? ^(TOK_ALTERTABLE_ADDPARTS ifNotExists? alterStatementSuffixAddPartitionsElement+)
-    ->            ^(TOK_ALTERVIEW_ADDPARTS ifNotExists? alterStatementSuffixAddPartitionsElement+)
-    ;
-
-alterStatementSuffixAddPartitionsElement
-    : partitionSpec partitionLocation?
-    ;
-
-alterStatementSuffixTouch
-@init { pushMsg("touch statement", state); }
-@after { popMsg(state); }
-    : KW_TOUCH (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_TOUCH (partitionSpec)*)
-    ;
-
-alterStatementSuffixArchive
-@init { pushMsg("archive statement", state); }
-@after { popMsg(state); }
-    : KW_ARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_ARCHIVE (partitionSpec)*)
-    ;
-
-alterStatementSuffixUnArchive
-@init { pushMsg("unarchive statement", state); }
-@after { popMsg(state); }
-    : KW_UNARCHIVE (partitionSpec)*
-    -> ^(TOK_ALTERTABLE_UNARCHIVE (partitionSpec)*)
-    ;
-
-partitionLocation
-@init { pushMsg("partition location", state); }
-@after { popMsg(state); }
-    :
-      KW_LOCATION locn=StringLiteral -> ^(TOK_PARTITIONLOCATION $locn)
-    ;
-
-alterStatementSuffixDropPartitions[boolean table]
-@init { pushMsg("drop partition statement", state); }
-@after { popMsg(state); }
-    : KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* KW_PURGE? replicationClause?
-    -> { table }? ^(TOK_ALTERTABLE_DROPPARTS dropPartitionSpec+ ifExists? KW_PURGE? replicationClause?)
-    ->            ^(TOK_ALTERVIEW_DROPPARTS dropPartitionSpec+ ifExists? replicationClause?)
-    ;
-
-alterStatementSuffixProperties
-@init { pushMsg("alter properties statement", state); }
-@after { popMsg(state); }
-    : KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERTABLE_PROPERTIES tableProperties)
-    | KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_ALTERTABLE_DROPPROPERTIES tableProperties ifExists?)
-    ;
-
-alterViewSuffixProperties
-@init { pushMsg("alter view properties statement", state); }
-@after { popMsg(state); }
-    : KW_SET KW_TBLPROPERTIES tableProperties
-    -> ^(TOK_ALTERVIEW_PROPERTIES tableProperties)
-    | KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    -> ^(TOK_ALTERVIEW_DROPPROPERTIES tableProperties ifExists?)
-    ;
-
-alterStatementSuffixSerdeProperties
-@init { pushMsg("alter serdes statement", state); }
-@after { popMsg(state); }
-    : KW_SET KW_SERDE serdeName=StringLiteral (KW_WITH KW_SERDEPROPERTIES tableProperties)?
-    -> ^(TOK_ALTERTABLE_SERIALIZER $serdeName tableProperties?)
-    | KW_SET KW_SERDEPROPERTIES tableProperties
-    -> ^(TOK_ALTERTABLE_SERDEPROPERTIES tableProperties)
-    ;
-
-tablePartitionPrefix
-@init {pushMsg("table partition prefix", state);}
-@after {popMsg(state);}
-  : tableName partitionSpec?
-  ->^(TOK_TABLE_PARTITION tableName partitionSpec?)
-  ;
-
-alterStatementSuffixFileFormat
-@init {pushMsg("alter fileformat statement", state); }
-@after {popMsg(state);}
-	: KW_SET KW_FILEFORMAT fileFormat
-	-> ^(TOK_ALTERTABLE_FILEFORMAT fileFormat)
-	;
-
-alterStatementSuffixClusterbySortby
-@init {pushMsg("alter partition cluster by sort by statement", state);}
-@after {popMsg(state);}
-  : KW_NOT KW_CLUSTERED -> ^(TOK_ALTERTABLE_CLUSTER_SORT TOK_NOT_CLUSTERED)
-  | KW_NOT KW_SORTED -> ^(TOK_ALTERTABLE_CLUSTER_SORT TOK_NOT_SORTED)
-  | tableBuckets -> ^(TOK_ALTERTABLE_CLUSTER_SORT tableBuckets)
-  ;
-
-alterTblPartitionStatementSuffixSkewedLocation
-@init {pushMsg("alter partition skewed location", state);}
-@after {popMsg(state);}
-  : KW_SET KW_SKEWED KW_LOCATION skewedLocations
-  -> ^(TOK_ALTERTABLE_SKEWED_LOCATION skewedLocations)
-  ;
-
-skewedLocations
-@init { pushMsg("skewed locations", state); }
-@after { popMsg(state); }
-    :
-      LPAREN skewedLocationsList RPAREN -> ^(TOK_SKEWED_LOCATIONS skewedLocationsList)
-    ;
-
-skewedLocationsList
-@init { pushMsg("skewed locations list", state); }
-@after { popMsg(state); }
-    :
-      skewedLocationMap (COMMA skewedLocationMap)* -> ^(TOK_SKEWED_LOCATION_LIST skewedLocationMap+)
-    ;
-
-skewedLocationMap
-@init { pushMsg("specifying skewed location map", state); }
-@after { popMsg(state); }
-    :
-      key=skewedValueLocationElement EQUAL value=StringLiteral -> ^(TOK_SKEWED_LOCATION_MAP $key $value)
-    ;
-
-alterStatementSuffixLocation
-@init {pushMsg("alter location", state);}
-@after {popMsg(state);}
-  : KW_SET KW_LOCATION newLoc=StringLiteral
-  -> ^(TOK_ALTERTABLE_LOCATION $newLoc)
-  ;
-
-
-alterStatementSuffixSkewedby
-@init {pushMsg("alter skewed by statement", state);}
-@after{popMsg(state);}
-	: tableSkewed
-	->^(TOK_ALTERTABLE_SKEWED tableSkewed)
-	|
-	 KW_NOT KW_SKEWED
-	->^(TOK_ALTERTABLE_SKEWED)
-	|
-	 KW_NOT storedAsDirs
-	->^(TOK_ALTERTABLE_SKEWED storedAsDirs)
-	;
-
-alterStatementSuffixExchangePartition
-@init {pushMsg("alter exchange partition", state);}
-@after{popMsg(state);}
-    : KW_EXCHANGE partitionSpec KW_WITH KW_TABLE exchangename=tableName
-    -> ^(TOK_ALTERTABLE_EXCHANGEPARTITION partitionSpec $exchangename)
-    ;
-
-alterStatementSuffixRenamePart
-@init { pushMsg("alter table rename partition statement", state); }
-@after { popMsg(state); }
-    : KW_RENAME KW_TO partitionSpec
-    ->^(TOK_ALTERTABLE_RENAMEPART partitionSpec)
-    ;
-
-alterStatementSuffixStatsPart
-@init { pushMsg("alter table stats partition statement", state); }
-@after { popMsg(state); }
-    : KW_UPDATE KW_STATISTICS KW_FOR KW_COLUMN? colName=identifier KW_SET tableProperties (KW_COMMENT comment=StringLiteral)?
-    ->^(TOK_ALTERTABLE_UPDATECOLSTATS $colName tableProperties $comment?)
-    ;
-
-alterStatementSuffixMergeFiles
-@init { pushMsg("", state); }
-@after { popMsg(state); }
-    : KW_CONCATENATE
-    -> ^(TOK_ALTERTABLE_MERGEFILES)
-    ;
-
-alterStatementSuffixBucketNum
-@init { pushMsg("", state); }
-@after { popMsg(state); }
-    : KW_INTO num=Number KW_BUCKETS
-    -> ^(TOK_ALTERTABLE_BUCKETS $num)
-    ;
-
-alterStatementSuffixCompact
-@init { msgs.push("compaction request"); }
-@after { msgs.pop(); }
-    : KW_COMPACT compactType=StringLiteral
-    -> ^(TOK_ALTERTABLE_COMPACT $compactType)
-    ;
-
-
-fileFormat
-@init { pushMsg("file format specification", state); }
-@after { popMsg(state); }
-    : KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral KW_SERDE serdeCls=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
-      -> ^(TOK_TABLEFILEFORMAT $inFmt $outFmt $serdeCls $inDriver? $outDriver?)
-    | genericSpec=identifier -> ^(TOK_FILEFORMAT_GENERIC $genericSpec)
-    ;
-
-tabTypeExpr
-@init { pushMsg("specifying table types", state); }
-@after { popMsg(state); }
-   : identifier (DOT^ identifier)?
-   (identifier (DOT^
-   (
-   (KW_ELEM_TYPE) => KW_ELEM_TYPE
-   |
-   (KW_KEY_TYPE) => KW_KEY_TYPE
-   |
-   (KW_VALUE_TYPE) => KW_VALUE_TYPE
-   | identifier
-   ))*
-   )?
-   ;
-
-partTypeExpr
-@init { pushMsg("specifying table partitions", state); }
-@after { popMsg(state); }
-    :  tabTypeExpr partitionSpec? -> ^(TOK_TABTYPE tabTypeExpr partitionSpec?)
-    ;
-
-tabPartColTypeExpr
-@init { pushMsg("specifying table partitions columnName", state); }
-@after { popMsg(state); }
-    :  tableName partitionSpec? extColumnName? -> ^(TOK_TABTYPE tableName partitionSpec? extColumnName?)
-    ;
-
-refreshStatement
-@init { pushMsg("refresh statement", state); }
-@after { popMsg(state); }
-    :
-    KW_REFRESH KW_TABLE tableName -> ^(TOK_REFRESHTABLE tableName)
-    ;
-
-descStatement
-@init { pushMsg("describe statement", state); }
-@after { popMsg(state); }
-    :
-    (KW_DESCRIBE|KW_DESC)
-    (
-    (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) KW_EXTENDED? (dbName=identifier) -> ^(TOK_DESCDATABASE $dbName KW_EXTENDED?)
-    |
-    (KW_FUNCTION) => KW_FUNCTION KW_EXTENDED? (name=descFuncNames) -> ^(TOK_DESCFUNCTION $name KW_EXTENDED?)
-    |
-    (KW_FORMATTED|KW_EXTENDED|KW_PRETTY) => ((descOptions=KW_FORMATTED|descOptions=KW_EXTENDED|descOptions=KW_PRETTY) parttype=tabPartColTypeExpr) -> ^(TOK_DESCTABLE $parttype $descOptions)
-    |
-    parttype=tabPartColTypeExpr -> ^(TOK_DESCTABLE $parttype)
-    )
-    ;
-
-analyzeStatement
-@init { pushMsg("analyze statement", state); }
-@after { popMsg(state); }
-    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN)
-                                                      | (KW_FOR KW_COLUMNS (statsColumnName=columnNameList)?))?
-      -> ^(TOK_ANALYZE $parttype $noscan? $partialscan? KW_COLUMNS? $statsColumnName?)
-    ;
-
-showStatement
-@init { pushMsg("show statement", state); }
-@after { popMsg(state); }
-    : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
-    | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  -> ^(TOK_SHOWTABLES ^(TOK_FROM $db_name)? showStmtIdentifier?)
-    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tableName ((KW_FROM|KW_IN) db_name=identifier)?
-    -> ^(TOK_SHOWCOLUMNS tableName $db_name?)
-    | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)?  -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)
-    | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?)
-    | KW_SHOW KW_CREATE (
-        (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) db_name=identifier -> ^(TOK_SHOW_CREATEDATABASE $db_name)
-        |
-        KW_TABLE tabName=tableName -> ^(TOK_SHOW_CREATETABLE $tabName)
-      )
-    | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
-    -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?)
-    | KW_SHOW KW_TBLPROPERTIES tableName (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES tableName $prptyName?)
-    | KW_SHOW KW_LOCKS
-      (
-      (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
-      |
-      (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWLOCKS $parttype? $isExtended?)
-      )
-    | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
-    -> ^(TOK_SHOWINDEXES showStmtIdentifier $showOptions? $db_name?)
-    | KW_SHOW KW_COMPACTIONS -> ^(TOK_SHOW_COMPACTIONS)
-    | KW_SHOW KW_TRANSACTIONS -> ^(TOK_SHOW_TRANSACTIONS)
-    | KW_SHOW KW_CONF StringLiteral -> ^(TOK_SHOWCONF StringLiteral)
-    ;
-
-lockStatement
-@init { pushMsg("lock statement", state); }
-@after { popMsg(state); }
-    : KW_LOCK KW_TABLE tableName partitionSpec? lockMode -> ^(TOK_LOCKTABLE tableName lockMode partitionSpec?)
-    ;
-
-lockDatabase
-@init { pushMsg("lock database statement", state); }
-@after { popMsg(state); }
-    : KW_LOCK (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) lockMode -> ^(TOK_LOCKDB $dbName lockMode)
-    ;
-
-lockMode
-@init { pushMsg("lock mode", state); }
-@after { popMsg(state); }
-    : KW_SHARED | KW_EXCLUSIVE
-    ;
-
-unlockStatement
-@init { pushMsg("unlock statement", state); }
-@after { popMsg(state); }
-    : KW_UNLOCK KW_TABLE tableName partitionSpec?  -> ^(TOK_UNLOCKTABLE tableName partitionSpec?)
-    ;
-
-unlockDatabase
-@init { pushMsg("unlock database statement", state); }
-@after { popMsg(state); }
-    : KW_UNLOCK (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) -> ^(TOK_UNLOCKDB $dbName)
-    ;
-
-createRoleStatement
-@init { pushMsg("create role", state); }
-@after { popMsg(state); }
-    : KW_CREATE KW_ROLE roleName=identifier
-    -> ^(TOK_CREATEROLE $roleName)
-    ;
-
-dropRoleStatement
-@init {pushMsg("drop role", state);}
-@after {popMsg(state);}
-    : KW_DROP KW_ROLE roleName=identifier
-    -> ^(TOK_DROPROLE $roleName)
-    ;
-
-grantPrivileges
-@init {pushMsg("grant privileges", state);}
-@after {popMsg(state);}
-    : KW_GRANT privList=privilegeList
-      privilegeObject?
-      KW_TO principalSpecification
-      withGrantOption?
-    -> ^(TOK_GRANT $privList principalSpecification privilegeObject? withGrantOption?)
-    ;
-
-revokePrivileges
-@init {pushMsg("revoke privileges", state);}
-@afer {popMsg(state);}
-    : KW_REVOKE grantOptionFor? privilegeList privilegeObject? KW_FROM principalSpecification
-    -> ^(TOK_REVOKE privilegeList principalSpecification privilegeObject? grantOptionFor?)
-    ;
-
-grantRole
-@init {pushMsg("grant role", state);}
-@after {popMsg(state);}
-    : KW_GRANT KW_ROLE? identifier (COMMA identifier)* KW_TO principalSpecification withAdminOption?
-    -> ^(TOK_GRANT_ROLE principalSpecification withAdminOption? identifier+)
-    ;
-
-revokeRole
-@init {pushMsg("revoke role", state);}
-@after {popMsg(state);}
-    : KW_REVOKE adminOptionFor? KW_ROLE? identifier (COMMA identifier)* KW_FROM principalSpecification
-    -> ^(TOK_REVOKE_ROLE principalSpecification adminOptionFor? identifier+)
-    ;
-
-showRoleGrants
-@init {pushMsg("show role grants", state);}
-@after {popMsg(state);}
-    : KW_SHOW KW_ROLE KW_GRANT principalName
-    -> ^(TOK_SHOW_ROLE_GRANT principalName)
-    ;
-
-
-showRoles
-@init {pushMsg("show roles", state);}
-@after {popMsg(state);}
-    : KW_SHOW KW_ROLES
-    -> ^(TOK_SHOW_ROLES)
-    ;
-
-showCurrentRole
-@init {pushMsg("show current role", state);}
-@after {popMsg(state);}
-    : KW_SHOW KW_CURRENT KW_ROLES
-    -> ^(TOK_SHOW_SET_ROLE)
-    ;
-
-setRole
-@init {pushMsg("set role", state);}
-@after {popMsg(state);}
-    : KW_SET KW_ROLE
-    (
-    (KW_ALL) => (all=KW_ALL) -> ^(TOK_SHOW_SET_ROLE Identifier[$all.text])
-    |
-    (KW_NONE) => (none=KW_NONE) -> ^(TOK_SHOW_SET_ROLE Identifier[$none.text])
-    |
-    identifier -> ^(TOK_SHOW_SET_ROLE identifier)
-    )
-    ;
-
-showGrants
-@init {pushMsg("show grants", state);}
-@after {popMsg(state);}
-    : KW_SHOW KW_GRANT principalName? (KW_ON privilegeIncludeColObject)?
-    -> ^(TOK_SHOW_GRANT principalName? privilegeIncludeColObject?)
-    ;
-
-showRolePrincipals
-@init {pushMsg("show role principals", state);}
-@after {popMsg(state);}
-    : KW_SHOW KW_PRINCIPALS roleName=identifier
-    -> ^(TOK_SHOW_ROLE_PRINCIPALS $roleName)
-    ;
-
-
-privilegeIncludeColObject
-@init {pushMsg("privilege object including columns", state);}
-@after {popMsg(state);}
-    : (KW_ALL) => KW_ALL -> ^(TOK_RESOURCE_ALL)
-    | privObjectCols -> ^(TOK_PRIV_OBJECT_COL privObjectCols)
-    ;
-
-privilegeObject
-@init {pushMsg("privilege object", state);}
-@after {popMsg(state);}
-    : KW_ON privObject -> ^(TOK_PRIV_OBJECT privObject)
-    ;
-
-// database or table type. Type is optional, default type is table
-privObject
-    : (KW_DATABASE|KW_SCHEMA) identifier -> ^(TOK_DB_TYPE identifier)
-    | KW_TABLE? tableName partitionSpec? -> ^(TOK_TABLE_TYPE tableName partitionSpec?)
-    | KW_URI (path=StringLiteral) ->  ^(TOK_URI_TYPE $path)
-    | KW_SERVER identifier -> ^(TOK_SERVER_TYPE identifier)
-    ;
-
-privObjectCols
-    : (KW_DATABASE|KW_SCHEMA) identifier -> ^(TOK_DB_TYPE identifier)
-    | KW_TABLE? tableName (LPAREN cols=columnNameList RPAREN)? partitionSpec? -> ^(TOK_TABLE_TYPE tableName $cols? partitionSpec?)
-    | KW_URI (path=StringLiteral) ->  ^(TOK_URI_TYPE $path)
-    | KW_SERVER identifier -> ^(TOK_SERVER_TYPE identifier)
-    ;
-
-privilegeList
-@init {pushMsg("grant privilege list", state);}
-@after {popMsg(state);}
-    : privlegeDef (COMMA privlegeDef)*
-    -> ^(TOK_PRIVILEGE_LIST privlegeDef+)
-    ;
-
-privlegeDef
-@init {pushMsg("grant privilege", state);}
-@after {popMsg(state);}
-    : privilegeType (LPAREN cols=columnNameList RPAREN)?
-    -> ^(TOK_PRIVILEGE privilegeType $cols?)
-    ;
-
-privilegeType
-@init {pushMsg("privilege type", state);}
-@after {popMsg(state);}
-    : KW_ALL -> ^(TOK_PRIV_ALL)
-    | KW_ALTER -> ^(TOK_PRIV_ALTER_METADATA)
-    | KW_UPDATE -> ^(TOK_PRIV_ALTER_DATA)
-    | KW_CREATE -> ^(TOK_PRIV_CREATE)
-    | KW_DROP -> ^(TOK_PRIV_DROP)
-    | KW_INDEX -> ^(TOK_PRIV_INDEX)
-    | KW_LOCK -> ^(TOK_PRIV_LOCK)
-    | KW_SELECT -> ^(TOK_PRIV_SELECT)
-    | KW_SHOW_DATABASE -> ^(TOK_PRIV_SHOW_DATABASE)
-    | KW_INSERT -> ^(TOK_PRIV_INSERT)
-    | KW_DELETE -> ^(TOK_PRIV_DELETE)
-    ;
-
-principalSpecification
-@init { pushMsg("user/group/role name list", state); }
-@after { popMsg(state); }
-    : principalName (COMMA principalName)* -> ^(TOK_PRINCIPAL_NAME principalName+)
-    ;
-
-principalName
-@init {pushMsg("user|group|role name", state);}
-@after {popMsg(state);}
-    : KW_USER principalIdentifier -> ^(TOK_USER principalIdentifier)
-    | KW_GROUP principalIdentifier -> ^(TOK_GROUP principalIdentifier)
-    | KW_ROLE identifier -> ^(TOK_ROLE identifier)
-    ;
-
-withGrantOption
-@init {pushMsg("with grant option", state);}
-@after {popMsg(state);}
-    : KW_WITH KW_GRANT KW_OPTION
-    -> ^(TOK_GRANT_WITH_OPTION)
-    ;
-
-grantOptionFor
-@init {pushMsg("grant option for", state);}
-@after {popMsg(state);}
-    : KW_GRANT KW_OPTION KW_FOR
-    -> ^(TOK_GRANT_OPTION_FOR)
-;
-
-adminOptionFor
-@init {pushMsg("admin option for", state);}
-@after {popMsg(state);}
-    : KW_ADMIN KW_OPTION KW_FOR
-    -> ^(TOK_ADMIN_OPTION_FOR)
-;
-
-withAdminOption
-@init {pushMsg("with admin option", state);}
-@after {popMsg(state);}
-    : KW_WITH KW_ADMIN KW_OPTION
-    -> ^(TOK_GRANT_WITH_ADMIN_OPTION)
-    ;
-
-metastoreCheck
-@init { pushMsg("metastore check statement", state); }
-@after { popMsg(state); }
-    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE tableName partitionSpec? (COMMA partitionSpec)*)?
-    -> ^(TOK_MSCK $repair? (tableName partitionSpec*)?)
-    ;
-
-resourceList
-@init { pushMsg("resource list", state); }
-@after { popMsg(state); }
-  :
-  resource (COMMA resource)* -> ^(TOK_RESOURCE_LIST resource+)
-  ;
-
-resource
-@init { pushMsg("resource", state); }
-@after { popMsg(state); }
-  :
-  resType=resourceType resPath=StringLiteral -> ^(TOK_RESOURCE_URI $resType $resPath)
-  ;
-
-resourceType
-@init { pushMsg("resource type", state); }
-@after { popMsg(state); }
-  :
-  KW_JAR -> ^(TOK_JAR)
-  |
-  KW_FILE -> ^(TOK_FILE)
-  |
-  KW_ARCHIVE -> ^(TOK_ARCHIVE)
-  ;
-
-createFunctionStatement
-@init { pushMsg("create function statement", state); }
-@after { popMsg(state); }
-    : KW_CREATE (temp=KW_TEMPORARY)? KW_FUNCTION functionIdentifier KW_AS StringLiteral
-      (KW_USING rList=resourceList)?
-    -> {$temp != null}? ^(TOK_CREATEFUNCTION functionIdentifier StringLiteral $rList? TOK_TEMPORARY)
-    ->                  ^(TOK_CREATEFUNCTION functionIdentifier StringLiteral $rList?)
-    ;
-
-dropFunctionStatement
-@init { pushMsg("drop function statement", state); }
-@after { popMsg(state); }
-    : KW_DROP (temp=KW_TEMPORARY)? KW_FUNCTION ifExists? functionIdentifier
-    -> {$temp != null}? ^(TOK_DROPFUNCTION functionIdentifier ifExists? TOK_TEMPORARY)
-    ->                  ^(TOK_DROPFUNCTION functionIdentifier ifExists?)
-    ;
-
-reloadFunctionStatement
-@init { pushMsg("reload function statement", state); }
-@after { popMsg(state); }
-    : KW_RELOAD KW_FUNCTION -> ^(TOK_RELOADFUNCTION);
-
-createMacroStatement
-@init { pushMsg("create macro statement", state); }
-@after { popMsg(state); }
-    : KW_CREATE KW_TEMPORARY KW_MACRO Identifier
-      LPAREN columnNameTypeList? RPAREN expression
-    -> ^(TOK_CREATEMACRO Identifier columnNameTypeList? expression)
-    ;
-
-dropMacroStatement
-@init { pushMsg("drop macro statement", state); }
-@after { popMsg(state); }
-    : KW_DROP KW_TEMPORARY KW_MACRO ifExists? Identifier
-    -> ^(TOK_DROPMACRO Identifier ifExists?)
-    ;
-
-createViewStatement
-@init {
-    pushMsg("create view statement", state);
-}
-@after { popMsg(state); }
-    : KW_CREATE (orReplace)? KW_VIEW (ifNotExists)? name=tableName
-        (LPAREN columnNameCommentList RPAREN)? tableComment? viewPartition?
-        tablePropertiesPrefixed?
-        KW_AS
-        selectStatementWithCTE
-    -> ^(TOK_CREATEVIEW $name orReplace?
-         ifNotExists?
-         columnNameCommentList?
-         tableComment?
-         viewPartition?
-         tablePropertiesPrefixed?
-         selectStatementWithCTE
-        )
-    ;
-
-viewPartition
-@init { pushMsg("view partition specification", state); }
-@after { popMsg(state); }
-    : KW_PARTITIONED KW_ON LPAREN columnNameList RPAREN
-    -> ^(TOK_VIEWPARTCOLS columnNameList)
-    ;
-
-dropViewStatement
-@init { pushMsg("drop view statement", state); }
-@after { popMsg(state); }
-    : KW_DROP KW_VIEW ifExists? viewName -> ^(TOK_DROPVIEW viewName ifExists?)
-    ;
-
-showFunctionIdentifier
-@init { pushMsg("identifier for show function statement", state); }
-@after { popMsg(state); }
-    : functionIdentifier
-    | StringLiteral
-    ;
-
-showStmtIdentifier
-@init { pushMsg("identifier for show statement", state); }
-@after { popMsg(state); }
-    : identifier
-    | StringLiteral
-    ;
-
-tableProvider
-@init { pushMsg("table's provider", state); }
-@after { popMsg(state); }
-    :
-      KW_USING Identifier (DOT Identifier)*
-    -> ^(TOK_TABLEPROVIDER Identifier+)
-    ;
-
-optionKeyValue
-@init { pushMsg("table's option specification", state); }
-@after { popMsg(state); }
-    :
-       (looseIdentifier (DOT looseIdentifier)*) StringLiteral
-    -> ^(TOK_TABLEOPTION looseIdentifier+ StringLiteral)
-    ;
-
-tableOpts
-@init { pushMsg("table's options", state); }
-@after { popMsg(state); }
-    :
-      KW_OPTIONS LPAREN optionKeyValue (COMMA optionKeyValue)* RPAREN
-    -> ^(TOK_TABLEOPTIONS optionKeyValue+)
-    ;
-
-tableComment
-@init { pushMsg("table's comment", state); }
-@after { popMsg(state); }
-    :
-      KW_COMMENT comment=StringLiteral  -> ^(TOK_TABLECOMMENT $comment)
-    ;
-
-tablePartition
-@init { pushMsg("table partition specification", state); }
-@after { popMsg(state); }
-    : KW_PARTITIONED KW_BY LPAREN columnNameTypeList RPAREN
-    -> ^(TOK_TABLEPARTCOLS columnNameTypeList)
-    ;
-
-tableBuckets
-@init { pushMsg("table buckets specification", state); }
-@after { popMsg(state); }
-    :
-      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS
-    -> ^(TOK_ALTERTABLE_BUCKETS $bucketCols $sortCols? $num)
-    ;
-
-tableSkewed
-@init { pushMsg("table skewed specification", state); }
-@after { popMsg(state); }
-    :
-     KW_SKEWED KW_BY LPAREN skewedCols=columnNameList RPAREN KW_ON LPAREN (skewedValues=skewedValueElement) RPAREN ((storedAsDirs) => storedAsDirs)?
-    -> ^(TOK_TABLESKEWED $skewedCols $skewedValues storedAsDirs?)
-    ;
-
-rowFormat
-@init { pushMsg("serde specification", state); }
-@after { popMsg(state); }
-    : rowFormatSerde -> ^(TOK_SERDE rowFormatSerde)
-    | rowFormatDelimited -> ^(TOK_SERDE rowFormatDelimited)
-    |   -> ^(TOK_SERDE)
-    ;
-
-recordReader
-@init { pushMsg("record reader specification", state); }
-@after { popMsg(state); }
-    : KW_RECORDREADER StringLiteral -> ^(TOK_RECORDREADER StringLiteral)
-    |   -> ^(TOK_RECORDREADER)
-    ;
-
-recordWriter
-@init { pushMsg("record writer specification", state); }
-@after { popMsg(state); }
-    : KW_RECORDWRITER StringLiteral -> ^(TOK_RECORDWRITER StringLiteral)
-    |   -> ^(TOK_RECORDWRITER)
-    ;
-
-rowFormatSerde
-@init { pushMsg("serde format specification", state); }
-@after { popMsg(state); }
-    : KW_ROW KW_FORMAT KW_SERDE name=StringLiteral (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
-    -> ^(TOK_SERDENAME $name $serdeprops?)
-    ;
-
-rowFormatDelimited
-@init { pushMsg("serde properties specification", state); }
-@after { popMsg(state); }
-    :
-      KW_ROW KW_FORMAT KW_DELIMITED tableRowFormatFieldIdentifier? tableRowFormatCollItemsIdentifier? tableRowFormatMapKeysIdentifier? tableRowFormatLinesIdentifier? tableRowNullFormat?
-    -> ^(TOK_SERDEPROPS tableRowFormatFieldIdentifier? tableRowFormatCollItemsIdentifier? tableRowFormatMapKeysIdentifier? tableRowFormatLinesIdentifier? tableRowNullFormat?)
-    ;
-
-tableRowFormat
-@init { pushMsg("table row format specification", state); }
-@after { popMsg(state); }
-    :
-      rowFormatDelimited
-    -> ^(TOK_TABLEROWFORMAT rowFormatDelimited)
-    | rowFormatSerde
-    -> ^(TOK_TABLESERIALIZER rowFormatSerde)
-    ;
-
-tablePropertiesPrefixed
-@init { pushMsg("table properties with prefix", state); }
-@after { popMsg(state); }
-    :
-        KW_TBLPROPERTIES! tableProperties
-    ;
-
-tableProperties
-@init { pushMsg("table properties", state); }
-@after { popMsg(state); }
-    :
-      LPAREN tablePropertiesList RPAREN -> ^(TOK_TABLEPROPERTIES tablePropertiesList)
-    ;
-
-tablePropertiesList
-@init { pushMsg("table properties list", state); }
-@after { popMsg(state); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* -> ^(TOK_TABLEPROPLIST keyValueProperty+)
-    |
-      keyProperty (COMMA keyProperty)* -> ^(TOK_TABLEPROPLIST keyProperty+)
-    ;
-
-keyValueProperty
-@init { pushMsg("specifying key/value property", state); }
-@after { popMsg(state); }
-    :
-      key=StringLiteral EQUAL value=StringLiteral -> ^(TOK_TABLEPROPERTY $key $value)
-    ;
-
-keyProperty
-@init { pushMsg("specifying key property", state); }
-@after { popMsg(state); }
-    :
-      key=StringLiteral -> ^(TOK_TABLEPROPERTY $key TOK_NULL)
-    ;
-
-tableRowFormatFieldIdentifier
-@init { pushMsg("table row format's field separator", state); }
-@after { popMsg(state); }
-    :
-      KW_FIELDS KW_TERMINATED KW_BY fldIdnt=StringLiteral (KW_ESCAPED KW_BY fldEscape=StringLiteral)?
-    -> ^(TOK_TABLEROWFORMATFIELD $fldIdnt $fldEscape?)
-    ;
-
-tableRowFormatCollItemsIdentifier
-@init { pushMsg("table row format's column separator", state); }
-@after { popMsg(state); }
-    :
-      KW_COLLECTION KW_ITEMS KW_TERMINATED KW_BY collIdnt=StringLiteral
-    -> ^(TOK_TABLEROWFORMATCOLLITEMS $collIdnt)
-    ;
-
-tableRowFormatMapKeysIdentifier
-@init { pushMsg("table row format's map key separator", state); }
-@after { popMsg(state); }
-    :
-      KW_MAP KW_KEYS KW_TERMINATED KW_BY mapKeysIdnt=StringLiteral
-    -> ^(TOK_TABLEROWFORMATMAPKEYS $mapKeysIdnt)
-    ;
-
-tableRowFormatLinesIdentifier
-@init { pushMsg("table row format's line separator", state); }
-@after { popMsg(state); }
-    :
-      KW_LINES KW_TERMINATED KW_BY linesIdnt=StringLiteral
-    -> ^(TOK_TABLEROWFORMATLINES $linesIdnt)
-    ;
-
-tableRowNullFormat
-@init { pushMsg("table row format's null specifier", state); }
-@after { popMsg(state); }
-    :
-      KW_NULL KW_DEFINED KW_AS nullIdnt=StringLiteral
-    -> ^(TOK_TABLEROWFORMATNULL $nullIdnt)
-    ;
-tableFileFormat
-@init { pushMsg("table file format specification", state); }
-@after { popMsg(state); }
-    :
-      (KW_STORED KW_AS KW_INPUTFORMAT) => KW_STORED KW_AS KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
-      -> ^(TOK_TABLEFILEFORMAT $inFmt $outFmt $inDriver? $outDriver?)
-      | KW_STORED KW_BY storageHandler=StringLiteral
-         (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
-      -> ^(TOK_STORAGEHANDLER $storageHandler $serdeprops?)
-      | KW_STORED KW_AS genericSpec=identifier
-      -> ^(TOK_FILEFORMAT_GENERIC $genericSpec)
-    ;
-
-tableLocation
-@init { pushMsg("table location specification", state); }
-@after { popMsg(state); }
-    :
-      KW_LOCATION locn=StringLiteral -> ^(TOK_TABLELOCATION $locn)
-    ;
-
-columnNameTypeList
-@init { pushMsg("column name type list", state); }
-@after { popMsg(state); }
-    : columnNameType (COMMA columnNameType)* -> ^(TOK_TABCOLLIST columnNameType+)
-    ;
-
-columnNameColonTypeList
-@init { pushMsg("column name type list", state); }
-@after { popMsg(state); }
-    : columnNameColonType (COMMA columnNameColonType)* -> ^(TOK_TABCOLLIST columnNameColonType+)
-    ;
-
-columnNameList
-@init { pushMsg("column name list", state); }
-@after { popMsg(state); }
-    : columnName (COMMA columnName)* -> ^(TOK_TABCOLNAME columnName+)
-    ;
-
-columnName
-@init { pushMsg("column name", state); }
-@after { popMsg(state); }
-    :
-      identifier
-    ;
-
-extColumnName
-@init { pushMsg("column name for complex types", state); }
-@after { popMsg(state); }
-    :
-      identifier (DOT^ ((KW_ELEM_TYPE) => KW_ELEM_TYPE | (KW_KEY_TYPE) => KW_KEY_TYPE | (KW_VALUE_TYPE) => KW_VALUE_TYPE | identifier))*
-    ;
-
-columnNameOrderList
-@init { pushMsg("column name order list", state); }
-@after { popMsg(state); }
-    : columnNameOrder (COMMA columnNameOrder)* -> ^(TOK_TABCOLNAME columnNameOrder+)
-    ;
-
-skewedValueElement
-@init { pushMsg("skewed value element", state); }
-@after { popMsg(state); }
-    :
-      skewedColumnValues
-     | skewedColumnValuePairList
-    ;
-
-skewedColumnValuePairList
-@init { pushMsg("column value pair list", state); }
-@after { popMsg(state); }
-    : skewedColumnValuePair (COMMA skewedColumnValuePair)* -> ^(TOK_TABCOLVALUE_PAIR skewedColumnValuePair+)
-    ;
-
-skewedColumnValuePair
-@init { pushMsg("column value pair", state); }
-@after { popMsg(state); }
-    :
-      LPAREN colValues=skewedColumnValues RPAREN
-      -> ^(TOK_TABCOLVALUES $colValues)
-    ;
-
-skewedColumnValues
-@init { pushMsg("column values", state); }
-@after { popMsg(state); }
-    : skewedColumnValue (COMMA skewedColumnValue)* -> ^(TOK_TABCOLVALUE skewedColumnValue+)
-    ;
-
-skewedColumnValue
-@init { pushMsg("column value", state); }
-@after { popMsg(state); }
-    :
-      constant
-    ;
-
-skewedValueLocationElement
-@init { pushMsg("skewed value location element", state); }
-@after { popMsg(state); }
-    :
-      skewedColumnValue
-     | skewedColumnValuePair
-    ;
-
-columnNameOrder
-@init { pushMsg("column name order", state); }
-@after { popMsg(state); }
-    : identifier (asc=KW_ASC | desc=KW_DESC)?
-    -> {$desc == null}? ^(TOK_TABSORTCOLNAMEASC identifier)
-    ->                  ^(TOK_TABSORTCOLNAMEDESC identifier)
-    ;
-
-columnNameCommentList
-@init { pushMsg("column name comment list", state); }
-@after { popMsg(state); }
-    : columnNameComment (COMMA columnNameComment)* -> ^(TOK_TABCOLNAME columnNameComment+)
-    ;
-
-columnNameComment
-@init { pushMsg("column name comment", state); }
-@after { popMsg(state); }
-    : colName=identifier (KW_COMMENT comment=StringLiteral)?
-    -> ^(TOK_TABCOL $colName TOK_NULL $comment?)
-    ;
-
-columnRefOrder
-@init { pushMsg("column order", state); }
-@after { popMsg(state); }
-    : expression (asc=KW_ASC | desc=KW_DESC)?
-    -> {$desc == null}? ^(TOK_TABSORTCOLNAMEASC expression)
-    ->                  ^(TOK_TABSORTCOLNAMEDESC expression)
-    ;
-
-columnNameType
-@init { pushMsg("column specification", state); }
-@after { popMsg(state); }
-    : colName=identifier colType (KW_COMMENT comment=StringLiteral)?
-    -> {containExcludedCharForCreateTableColumnName($colName.text)}? {throwColumnNameException()}
-    -> {$comment == null}? ^(TOK_TABCOL $colName colType)
-    ->                     ^(TOK_TABCOL $colName colType $comment)
-    ;
-
-columnNameColonType
-@init { pushMsg("column specification", state); }
-@after { popMsg(state); }
-    : colName=identifier COLON colType (KW_COMMENT comment=StringLiteral)?
-    -> {$comment == null}? ^(TOK_TABCOL $colName colType)
-    ->                     ^(TOK_TABCOL $colName colType $comment)
-    ;
-
-colType
-@init { pushMsg("column type", state); }
-@after { popMsg(state); }
-    : type
-    ;
-
-colTypeList
-@init { pushMsg("column type list", state); }
-@after { popMsg(state); }
-    : colType (COMMA colType)* -> ^(TOK_COLTYPELIST colType+)
-    ;
-
-type
-    : primitiveType
-    | listType
-    | structType
-    | mapType
-    | unionType;
-
-primitiveType
-@init { pushMsg("primitive type specification", state); }
-@after { popMsg(state); }
-    : KW_TINYINT       ->    TOK_TINYINT
-    | KW_SMALLINT      ->    TOK_SMALLINT
-    | KW_INT           ->    TOK_INT
-    | KW_BIGINT        ->    TOK_BIGINT
-    | KW_LONG          ->    TOK_BIGINT
-    | KW_BOOLEAN       ->    TOK_BOOLEAN
-    | KW_FLOAT         ->    TOK_FLOAT
-    | KW_DOUBLE        ->    TOK_DOUBLE
-    | KW_DATE          ->    TOK_DATE
-    | KW_DATETIME      ->    TOK_DATETIME
-    | KW_TIMESTAMP     ->    TOK_TIMESTAMP
-    // Uncomment to allow intervals as table column types
-    //| KW_INTERVAL KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH
-    //| KW_INTERVAL KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME
-    | KW_STRING        ->    TOK_STRING
-    | KW_BINARY        ->    TOK_BINARY
-    | KW_DECIMAL (LPAREN prec=Number (COMMA scale=Number)? RPAREN)? -> ^(TOK_DECIMAL $prec? $scale?)
-    | KW_VARCHAR LPAREN length=Number RPAREN      ->    ^(TOK_VARCHAR $length)
-    | KW_CHAR LPAREN length=Number RPAREN      ->    ^(TOK_CHAR $length)
-    ;
-
-listType
-@init { pushMsg("list type", state); }
-@after { popMsg(state); }
-    : KW_ARRAY LESSTHAN type GREATERTHAN   -> ^(TOK_LIST type)
-    ;
-
-structType
-@init { pushMsg("struct type", state); }
-@after { popMsg(state); }
-    : KW_STRUCT LESSTHAN columnNameColonTypeList GREATERTHAN -> ^(TOK_STRUCT columnNameColonTypeList)
-    ;
-
-mapType
-@init { pushMsg("map type", state); }
-@after { popMsg(state); }
-    : KW_MAP LESSTHAN left=type COMMA right=type GREATERTHAN
-    -> ^(TOK_MAP $left $right)
-    ;
-
-unionType
-@init { pushMsg("uniontype type", state); }
-@after { popMsg(state); }
-    : KW_UNIONTYPE LESSTHAN colTypeList GREATERTHAN -> ^(TOK_UNIONTYPE colTypeList)
-    ;
-
-setOperator
-@init { pushMsg("set operator", state); }
-@after { popMsg(state); }
-    : KW_UNION KW_ALL -> ^(TOK_UNIONALL)
-    | KW_UNION KW_DISTINCT? -> ^(TOK_UNIONDISTINCT)
-    | KW_EXCEPT -> ^(TOK_EXCEPT)
-    | KW_INTERSECT -> ^(TOK_INTERSECT)
-    ;
-
-queryStatementExpression[boolean topLevel]
-    :
-    /* Would be nice to do this as a gated semantic perdicate
-       But the predicate gets pushed as a lookahead decision.
-       Calling rule doesnot know about topLevel
-    */
-    (w=withClause {topLevel}?)?
-    queryStatementExpressionBody[topLevel] {
-      if ($w.tree != null) {
-      $queryStatementExpressionBody.tree.insertChild(0, $w.tree);
-      }
-    }
-    ->  queryStatementExpressionBody
-    ;
-
-queryStatementExpressionBody[boolean topLevel]
-    :
-    fromStatement[topLevel]
-    | regularBody[topLevel]
-    ;
-
-withClause
-  :
-  KW_WITH cteStatement (COMMA cteStatement)* -> ^(TOK_CTE cteStatement+)
-;
-
-cteStatement
-   :
-   identifier KW_AS LPAREN queryStatementExpression[false] RPAREN
-   -> ^(TOK_SUBQUERY queryStatementExpression identifier)
-;
-
-fromStatement[boolean topLevel]
-: (singleFromStatement  -> singleFromStatement)
-	(u=setOperator r=singleFromStatement
-	  -> ^($u {$fromStatement.tree} $r)
-	)*
-	 -> {u != null && topLevel}? ^(TOK_QUERY
-	       ^(TOK_FROM
-	         ^(TOK_SUBQUERY
-	           {$fromStatement.tree}
-	            {adaptor.create(Identifier, generateUnionAlias())}
-	           )
-	        )
-	       ^(TOK_INSERT
-	          ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-	          ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
-	        )
-	      )
-    -> {$fromStatement.tree}
-	;
-
-
-singleFromStatement
-    :
-    fromClause
-    ( b+=body )+ -> ^(TOK_QUERY fromClause body+)
-    ;
-
-/*
-The valuesClause rule below ensures that the parse tree for
-"insert into table FOO values (1,2),(3,4)" looks the same as
-"insert into table FOO select a,b from (values(1,2),(3,4)) as BAR(a,b)" which itself is made to look
-very similar to the tree for "insert into table FOO select a,b from BAR".  Since virtual table name
-is implicit, it's represented as TOK_ANONYMOUS.
-*/
-regularBody[boolean topLevel]
-   :
-   i=insertClause
-   (
-   s=selectStatement[topLevel]
-     {$s.tree.getFirstChildWithType(TOK_INSERT).replaceChildren(0, 0, $i.tree);} -> {$s.tree}
-     |
-     valuesClause
-      -> ^(TOK_QUERY
-            ^(TOK_FROM
-              ^(TOK_VIRTUAL_TABLE ^(TOK_VIRTUAL_TABREF ^(TOK_ANONYMOUS)) valuesClause)
-             )
-            ^(TOK_INSERT {$i.tree} ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF)))
-          )
-   )
-   |
-   selectStatement[topLevel]
-   ;
-
-selectStatement[boolean topLevel]
-   :
-   (
-   (
-   LPAREN
-   s=selectClause
-   f=fromClause?
-   w=whereClause?
-   g=groupByClause?
-   h=havingClause?
-   o=orderByClause?
-   c=clusterByClause?
-   d=distributeByClause?
-   sort=sortByClause?
-   win=window_clause?
-   l=limitClause?
-   RPAREN
-   |
-   s=selectClause
-   f=fromClause?
-   w=whereClause?
-   g=groupByClause?
-   h=havingClause?
-   o=orderByClause?
-   c=clusterByClause?
-   d=distributeByClause?
-   sort=sortByClause?
-   win=window_clause?
-   l=limitClause?
-   )
-   -> ^(TOK_QUERY $f? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-                     $s $w? $g? $h? $o? $c?
-                     $d? $sort? $win? $l?))
-   )
-   (set=setOpSelectStatement[$selectStatement.tree, topLevel])?
-   -> {set == null}?
-      {$selectStatement.tree}
-   -> {o==null && c==null && d==null && sort==null && l==null}?
-      {$set.tree}
-   -> {throwSetOpException()}
-   ;
-
-setOpSelectStatement[CommonTree t, boolean topLevel]
-   :
-   ((
-    u=setOperator LPAREN b=simpleSelectStatement RPAREN
-    |
-    u=setOperator b=simpleSelectStatement)
-   -> {$setOpSelectStatement.tree != null}?
-      ^($u {$setOpSelectStatement.tree} $b)
-   -> ^($u {$t} $b)
-   )+
-   o=orderByClause?
-   c=clusterByClause?
-   d=distributeByClause?
-   sort=sortByClause?
-   win=window_clause?
-   l=limitClause?
-   -> {o==null && c==null && d==null && sort==null && win==null && l==null && !topLevel}?
-      {$setOpSelectStatement.tree}
-   -> ^(TOK_QUERY
-          ^(TOK_FROM
-            ^(TOK_SUBQUERY
-              {$setOpSelectStatement.tree}
-              {adaptor.create(Identifier, generateUnionAlias())}
-             )
-          )
-          ^(TOK_INSERT
-             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-             ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
-             $o? $c? $d? $sort? $win? $l?
-          )
-       )
-   ;
-
-simpleSelectStatement
-   :
-   selectClause
-   fromClause?
-   whereClause?
-   groupByClause?
-   havingClause?
-   ((window_clause) => window_clause)?
-   -> ^(TOK_QUERY fromClause? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-                     selectClause whereClause? groupByClause? havingClause? window_clause?))
-   ;
-
-selectStatementWithCTE
-    :
-    (w=withClause)?
-    selectStatement[true] {
-      if ($w.tree != null) {
-      $selectStatement.tree.insertChild(0, $w.tree);
-      }
-    }
-    ->  selectStatement
-    ;
-
-body
-   :
-   insertClause
-   selectClause
-   lateralView?
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? -> ^(TOK_INSERT insertClause
-                     selectClause lateralView? whereClause? groupByClause? havingClause? orderByClause? clusterByClause?
-                     distributeByClause? sortByClause? window_clause? limitClause?)
-   |
-   selectClause
-   lateralView?
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? -> ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-                     selectClause lateralView? whereClause? groupByClause? havingClause? orderByClause? clusterByClause?
-                     distributeByClause? sortByClause? window_clause? limitClause?)
-   ;
-
-insertClause
-@init { pushMsg("insert clause", state); }
-@after { popMsg(state); }
-   :
-     KW_INSERT KW_OVERWRITE destination ifNotExists? -> ^(TOK_DESTINATION destination ifNotExists?)
-   | KW_INSERT KW_INTO KW_TABLE? tableOrPartition (LPAREN targetCols=columnNameList RPAREN)?
-       -> ^(TOK_INSERT_INTO tableOrPartition $targetCols?)
-   ;
-
-destination
-@init { pushMsg("destination specification", state); }
-@after { popMsg(state); }
-   :
-     (local = KW_LOCAL)? KW_DIRECTORY StringLiteral tableRowFormat? tableFileFormat?
-       -> ^(TOK_DIR StringLiteral $local? tableRowFormat? tableFileFormat?)
-   | KW_TABLE tableOrPartition -> tableOrPartition
-   ;
-
-limitClause
-@init { pushMsg("limit clause", state); }
-@after { popMsg(state); }
-   :
-   KW_LIMIT num=Number -> ^(TOK_LIMIT $num)
-   ;
-
-//DELETE FROM <tableName> WHERE ...;
-deleteStatement
-@init { pushMsg("delete statement", state); }
-@after { popMsg(state); }
-   :
-   KW_DELETE KW_FROM tableName (whereClause)? -> ^(TOK_DELETE_FROM tableName whereClause?)
-   ;
-
-/*SET <columName> = (3 + col2)*/
-columnAssignmentClause
-   :
-   tableOrColumn EQUAL^ precedencePlusExpression
-   ;
-
-/*SET col1 = 5, col2 = (4 + col4), ...*/
-setColumnsClause
-   :
-   KW_SET columnAssignmentClause (COMMA columnAssignmentClause)* -> ^(TOK_SET_COLUMNS_CLAUSE columnAssignmentClause* )
-   ;
-
-/*
-  UPDATE <table>
-  SET col1 = val1, col2 = val2... WHERE ...
-*/
-updateStatement
-@init { pushMsg("update statement", state); }
-@after { popMsg(state); }
-   :
-   KW_UPDATE tableName setColumnsClause whereClause? -> ^(TOK_UPDATE_TABLE tableName setColumnsClause whereClause?)
-   ;
-
-/*
-BEGIN user defined transaction boundaries; follows SQL 2003 standard exactly except for addition of
-"setAutoCommitStatement" which is not in the standard doc but is supported by most SQL engines.
-*/
-sqlTransactionStatement
-@init { pushMsg("transaction statement", state); }
-@after { popMsg(state); }
-  : startTransactionStatement
-  | commitStatement
-  | rollbackStatement
-  | setAutoCommitStatement
-  ;
-
-startTransactionStatement
-  :
-  KW_START KW_TRANSACTION ( transactionMode  ( COMMA transactionMode  )* )? -> ^(TOK_START_TRANSACTION transactionMode*)
-  ;
-
-transactionMode
-  :
-  isolationLevel
-  | transactionAccessMode -> ^(TOK_TXN_ACCESS_MODE transactionAccessMode)
-  ;
-
-transactionAccessMode
-  :
-  KW_READ KW_ONLY -> TOK_TXN_READ_ONLY
-  | KW_READ KW_WRITE -> TOK_TXN_READ_WRITE
-  ;
-
-isolationLevel
-  :
-  KW_ISOLATION KW_LEVEL levelOfIsolation -> ^(TOK_ISOLATION_LEVEL levelOfIsolation)
-  ;
-
-/*READ UNCOMMITTED | READ COMMITTED | REPEATABLE READ | SERIALIZABLE may be supported later*/
-levelOfIsolation
-  :
-  KW_SNAPSHOT -> TOK_ISOLATION_SNAPSHOT
-  ;
-
-commitStatement
-  :
-  KW_COMMIT ( KW_WORK )? -> TOK_COMMIT
-  ;
-
-rollbackStatement
-  :
-  KW_ROLLBACK ( KW_WORK )? -> TOK_ROLLBACK
-  ;
-setAutoCommitStatement
-  :
-  KW_SET KW_AUTOCOMMIT booleanValueTok -> ^(TOK_SET_AUTOCOMMIT booleanValueTok)
-  ;
-/*
-END user defined transaction boundaries
-*/
-
-/*
-Table Caching statements.
- */
-cacheStatement
-@init { pushMsg("cache statement", state); }
-@after { popMsg(state); }
-  :
-  cacheTableStatement
-  | uncacheTableStatement
-  | clearCacheStatement
-  ;
-
-cacheTableStatement
-  :
-  KW_CACHE (lazy=KW_LAZY)? KW_TABLE identifier (KW_AS selectStatementWithCTE)? -> ^(TOK_CACHETABLE identifier $lazy? selectStatementWithCTE?)
-  ;
-
-uncacheTableStatement
-  :
-  KW_UNCACHE KW_TABLE identifier -> ^(TOK_UNCACHETABLE identifier)
-  ;
-
-clearCacheStatement
-  :
-  KW_CLEAR KW_CACHE -> ^(TOK_CLEARCACHE)
-  ;
-


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[3/8] spark git commit: [SPARK-14211][SQL] Remove ANTLR3 based parser

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala
deleted file mode 100644
index 5a64c41..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala
+++ /dev/null
@@ -1,1452 +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.parser.ng
-
-import java.sql.{Date, Timestamp}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.antlr.v4.runtime.{ParserRuleContext, Token}
-import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
-import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
-import org.apache.spark.util.random.RandomSampler
-
-/**
- * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
- * TableIdentifier.
- */
-class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
-  import ParserUtils._
-
-  protected def typedVisit[T](ctx: ParseTree): T = {
-    ctx.accept(this).asInstanceOf[T]
-  }
-
-  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
-    visit(ctx.statement).asInstanceOf[LogicalPlan]
-  }
-
-  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
-    visitNamedExpression(ctx.namedExpression)
-  }
-
-  override def visitSingleTableIdentifier(
-      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
-    visitTableIdentifier(ctx.tableIdentifier)
-  }
-
-  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
-    visit(ctx.dataType).asInstanceOf[DataType]
-  }
-
-  /* ********************************************************************************************
-   * Plan parsing
-   * ******************************************************************************************** */
-  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
-
-  /**
-   * Make sure we do not try to create a plan for a native command.
-   */
-  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
-
-  /**
-   * Create a plan for a SHOW FUNCTIONS command.
-   */
-  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
-    import ctx._
-    if (qualifiedName != null) {
-      val names = qualifiedName().identifier().asScala.map(_.getText).toList
-      names match {
-        case db :: name :: Nil =>
-          ShowFunctions(Some(db), Some(name))
-        case name :: Nil =>
-          ShowFunctions(None, Some(name))
-        case _ =>
-          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
-      }
-    } else if (pattern != null) {
-      ShowFunctions(None, Some(string(pattern)))
-    } else {
-      ShowFunctions(None, None)
-    }
-  }
-
-  /**
-   * Create a plan for a DESCRIBE FUNCTION command.
-   */
-  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
-    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
-    DescribeFunction(functionName, ctx.EXTENDED != null)
-  }
-
-  /**
-   * Create a top-level plan with Common Table Expressions.
-   */
-  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
-    val query = plan(ctx.queryNoWith)
-
-    // Apply CTEs
-    query.optional(ctx.ctes) {
-      val ctes = ctx.ctes.namedQuery.asScala.map {
-        case nCtx =>
-          val namedQuery = visitNamedQuery(nCtx)
-          (namedQuery.alias, namedQuery)
-      }
-
-      // Check for duplicate names.
-      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
-        case (name, _) =>
-          throw new ParseException(
-            s"Name '$name' is used for multiple common table expressions", ctx)
-      }
-
-      With(query, ctes.toMap)
-    }
-  }
-
-  /**
-   * Create a named logical plan.
-   *
-   * This is only used for Common Table Expressions.
-   */
-  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
-    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
-  }
-
-  /**
-   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
-   * queries have the following SQL form:
-   * {{{
-   *   [WITH cte...]?
-   *   FROM src
-   *   [INSERT INTO tbl1 SELECT *]+
-   * }}}
-   * For example:
-   * {{{
-   *   FROM db.tbl1 A
-   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
-   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
-   * }}}
-   * This (Hive) feature cannot be combined with set-operators.
-   */
-  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
-    val from = visitFromClause(ctx.fromClause)
-
-    // Build the insert clauses.
-    val inserts = ctx.multiInsertQueryBody.asScala.map {
-      body =>
-        assert(body.querySpecification.fromClause == null,
-          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
-          body)
-
-        withQuerySpecification(body.querySpecification, from).
-          // Add organization statements.
-          optionalMap(body.queryOrganization)(withQueryResultClauses).
-          // Add insert.
-          optionalMap(body.insertInto())(withInsertInto)
-    }
-
-    // If there are multiple INSERTS just UNION them together into one query.
-    inserts match {
-      case Seq(query) => query
-      case queries => Union(queries)
-    }
-  }
-
-  /**
-   * Create a logical plan for a regular (single-insert) query.
-   */
-  override def visitSingleInsertQuery(
-      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
-    plan(ctx.queryTerm).
-      // Add organization statements.
-      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
-      // Add insert.
-      optionalMap(ctx.insertInto())(withInsertInto)
-  }
-
-  /**
-   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
-   */
-  private def withInsertInto(
-      ctx: InsertIntoContext,
-      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
-    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
-
-    InsertIntoTable(
-      UnresolvedRelation(tableIdent, None),
-      partitionKeys,
-      query,
-      ctx.OVERWRITE != null,
-      ctx.EXISTS != null)
-  }
-
-  /**
-   * Create a partition specification map.
-   */
-  override def visitPartitionSpec(
-      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
-    ctx.partitionVal.asScala.map { pVal =>
-      val name = pVal.identifier.getText.toLowerCase
-      val value = Option(pVal.constant).map(visitStringConstant)
-      name -> value
-    }.toMap
-  }
-
-  /**
-   * Create a partition specification map without optional values.
-   */
-  protected def visitNonOptionalPartitionSpec(
-      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
-    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
-  }
-
-  /**
-   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
-   * main purpose is to prevent slight differences due to back to back conversions i.e.:
-   * String -> Literal -> String.
-   */
-  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
-    ctx match {
-      case s: StringLiteralContext => createString(s)
-      case o => o.getText
-    }
-  }
-
-  /**
-   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
-   * clauses determine the shape (ordering/partitioning/rows) of the query result.
-   */
-  private def withQueryResultClauses(
-      ctx: QueryOrganizationContext,
-      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    import ctx._
-
-    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
-    val withOrder = if (
-      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
-      // ORDER BY ...
-      Sort(order.asScala.map(visitSortItem), global = true, query)
-    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
-      // SORT BY ...
-      Sort(sort.asScala.map(visitSortItem), global = false, query)
-    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
-      // DISTRIBUTE BY ...
-      RepartitionByExpression(expressionList(distributeBy), query)
-    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
-      // SORT BY ... DISTRIBUTE BY ...
-      Sort(
-        sort.asScala.map(visitSortItem),
-        global = false,
-        RepartitionByExpression(expressionList(distributeBy), query))
-    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
-      // CLUSTER BY ...
-      val expressions = expressionList(clusterBy)
-      Sort(
-        expressions.map(SortOrder(_, Ascending)),
-        global = false,
-        RepartitionByExpression(expressions, query))
-    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
-      // [EMPTY]
-      query
-    } else {
-      throw new ParseException(
-        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
-    }
-
-    // WINDOWS
-    val withWindow = withOrder.optionalMap(windows)(withWindows)
-
-    // LIMIT
-    withWindow.optional(limit) {
-      Limit(typedVisit(limit), withWindow)
-    }
-  }
-
-  /**
-   * Create a logical plan using a query specification.
-   */
-  override def visitQuerySpecification(
-      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
-    val from = OneRowRelation.optional(ctx.fromClause) {
-      visitFromClause(ctx.fromClause)
-    }
-    withQuerySpecification(ctx, from)
-  }
-
-  /**
-   * Add a query specification to a logical plan. The query specification is the core of the logical
-   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
-   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
-   *
-   * Note that query hints are ignored (both by the parser and the builder).
-   */
-  private def withQuerySpecification(
-      ctx: QuerySpecificationContext,
-      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    import ctx._
-
-    // WHERE
-    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
-      Filter(expression(ctx), plan)
-    }
-
-    // Expressions.
-    val expressions = Option(namedExpressionSeq).toSeq
-      .flatMap(_.namedExpression.asScala)
-      .map(typedVisit[Expression])
-
-    // Create either a transform or a regular query.
-    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
-    specType match {
-      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
-        // Transform
-
-        // Add where.
-        val withFilter = relation.optionalMap(where)(filter)
-
-        // Create the attributes.
-        val (attributes, schemaLess) = if (colTypeList != null) {
-          // Typed return columns.
-          (createStructType(colTypeList).toAttributes, false)
-        } else if (identifierSeq != null) {
-          // Untyped return columns.
-          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
-            AttributeReference(name, StringType, nullable = true)()
-          }
-          (attrs, false)
-        } else {
-          (Seq(AttributeReference("key", StringType)(),
-            AttributeReference("value", StringType)()), true)
-        }
-
-        // Create the transform.
-        ScriptTransformation(
-          expressions,
-          string(script),
-          attributes,
-          withFilter,
-          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
-
-      case SqlBaseParser.SELECT =>
-        // Regular select
-
-        // Add lateral views.
-        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
-
-        // Add where.
-        val withFilter = withLateralView.optionalMap(where)(filter)
-
-        // Add aggregation or a project.
-        val namedExpressions = expressions.map {
-          case e: NamedExpression => e
-          case e: Expression => UnresolvedAlias(e)
-        }
-        val withProject = if (aggregation != null) {
-          withAggregation(aggregation, namedExpressions, withFilter)
-        } else if (namedExpressions.nonEmpty) {
-          Project(namedExpressions, withFilter)
-        } else {
-          withFilter
-        }
-
-        // Having
-        val withHaving = withProject.optional(having) {
-          // 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(expression(having), BooleanType), withProject)
-        }
-
-        // Distinct
-        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
-          Distinct(withHaving)
-        } else {
-          withHaving
-        }
-
-        // Window
-        withDistinct.optionalMap(windows)(withWindows)
-    }
-  }
-
-  /**
-   * Create a (Hive based) [[ScriptInputOutputSchema]].
-   */
-  protected def withScriptIOSchema(
-      inRowFormat: RowFormatContext,
-      recordWriter: Token,
-      outRowFormat: RowFormatContext,
-      recordReader: Token,
-      schemaLess: Boolean): ScriptInputOutputSchema = null
-
-  /**
-   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
-   * separated) relations here, these get converted into a single plan by condition-less inner join.
-   */
-  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
-    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
-    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
-  }
-
-  /**
-   * Connect two queries by a Set operator.
-   *
-   * Supported Set operators are:
-   * - UNION [DISTINCT]
-   * - UNION ALL
-   * - EXCEPT [DISTINCT]
-   * - INTERSECT [DISTINCT]
-   */
-  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
-    val left = plan(ctx.left)
-    val right = plan(ctx.right)
-    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
-    ctx.operator.getType match {
-      case SqlBaseParser.UNION if all =>
-        Union(left, right)
-      case SqlBaseParser.UNION =>
-        Distinct(Union(left, right))
-      case SqlBaseParser.INTERSECT if all =>
-        throw new ParseException("INTERSECT ALL is not supported.", ctx)
-      case SqlBaseParser.INTERSECT =>
-        Intersect(left, right)
-      case SqlBaseParser.EXCEPT if all =>
-        throw new ParseException("EXCEPT ALL is not supported.", ctx)
-      case SqlBaseParser.EXCEPT =>
-        Except(left, right)
-    }
-  }
-
-  /**
-   * Add a [[WithWindowDefinition]] operator to a logical plan.
-   */
-  private def withWindows(
-      ctx: WindowsContext,
-      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    // Collect all window specifications defined in the WINDOW clause.
-    val baseWindowMap = ctx.namedWindow.asScala.map {
-      wCtx =>
-        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
-    }.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 windowMapView = baseWindowMap.mapValues {
-      case WindowSpecReference(name) =>
-        baseWindowMap.get(name) match {
-          case Some(spec: WindowSpecDefinition) =>
-            spec
-          case Some(ref) =>
-            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
-          case None =>
-            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
-        }
-      case spec: WindowSpecDefinition => spec
-    }
-
-    // Note that mapValues creates a view instead of materialized map. We force materialization by
-    // mapping over identity.
-    WithWindowDefinition(windowMapView.map(identity), query)
-  }
-
-  /**
-   * Add an [[Aggregate]] to a logical plan.
-   */
-  private def withAggregation(
-      ctx: AggregationContext,
-      selectExpressions: Seq[NamedExpression],
-      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    import ctx._
-    val groupByExpressions = expressionList(groupingExpressions)
-
-    if (GROUPING != null) {
-      // GROUP BY .... GROUPING SETS (...)
-      val expressionMap = groupByExpressions.zipWithIndex.toMap
-      val numExpressions = expressionMap.size
-      val mask = (1 << numExpressions) - 1
-      val masks = ctx.groupingSet.asScala.map {
-        _.expression.asScala.foldLeft(mask) {
-          case (bitmap, eCtx) =>
-            // Find the index of the expression.
-            val e = typedVisit[Expression](eCtx)
-            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
-              throw new ParseException(
-                s"$e doesn't show up in the GROUP BY list", ctx))
-            // 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 << (numExpressions - 1 - index))
-        }
-      }
-      GroupingSets(masks, groupByExpressions, query, selectExpressions)
-    } else {
-      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
-      val mappedGroupByExpressions = if (CUBE != null) {
-        Seq(Cube(groupByExpressions))
-      } else if (ROLLUP != null) {
-        Seq(Rollup(groupByExpressions))
-      } else {
-        groupByExpressions
-      }
-      Aggregate(mappedGroupByExpressions, selectExpressions, query)
-    }
-  }
-
-  /**
-   * Add a [[Generate]] (Lateral View) to a logical plan.
-   */
-  private def withGenerate(
-      query: LogicalPlan,
-      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
-    val expressions = expressionList(ctx.expression)
-
-    // Create the generator.
-    val generator = ctx.qualifiedName.getText.toLowerCase match {
-      case "explode" if expressions.size == 1 =>
-        Explode(expressions.head)
-      case "json_tuple" =>
-        JsonTuple(expressions)
-      case other =>
-        withGenerator(other, expressions, ctx)
-    }
-
-    Generate(
-      generator,
-      join = true,
-      outer = ctx.OUTER != null,
-      Some(ctx.tblName.getText.toLowerCase),
-      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
-      query)
-  }
-
-  /**
-   * Create a [[Generator]]. Override this method in order to support custom Generators.
-   */
-  protected def withGenerator(
-      name: String,
-      expressions: Seq[Expression],
-      ctx: LateralViewContext): Generator = {
-    throw new ParseException(s"Generator function '$name' is not supported", ctx)
-  }
-
-  /**
-   * Create a joins between two or more logical plans.
-   */
-  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
-    /** Build a join between two plans. */
-    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
-      val baseJoinType = ctx.joinType match {
-        case null => Inner
-        case jt if jt.FULL != null => FullOuter
-        case jt if jt.SEMI != null => LeftSemi
-        case jt if jt.LEFT != null => LeftOuter
-        case jt if jt.RIGHT != null => RightOuter
-        case _ => Inner
-      }
-
-      // Resolve the join type and join condition
-      val (joinType, condition) = Option(ctx.joinCriteria) match {
-        case Some(c) if c.USING != null =>
-          val columns = c.identifier.asScala.map { column =>
-            UnresolvedAttribute.quoted(column.getText)
-          }
-          (UsingJoin(baseJoinType, columns), None)
-        case Some(c) if c.booleanExpression != null =>
-          (baseJoinType, Option(expression(c.booleanExpression)))
-        case None if ctx.NATURAL != null =>
-          (NaturalJoin(baseJoinType), None)
-        case None =>
-          (baseJoinType, None)
-      }
-      Join(left, right, joinType, condition)
-    }
-
-    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
-    // first join clause is at the top. However fields of previously referenced tables can be used
-    // in following join clauses. The tree needs to be reversed in order to make this work.
-    var result = plan(ctx.left)
-    var current = ctx
-    while (current != null) {
-      current.right match {
-        case right: JoinRelationContext =>
-          result = join(current, result, plan(right.left))
-          current = right
-        case right =>
-          result = join(current, result, plan(right))
-          current = null
-      }
-    }
-    result
-  }
-
-  /**
-   * Add a [[Sample]] to a logical plan.
-   *
-   * This currently supports the following sampling methods:
-   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
-   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
-   * are defined as a number between 0 and 100.
-   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
-   */
-  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
-    // Create a sampled plan if we need one.
-    def sample(fraction: Double): Sample = {
-      // 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.
-      val eps = RandomSampler.roundingEpsilon
-      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
-        s"Sampling fraction ($fraction) must be on interval [0, 1]",
-        ctx)
-      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
-    }
-
-    ctx.sampleType.getType match {
-      case SqlBaseParser.ROWS =>
-        Limit(expression(ctx.expression), query)
-
-      case SqlBaseParser.PERCENTLIT =>
-        val fraction = ctx.percentage.getText.toDouble
-        sample(fraction / 100.0d)
-
-      case SqlBaseParser.BUCKET if ctx.ON != null =>
-        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
-
-      case SqlBaseParser.BUCKET =>
-        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
-    }
-  }
-
-  /**
-   * Create a logical plan for a sub-query.
-   */
-  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
-    plan(ctx.queryNoWith)
-  }
-
-  /**
-   * Create an un-aliased table reference. This is typically used for top-level table references,
-   * for example:
-   * {{{
-   *   INSERT INTO db.tbl2
-   *   TABLE db.tbl1
-   * }}}
-   */
-  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
-    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
-  }
-
-  /**
-   * Create an aliased table reference. This is typically used in FROM clauses.
-   */
-  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
-    val table = UnresolvedRelation(
-      visitTableIdentifier(ctx.tableIdentifier),
-      Option(ctx.identifier).map(_.getText))
-    table.optionalMap(ctx.sample)(withSample)
-  }
-
-  /**
-   * Create an inline table (a virtual table in Hive parlance).
-   */
-  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
-    // Get the backing expressions.
-    val expressions = ctx.expression.asScala.map { eCtx =>
-      val e = expression(eCtx)
-      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
-      e
-    }
-
-    // Validate and evaluate the rows.
-    val (structType, structConstructor) = expressions.head.dataType match {
-      case st: StructType =>
-        (st, (e: Expression) => e)
-      case dt =>
-        val st = CreateStruct(Seq(expressions.head)).dataType
-        (st, (e: Expression) => CreateStruct(Seq(e)))
-    }
-    val rows = expressions.map {
-      case expression =>
-        val safe = Cast(structConstructor(expression), structType)
-        safe.eval().asInstanceOf[InternalRow]
-    }
-
-    // Construct attributes.
-    val baseAttributes = structType.toAttributes.map(_.withNullability(true))
-    val attributes = if (ctx.identifierList != null) {
-      val aliases = visitIdentifierList(ctx.identifierList)
-      assert(aliases.size == baseAttributes.size,
-        "Number of aliases must match the number of fields in an inline table.", ctx)
-      baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
-    } else {
-      baseAttributes
-    }
-
-    // Create plan and add an alias if a name has been defined.
-    LocalRelation(attributes, rows).optionalMap(ctx.identifier)(aliasPlan)
-  }
-
-  /**
-   * Create an alias (SubqueryAlias) for a join relation. This is practically the same as
-   * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
-   * hooks.
-   */
-  override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
-    plan(ctx.relation).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan)
-  }
-
-  /**
-   * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as
-   * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different
-   * hooks.
-   */
-  override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
-    plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan)
-  }
-
-  /**
-   * Create an alias (SubqueryAlias) for a LogicalPlan.
-   */
-  private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = {
-    SubqueryAlias(alias.getText, plan)
-  }
-
-  /**
-   * Create a Sequence of Strings for a parenthesis enclosed alias list.
-   */
-  override def visitIdentifierList(ctx: IdentifierListContext): Seq[String] = withOrigin(ctx) {
-    visitIdentifierSeq(ctx.identifierSeq)
-  }
-
-  /**
-   * Create a Sequence of Strings for an identifier list.
-   */
-  override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) {
-    ctx.identifier.asScala.map(_.getText)
-  }
-
-  /* ********************************************************************************************
-   * Table Identifier parsing
-   * ******************************************************************************************** */
-  /**
-   * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern.
-   */
-  override def visitTableIdentifier(
-      ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) {
-    TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
-  }
-
-  /* ********************************************************************************************
-   * Expression parsing
-   * ******************************************************************************************** */
-  /**
-   * Create an expression from the given context. This method just passes the context on to the
-   * vistor and only takes care of typing (We assume that the visitor returns an Expression here).
-   */
-  protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx)
-
-  /**
-   * Create sequence of expressions from the given sequence of contexts.
-   */
-  private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = {
-    trees.asScala.map(expression)
-  }
-
-  /**
-   * Invert a boolean expression if it has a valid NOT clause.
-   */
-  private def invertIfNotDefined(expression: Expression, not: TerminalNode): Expression = {
-    if (not != null) {
-      Not(expression)
-    } else {
-      expression
-    }
-  }
-
-  /**
-   * Create a star (i.e. all) expression; this selects all elements (in the specified object).
-   * Both un-targeted (global) and targeted aliases are supported.
-   */
-  override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) {
-    UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText)))
-  }
-
-  /**
-   * Create an aliased expression if an alias is specified. Both single and multi-aliases are
-   * supported.
-   */
-  override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
-    val e = expression(ctx.expression)
-    if (ctx.identifier != null) {
-      Alias(e, ctx.identifier.getText)()
-    } else if (ctx.identifierList != null) {
-      MultiAlias(e, visitIdentifierList(ctx.identifierList))
-    } else {
-      e
-    }
-  }
-
-  /**
-   * Combine a number of boolean expressions into a balanced expression tree. These expressions are
-   * either combined by a logical [[And]] or a logical [[Or]].
-   *
-   * A balanced binary tree is created because regular left recursive trees cause considerable
-   * performance degradations and can cause stack overflows.
-   */
-  override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) {
-    val expressionType = ctx.operator.getType
-    val expressionCombiner = expressionType match {
-      case SqlBaseParser.AND => And.apply _
-      case SqlBaseParser.OR => Or.apply _
-    }
-
-    // Collect all similar left hand contexts.
-    val contexts = ArrayBuffer(ctx.right)
-    var current = ctx.left
-    def collectContexts: Boolean = current match {
-      case lbc: LogicalBinaryContext if lbc.operator.getType == expressionType =>
-        contexts += lbc.right
-        current = lbc.left
-        true
-      case _ =>
-        contexts += current
-        false
-    }
-    while (collectContexts) {
-      // No body - all updates take place in the collectContexts.
-    }
-
-    // Reverse the contexts to have them in the same sequence as in the SQL statement & turn them
-    // into expressions.
-    val expressions = contexts.reverse.map(expression)
-
-    // Create a balanced tree.
-    def reduceToExpressionTree(low: Int, high: Int): Expression = high - low match {
-      case 0 =>
-        expressions(low)
-      case 1 =>
-        expressionCombiner(expressions(low), expressions(high))
-      case x =>
-        val mid = low + x / 2
-        expressionCombiner(
-          reduceToExpressionTree(low, mid),
-          reduceToExpressionTree(mid + 1, high))
-    }
-    reduceToExpressionTree(0, expressions.size - 1)
-  }
-
-  /**
-   * Invert a boolean expression.
-   */
-  override def visitLogicalNot(ctx: LogicalNotContext): Expression = withOrigin(ctx) {
-    Not(expression(ctx.booleanExpression()))
-  }
-
-  /**
-   * Create a filtering correlated sub-query. This is not supported yet.
-   */
-  override def visitExists(ctx: ExistsContext): Expression = {
-    throw new ParseException("EXISTS clauses are not supported.", ctx)
-  }
-
-  /**
-   * Create a comparison expression. This compares two expressions. The following comparison
-   * operators are supported:
-   * - Equal: '=' or '=='
-   * - Null-safe Equal: '<=>'
-   * - Not Equal: '<>' or '!='
-   * - Less than: '<'
-   * - Less then or Equal: '<='
-   * - Greater than: '>'
-   * - Greater then or Equal: '>='
-   */
-  override def visitComparison(ctx: ComparisonContext): Expression = withOrigin(ctx) {
-    val left = expression(ctx.left)
-    val right = expression(ctx.right)
-    val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
-    operator.getSymbol.getType match {
-      case SqlBaseParser.EQ =>
-        EqualTo(left, right)
-      case SqlBaseParser.NSEQ =>
-        EqualNullSafe(left, right)
-      case SqlBaseParser.NEQ | SqlBaseParser.NEQJ =>
-        Not(EqualTo(left, right))
-      case SqlBaseParser.LT =>
-        LessThan(left, right)
-      case SqlBaseParser.LTE =>
-        LessThanOrEqual(left, right)
-      case SqlBaseParser.GT =>
-        GreaterThan(left, right)
-      case SqlBaseParser.GTE =>
-        GreaterThanOrEqual(left, right)
-    }
-  }
-
-  /**
-   * Create a BETWEEN expression. This tests if an expression lies with in the bounds set by two
-   * other expressions. The inverse can also be created.
-   */
-  override def visitBetween(ctx: BetweenContext): Expression = withOrigin(ctx) {
-    val value = expression(ctx.value)
-    val between = And(
-      GreaterThanOrEqual(value, expression(ctx.lower)),
-      LessThanOrEqual(value, expression(ctx.upper)))
-    invertIfNotDefined(between, ctx.NOT)
-  }
-
-  /**
-   * Create an IN expression. This tests if the value of the left hand side expression is
-   * contained by the sequence of expressions on the right hand side.
-   */
-  override def visitInList(ctx: InListContext): Expression = withOrigin(ctx) {
-    val in = In(expression(ctx.value), ctx.expression().asScala.map(expression))
-    invertIfNotDefined(in, ctx.NOT)
-  }
-
-  /**
-   * Create an IN expression, where the the right hand side is a query. This is unsupported.
-   */
-  override def visitInSubquery(ctx: InSubqueryContext): Expression = {
-    throw new ParseException("IN with a Sub-query is currently not supported.", ctx)
-  }
-
-  /**
-   * Create a (R)LIKE/REGEXP expression.
-   */
-  override def visitLike(ctx: LikeContext): Expression = {
-    val left = expression(ctx.value)
-    val right = expression(ctx.pattern)
-    val like = ctx.like.getType match {
-      case SqlBaseParser.LIKE =>
-        Like(left, right)
-      case SqlBaseParser.RLIKE =>
-        RLike(left, right)
-    }
-    invertIfNotDefined(like, ctx.NOT)
-  }
-
-  /**
-   * Create an IS (NOT) NULL expression.
-   */
-  override def visitNullPredicate(ctx: NullPredicateContext): Expression = withOrigin(ctx) {
-    val value = expression(ctx.value)
-    if (ctx.NOT != null) {
-      IsNotNull(value)
-    } else {
-      IsNull(value)
-    }
-  }
-
-  /**
-   * Create a binary arithmetic expression. The following arithmetic operators are supported:
-   * - Mulitplication: '*'
-   * - Division: '/'
-   * - Hive Long Division: 'DIV'
-   * - Modulo: '%'
-   * - Addition: '+'
-   * - Subtraction: '-'
-   * - Binary AND: '&'
-   * - Binary XOR
-   * - Binary OR: '|'
-   */
-  override def visitArithmeticBinary(ctx: ArithmeticBinaryContext): Expression = withOrigin(ctx) {
-    val left = expression(ctx.left)
-    val right = expression(ctx.right)
-    ctx.operator.getType match {
-      case SqlBaseParser.ASTERISK =>
-        Multiply(left, right)
-      case SqlBaseParser.SLASH =>
-        Divide(left, right)
-      case SqlBaseParser.PERCENT =>
-        Remainder(left, right)
-      case SqlBaseParser.DIV =>
-        Cast(Divide(left, right), LongType)
-      case SqlBaseParser.PLUS =>
-        Add(left, right)
-      case SqlBaseParser.MINUS =>
-        Subtract(left, right)
-      case SqlBaseParser.AMPERSAND =>
-        BitwiseAnd(left, right)
-      case SqlBaseParser.HAT =>
-        BitwiseXor(left, right)
-      case SqlBaseParser.PIPE =>
-        BitwiseOr(left, right)
-    }
-  }
-
-  /**
-   * Create a unary arithmetic expression. The following arithmetic operators are supported:
-   * - Plus: '+'
-   * - Minus: '-'
-   * - Bitwise Not: '~'
-   */
-  override def visitArithmeticUnary(ctx: ArithmeticUnaryContext): Expression = withOrigin(ctx) {
-    val value = expression(ctx.valueExpression)
-    ctx.operator.getType match {
-      case SqlBaseParser.PLUS =>
-        value
-      case SqlBaseParser.MINUS =>
-        UnaryMinus(value)
-      case SqlBaseParser.TILDE =>
-        BitwiseNot(value)
-    }
-  }
-
-  /**
-   * Create a [[Cast]] expression.
-   */
-  override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) {
-    Cast(expression(ctx.expression), typedVisit(ctx.dataType))
-  }
-
-  /**
-   * Create a (windowed) Function expression.
-   */
-  override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) {
-    // Create the function call.
-    val name = ctx.qualifiedName.getText
-    val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null)
-    val arguments = ctx.expression().asScala.map(expression) match {
-      case Seq(UnresolvedStar(None)) if name.toLowerCase == "count" && !isDistinct =>
-        // Transform COUNT(*) into COUNT(1). Move this to analysis?
-        Seq(Literal(1))
-      case expressions =>
-        expressions
-    }
-    val function = UnresolvedFunction(name, arguments, isDistinct)
-
-    // Check if the function is evaluated in a windowed context.
-    ctx.windowSpec match {
-      case spec: WindowRefContext =>
-        UnresolvedWindowExpression(function, visitWindowRef(spec))
-      case spec: WindowDefContext =>
-        WindowExpression(function, visitWindowDef(spec))
-      case _ => function
-    }
-  }
-
-  /**
-   * Create a reference to a window frame, i.e. [[WindowSpecReference]].
-   */
-  override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) {
-    WindowSpecReference(ctx.identifier.getText)
-  }
-
-  /**
-   * Create a window definition, i.e. [[WindowSpecDefinition]].
-   */
-  override def visitWindowDef(ctx: WindowDefContext): WindowSpecDefinition = withOrigin(ctx) {
-    // CLUSTER BY ... | PARTITION BY ... ORDER BY ...
-    val partition = ctx.partition.asScala.map(expression)
-    val order = ctx.sortItem.asScala.map(visitSortItem)
-
-    // RANGE/ROWS BETWEEN ...
-    val frameSpecOption = Option(ctx.windowFrame).map { frame =>
-      val frameType = frame.frameType.getType match {
-        case SqlBaseParser.RANGE => RangeFrame
-        case SqlBaseParser.ROWS => RowFrame
-      }
-
-      SpecifiedWindowFrame(
-        frameType,
-        visitFrameBound(frame.start),
-        Option(frame.end).map(visitFrameBound).getOrElse(CurrentRow))
-    }
-
-    WindowSpecDefinition(
-      partition,
-      order,
-      frameSpecOption.getOrElse(UnspecifiedFrame))
-  }
-
-  /**
-   * Create or resolve a [[FrameBoundary]]. Simple math expressions are allowed for Value
-   * Preceding/Following boundaries. These expressions must be constant (foldable) and return an
-   * integer value.
-   */
-  override def visitFrameBound(ctx: FrameBoundContext): FrameBoundary = withOrigin(ctx) {
-    // We currently only allow foldable integers.
-    def value: Int = {
-      val e = expression(ctx.expression)
-      assert(e.resolved && e.foldable && e.dataType == IntegerType,
-        "Frame bound value must be a constant integer.",
-        ctx)
-      e.eval().asInstanceOf[Int]
-    }
-
-    // Create the FrameBoundary
-    ctx.boundType.getType match {
-      case SqlBaseParser.PRECEDING if ctx.UNBOUNDED != null =>
-        UnboundedPreceding
-      case SqlBaseParser.PRECEDING =>
-        ValuePreceding(value)
-      case SqlBaseParser.CURRENT =>
-        CurrentRow
-      case SqlBaseParser.FOLLOWING if ctx.UNBOUNDED != null =>
-        UnboundedFollowing
-      case SqlBaseParser.FOLLOWING =>
-        ValueFollowing(value)
-    }
-  }
-
-  /**
-   * Create a [[CreateStruct]] expression.
-   */
-  override def visitRowConstructor(ctx: RowConstructorContext): Expression = withOrigin(ctx) {
-    CreateStruct(ctx.expression.asScala.map(expression))
-  }
-
-  /**
-   * Create a [[ScalarSubquery]] expression.
-   */
-  override def visitSubqueryExpression(
-      ctx: SubqueryExpressionContext): Expression = withOrigin(ctx) {
-    ScalarSubquery(plan(ctx.query))
-  }
-
-  /**
-   * Create a value based [[CaseWhen]] expression. This has the following SQL form:
-   * {{{
-   *   CASE [expression]
-   *    WHEN [value] THEN [expression]
-   *    ...
-   *    ELSE [expression]
-   *   END
-   * }}}
-   */
-  override def visitSimpleCase(ctx: SimpleCaseContext): Expression = withOrigin(ctx) {
-    val e = expression(ctx.valueExpression)
-    val branches = ctx.whenClause.asScala.map { wCtx =>
-      (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result))
-    }
-    CaseWhen(branches, Option(ctx.elseExpression).map(expression))
-  }
-
-  /**
-   * Create a condition based [[CaseWhen]] expression. This has the following SQL syntax:
-   * {{{
-   *   CASE
-   *    WHEN [predicate] THEN [expression]
-   *    ...
-   *    ELSE [expression]
-   *   END
-   * }}}
-   *
-   * @param ctx the parse tree
-   *    */
-  override def visitSearchedCase(ctx: SearchedCaseContext): Expression = withOrigin(ctx) {
-    val branches = ctx.whenClause.asScala.map { wCtx =>
-      (expression(wCtx.condition), expression(wCtx.result))
-    }
-    CaseWhen(branches, Option(ctx.elseExpression).map(expression))
-  }
-
-  /**
-   * Create a dereference expression. The return type depends on the type of the parent, this can
-   * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an
-   * [[UnresolvedExtractValue]] if the parent is some expression.
-   */
-  override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) {
-    val attr = ctx.fieldName.getText
-    expression(ctx.base) match {
-      case UnresolvedAttribute(nameParts) =>
-        UnresolvedAttribute(nameParts :+ attr)
-      case e =>
-        UnresolvedExtractValue(e, Literal(attr))
-    }
-  }
-
-  /**
-   * Create an [[UnresolvedAttribute]] expression.
-   */
-  override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) {
-    UnresolvedAttribute.quoted(ctx.getText)
-  }
-
-  /**
-   * Create an [[UnresolvedExtractValue]] expression, this is used for subscript access to an array.
-   */
-  override def visitSubscript(ctx: SubscriptContext): Expression = withOrigin(ctx) {
-    UnresolvedExtractValue(expression(ctx.value), expression(ctx.index))
-  }
-
-  /**
-   * Create an expression for an expression between parentheses. This is need because the ANTLR
-   * visitor cannot automatically convert the nested context into an expression.
-   */
-  override def visitParenthesizedExpression(
-     ctx: ParenthesizedExpressionContext): Expression = withOrigin(ctx) {
-    expression(ctx.expression)
-  }
-
-  /**
-   * Create a [[SortOrder]] expression.
-   */
-  override def visitSortItem(ctx: SortItemContext): SortOrder = withOrigin(ctx) {
-    if (ctx.DESC != null) {
-      SortOrder(expression(ctx.expression), Descending)
-    } else {
-      SortOrder(expression(ctx.expression), Ascending)
-    }
-  }
-
-  /**
-   * Create a typed Literal expression. A typed literal has the following SQL syntax:
-   * {{{
-   *   [TYPE] '[VALUE]'
-   * }}}
-   * Currently Date and Timestamp typed literals are supported.
-   *
-   * TODO what the added value of this over casting?
-   */
-  override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) {
-    val value = string(ctx.STRING)
-    ctx.identifier.getText.toUpperCase match {
-      case "DATE" =>
-        Literal(Date.valueOf(value))
-      case "TIMESTAMP" =>
-        Literal(Timestamp.valueOf(value))
-      case other =>
-        throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx)
-    }
-  }
-
-  /**
-   * Create a NULL literal expression.
-   */
-  override def visitNullLiteral(ctx: NullLiteralContext): Literal = withOrigin(ctx) {
-    Literal(null)
-  }
-
-  /**
-   * Create a Boolean literal expression.
-   */
-  override def visitBooleanLiteral(ctx: BooleanLiteralContext): Literal = withOrigin(ctx) {
-    if (ctx.getText.toBoolean) {
-      Literal.TrueLiteral
-    } else {
-      Literal.FalseLiteral
-    }
-  }
-
-  /**
-   * Create an integral literal expression. The code selects the most narrow integral type
-   * possible, either a BigDecimal, a Long or an Integer is returned.
-   */
-  override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) {
-    BigDecimal(ctx.getText) match {
-      case v if v.isValidInt =>
-        Literal(v.intValue())
-      case v if v.isValidLong =>
-        Literal(v.longValue())
-      case v => Literal(v.underlying())
-    }
-  }
-
-  /**
-   * Create a double literal for a number denoted in scientifc notation.
-   */
-  override def visitScientificDecimalLiteral(
-      ctx: ScientificDecimalLiteralContext): Literal = withOrigin(ctx) {
-    Literal(ctx.getText.toDouble)
-  }
-
-  /**
-   * Create a decimal literal for a regular decimal number.
-   */
-  override def visitDecimalLiteral(ctx: DecimalLiteralContext): Literal = withOrigin(ctx) {
-    Literal(BigDecimal(ctx.getText).underlying())
-  }
-
-  /** Create a numeric literal expression. */
-  private def numericLiteral(ctx: NumberContext)(f: String => Any): Literal = withOrigin(ctx) {
-    val raw = ctx.getText
-    try {
-      Literal(f(raw.substring(0, raw.length - 1)))
-    } catch {
-      case e: NumberFormatException =>
-        throw new ParseException(e.getMessage, ctx)
-    }
-  }
-
-  /**
-   * Create a Byte Literal expression.
-   */
-  override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = numericLiteral(ctx) {
-    _.toByte
-  }
-
-  /**
-   * Create a Short Literal expression.
-   */
-  override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = numericLiteral(ctx) {
-    _.toShort
-  }
-
-  /**
-   * Create a Long Literal expression.
-   */
-  override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = numericLiteral(ctx) {
-    _.toLong
-  }
-
-  /**
-   * Create a Double Literal expression.
-   */
-  override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = numericLiteral(ctx) {
-    _.toDouble
-  }
-
-  /**
-   * Create a String literal expression.
-   */
-  override def visitStringLiteral(ctx: StringLiteralContext): Literal = withOrigin(ctx) {
-    Literal(createString(ctx))
-  }
-
-  /**
-   * Create a String from a string literal context. This supports multiple consecutive string
-   * literals, these are concatenated, for example this expression "'hello' 'world'" will be
-   * converted into "helloworld".
-   *
-   * Special characters can be escaped by using Hive/C-style escaping.
-   */
-  private def createString(ctx: StringLiteralContext): String = {
-    ctx.STRING().asScala.map(string).mkString
-  }
-
-  /**
-   * Create a [[CalendarInterval]] literal expression. An interval expression can contain multiple
-   * unit value pairs, for instance: interval 2 months 2 days.
-   */
-  override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) {
-    val intervals = ctx.intervalField.asScala.map(visitIntervalField)
-    assert(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx)
-    Literal(intervals.reduce(_.add(_)))
-  }
-
-  /**
-   * Create a [[CalendarInterval]] for a unit value pair. Two unit configuration types are
-   * supported:
-   * - Single unit.
-   * - From-To unit (only 'YEAR TO MONTH' and 'DAY TO SECOND' are supported).
-   */
-  override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) {
-    import ctx._
-    val s = value.getText
-    val interval = (unit.getText.toLowerCase, Option(to).map(_.getText.toLowerCase)) match {
-      case (u, None) if u.endsWith("s") =>
-        // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/...
-        CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s)
-      case (u, None) =>
-        CalendarInterval.fromSingleUnitString(u, s)
-      case ("year", Some("month")) =>
-        CalendarInterval.fromYearMonthString(s)
-      case ("day", Some("second")) =>
-        CalendarInterval.fromDayTimeString(s)
-      case (from, Some(t)) =>
-        throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx)
-    }
-    assert(interval != null, "No interval can be constructed", ctx)
-    interval
-  }
-
-  /* ********************************************************************************************
-   * DataType parsing
-   * ******************************************************************************************** */
-  /**
-   * Resolve/create a primitive type.
-   */
-  override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) {
-    (ctx.identifier.getText.toLowerCase, ctx.INTEGER_VALUE().asScala.toList) match {
-      case ("boolean", Nil) => BooleanType
-      case ("tinyint" | "byte", Nil) => ByteType
-      case ("smallint" | "short", Nil) => ShortType
-      case ("int" | "integer", Nil) => IntegerType
-      case ("bigint" | "long", Nil) => LongType
-      case ("float", Nil) => FloatType
-      case ("double", Nil) => DoubleType
-      case ("date", Nil) => DateType
-      case ("timestamp", Nil) => TimestampType
-      case ("char" | "varchar" | "string", Nil) => StringType
-      case ("char" | "varchar", _ :: Nil) => StringType
-      case ("binary", Nil) => BinaryType
-      case ("decimal", Nil) => DecimalType.USER_DEFAULT
-      case ("decimal", precision :: Nil) => DecimalType(precision.getText.toInt, 0)
-      case ("decimal", precision :: scale :: Nil) =>
-        DecimalType(precision.getText.toInt, scale.getText.toInt)
-      case (dt, params) =>
-        throw new ParseException(
-          s"DataType $dt${params.mkString("(", ",", ")")} is not supported.", ctx)
-    }
-  }
-
-  /**
-   * Create a complex DataType. Arrays, Maps and Structures are supported.
-   */
-  override def visitComplexDataType(ctx: ComplexDataTypeContext): DataType = withOrigin(ctx) {
-    ctx.complex.getType match {
-      case SqlBaseParser.ARRAY =>
-        ArrayType(typedVisit(ctx.dataType(0)))
-      case SqlBaseParser.MAP =>
-        MapType(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1)))
-      case SqlBaseParser.STRUCT =>
-        createStructType(ctx.colTypeList())
-    }
-  }
-
-  /**
-    * Create a [[StructType]] from a sequence of [[StructField]]s.
-    */
-  protected def createStructType(ctx: ColTypeListContext): StructType = {
-    StructType(Option(ctx).toSeq.flatMap(visitColTypeList))
-  }
-
-  /**
-   * Create a [[StructType]] from a number of column definitions.
-   */
-  override def visitColTypeList(ctx: ColTypeListContext): Seq[StructField] = withOrigin(ctx) {
-    ctx.colType().asScala.map(visitColType)
-  }
-
-  /**
-   * Create a [[StructField]] from a column definition.
-   */
-  override def visitColType(ctx: ColTypeContext): StructField = withOrigin(ctx) {
-    import ctx._
-
-    // Add the comment to the metadata.
-    val builder = new MetadataBuilder
-    if (STRING != null) {
-      builder.putString("comment", string(STRING))
-    }
-
-    StructField(identifier.getText, typedVisit(dataType), nullable = true, builder.build())
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParseDriver.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParseDriver.scala
deleted file mode 100644
index c9a2863..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParseDriver.scala
+++ /dev/null
@@ -1,240 +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.parser.ng
-
-import org.antlr.v4.runtime._
-import org.antlr.v4.runtime.atn.PredictionMode
-import org.antlr.v4.runtime.misc.ParseCancellationException
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.parser.ParserInterface
-import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.trees.Origin
-import org.apache.spark.sql.types.DataType
-
-/**
- * Base SQL parsing infrastructure.
- */
-abstract class AbstractSqlParser extends ParserInterface with Logging {
-
-  /** Creates/Resolves DataType for a given SQL string. */
-  def parseDataType(sqlText: String): DataType = parse(sqlText) { parser =>
-    // TODO add this to the parser interface.
-    astBuilder.visitSingleDataType(parser.singleDataType())
-  }
-
-  /** Creates Expression for a given SQL string. */
-  override def parseExpression(sqlText: String): Expression = parse(sqlText) { parser =>
-    astBuilder.visitSingleExpression(parser.singleExpression())
-  }
-
-  /** Creates TableIdentifier for a given SQL string. */
-  override def parseTableIdentifier(sqlText: String): TableIdentifier = parse(sqlText) { parser =>
-    astBuilder.visitSingleTableIdentifier(parser.singleTableIdentifier())
-  }
-
-  /** Creates LogicalPlan for a given SQL string. */
-  override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
-    astBuilder.visitSingleStatement(parser.singleStatement()) match {
-      case plan: LogicalPlan => plan
-      case _ => nativeCommand(sqlText)
-    }
-  }
-
-  /** Get the builder (visitor) which converts a ParseTree into a AST. */
-  protected def astBuilder: AstBuilder
-
-  /** Create a native command, or fail when this is not supported. */
-  protected def nativeCommand(sqlText: String): LogicalPlan = {
-    val position = Origin(None, None)
-    throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position)
-  }
-
-  protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
-    logInfo(s"Parsing command: $command")
-
-    val lexer = new SqlBaseLexer(new ANTLRNoCaseStringStream(command))
-    lexer.removeErrorListeners()
-    lexer.addErrorListener(ParseErrorListener)
-
-    val tokenStream = new CommonTokenStream(lexer)
-    val parser = new SqlBaseParser(tokenStream)
-    parser.addParseListener(PostProcessor)
-    parser.removeErrorListeners()
-    parser.addErrorListener(ParseErrorListener)
-
-    try {
-      try {
-        // first, try parsing with potentially faster SLL mode
-        parser.getInterpreter.setPredictionMode(PredictionMode.SLL)
-        toResult(parser)
-      }
-      catch {
-        case e: ParseCancellationException =>
-          // if we fail, parse with LL mode
-          tokenStream.reset() // rewind input stream
-          parser.reset()
-
-          // Try Again.
-          parser.getInterpreter.setPredictionMode(PredictionMode.LL)
-          toResult(parser)
-      }
-    }
-    catch {
-      case e: ParseException if e.command.isDefined =>
-        throw e
-      case e: ParseException =>
-        throw e.withCommand(command)
-      case e: AnalysisException =>
-        val position = Origin(e.line, e.startPosition)
-        throw new ParseException(Option(command), e.message, position, position)
-    }
-  }
-}
-
-/**
- * Concrete SQL parser for Catalyst-only SQL statements.
- */
-object CatalystSqlParser extends AbstractSqlParser {
-  val astBuilder = new AstBuilder
-}
-
-/**
- * This string stream provides the lexer with upper case characters only. This greatly simplifies
- * lexing the stream, while we can maintain the original command.
- *
- * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver.ANTLRNoCaseStringStream
- *
- * The comment below (taken from the original class) describes the rationale for doing this:
- *
- * This class provides and implementation for a case insensitive token checker for the lexical
- * analysis part of antlr. By converting the token stream into upper case at the time when lexical
- * rules are checked, this class ensures that the lexical rules need to just match the token with
- * upper case letters as opposed to combination of upper case and lower case characters. This is
- * purely used for matching lexical rules. The actual token text is stored in the same way as the
- * user input without actually converting it into an upper case. The token values are generated by
- * the consume() function of the super class ANTLRStringStream. The LA() function is the lookahead
- * function and is purely used for matching lexical rules. This also means that the grammar will
- * only accept capitalized tokens in case it is run from other tools like antlrworks which do not
- * have the ANTLRNoCaseStringStream implementation.
- */
-
-private[parser] class ANTLRNoCaseStringStream(input: String) extends ANTLRInputStream(input) {
-  override def LA(i: Int): Int = {
-    val la = super.LA(i)
-    if (la == 0 || la == IntStream.EOF) la
-    else Character.toUpperCase(la)
-  }
-}
-
-/**
- * The ParseErrorListener converts parse errors into AnalysisExceptions.
- */
-case object ParseErrorListener extends BaseErrorListener {
-  override def syntaxError(
-      recognizer: Recognizer[_, _],
-      offendingSymbol: scala.Any,
-      line: Int,
-      charPositionInLine: Int,
-      msg: String,
-      e: RecognitionException): Unit = {
-    val position = Origin(Some(line), Some(charPositionInLine))
-    throw new ParseException(None, msg, position, position)
-  }
-}
-
-/**
- * A [[ParseException]] is an [[AnalysisException]] that is thrown during the parse process. It
- * contains fields and an extended error message that make reporting and diagnosing errors easier.
- */
-class ParseException(
-    val command: Option[String],
-    message: String,
-    val start: Origin,
-    val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) {
-
-  def this(message: String, ctx: ParserRuleContext) = {
-    this(Option(ParserUtils.command(ctx)),
-      message,
-      ParserUtils.position(ctx.getStart),
-      ParserUtils.position(ctx.getStop))
-  }
-
-  override def getMessage: String = {
-    val builder = new StringBuilder
-    builder ++= "\n" ++= message
-    start match {
-      case Origin(Some(l), Some(p)) =>
-        builder ++= s"(line $l, pos $p)\n"
-        command.foreach { cmd =>
-          val (above, below) = cmd.split("\n").splitAt(l)
-          builder ++= "\n== SQL ==\n"
-          above.foreach(builder ++= _ += '\n')
-          builder ++= (0 until p).map(_ => "-").mkString("") ++= "^^^\n"
-          below.foreach(builder ++= _ += '\n')
-        }
-      case _ =>
-        command.foreach { cmd =>
-          builder ++= "\n== SQL ==\n" ++= cmd
-        }
-    }
-    builder.toString
-  }
-
-  def withCommand(cmd: String): ParseException = {
-    new ParseException(Option(cmd), message, start, stop)
-  }
-}
-
-/**
- * The post-processor validates & cleans-up the parse tree during the parse process.
- */
-case object PostProcessor extends SqlBaseBaseListener {
-
-  /** Remove the back ticks from an Identifier. */
-  override def exitQuotedIdentifier(ctx: QuotedIdentifierContext): Unit = {
-    replaceTokenByIdentifier(ctx, 1) { token =>
-      // Remove the double back ticks in the string.
-      token.setText(token.getText.replace("``", "`"))
-      token
-    }
-  }
-
-  /** Treat non-reserved keywords as Identifiers. */
-  override def exitNonReserved(ctx: NonReservedContext): Unit = {
-    replaceTokenByIdentifier(ctx, 0)(identity)
-  }
-
-  private def replaceTokenByIdentifier(
-      ctx: ParserRuleContext,
-      stripMargins: Int)(
-      f: CommonToken => CommonToken = identity): Unit = {
-    val parent = ctx.getParent
-    parent.removeLastChild()
-    val token = ctx.getChild(0).getPayload.asInstanceOf[Token]
-    parent.addChild(f(new CommonToken(
-      new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream),
-      SqlBaseParser.IDENTIFIER,
-      token.getChannel,
-      token.getStartIndex + stripMargins,
-      token.getStopIndex - stripMargins)))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParserUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParserUtils.scala
deleted file mode 100644
index 1fbfa76..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParserUtils.scala
+++ /dev/null
@@ -1,118 +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.parser.ng
-
-import org.antlr.v4.runtime.{CharStream, ParserRuleContext, Token}
-import org.antlr.v4.runtime.misc.Interval
-import org.antlr.v4.runtime.tree.TerminalNode
-
-import org.apache.spark.sql.catalyst.parser.ParseUtils.unescapeSQLString
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
-
-/**
- * A collection of utility methods for use during the parsing process.
- */
-object ParserUtils {
-  /** Get the command which created the token. */
-  def command(ctx: ParserRuleContext): String = {
-    command(ctx.getStart.getInputStream)
-  }
-
-  /** Get the command which created the token. */
-  def command(stream: CharStream): String = {
-    stream.getText(Interval.of(0, stream.size()))
-  }
-
-  /** Get the code that creates the given node. */
-  def source(ctx: ParserRuleContext): String = {
-    val stream = ctx.getStart.getInputStream
-    stream.getText(Interval.of(ctx.getStart.getStartIndex, ctx.getStop.getStopIndex))
-  }
-
-  /** Get all the text which comes after the given rule. */
-  def remainder(ctx: ParserRuleContext): String = remainder(ctx.getStop)
-
-  /** Get all the text which comes after the given token. */
-  def remainder(token: Token): String = {
-    val stream = token.getInputStream
-    val interval = Interval.of(token.getStopIndex + 1, stream.size())
-    stream.getText(interval)
-  }
-
-  /** Convert a string token into a string. */
-  def string(token: Token): String = unescapeSQLString(token.getText)
-
-  /** Convert a string node into a string. */
-  def string(node: TerminalNode): String = unescapeSQLString(node.getText)
-
-  /** Get the origin (line and position) of the token. */
-  def position(token: Token): Origin = {
-    Origin(Option(token.getLine), Option(token.getCharPositionInLine))
-  }
-
-  /** Assert if a condition holds. If it doesn't throw a parse exception. */
-  def assert(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = {
-    if (!f) {
-      throw new ParseException(message, ctx)
-    }
-  }
-
-  /**
-   * Register the origin of the context. Any TreeNode created in the closure will be assigned the
-   * registered origin. This method restores the previously set origin after completion of the
-   * closure.
-   */
-  def withOrigin[T](ctx: ParserRuleContext)(f: => T): T = {
-    val current = CurrentOrigin.get
-    CurrentOrigin.set(position(ctx.getStart))
-    try {
-      f
-    } finally {
-      CurrentOrigin.set(current)
-    }
-  }
-
-  /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */
-  implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal {
-    /**
-     * Create a plan using the block of code when the given context exists. Otherwise return the
-     * original plan.
-     */
-    def optional(ctx: AnyRef)(f: => LogicalPlan): LogicalPlan = {
-      if (ctx != null) {
-        f
-      } else {
-        plan
-      }
-    }
-
-    /**
-     * Map a [[LogicalPlan]] to another [[LogicalPlan]] if the passed context exists using the
-     * passed function. The original plan is returned when the context does not exist.
-     */
-    def optionalMap[C <: ParserRuleContext](
-        ctx: C)(
-        f: (C, LogicalPlan) => LogicalPlan): LogicalPlan = {
-      if (ctx != null) {
-        f(ctx, plan)
-      } else {
-        plan
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ASTNodeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ASTNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ASTNodeSuite.scala
deleted file mode 100644
index 8b05f9e..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ASTNodeSuite.scala
+++ /dev/null
@@ -1,38 +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.parser
-
-import org.apache.spark.SparkFunSuite
-
-class ASTNodeSuite extends SparkFunSuite {
-  test("SPARK-13157 - remainder must return all input chars") {
-    val inputs = Seq(
-      ("add jar", "file:///tmp/ab/TestUDTF.jar"),
-      ("add jar", "file:///tmp/a@b/TestUDTF.jar"),
-      ("add jar", "c:\\windows32\\TestUDTF.jar"),
-      ("add jar", "some \nbad\t\tfile\r\n.\njar"),
-      ("ADD JAR", "@*#&@(!#@$^*!@^@#(*!@#"),
-      ("SET", "foo=bar"),
-      ("SET", "foo*)(@#^*@&!#^=bar")
-    )
-    inputs.foreach {
-      case (command, arguments) =>
-        val node = ParseDriver.parsePlan(s"$command $arguments", null)
-        assert(node.remainder === arguments)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala
deleted file mode 100644
index 223485e..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala
+++ /dev/null
@@ -1,223 +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.parser
-
-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.parser.ng.CatalystSqlParser
-import org.apache.spark.sql.catalyst.plans.PlanTest
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.unsafe.types.CalendarInterval
-
-class CatalystQlSuite extends PlanTest {
-  val parser = new CatalystQl()
-  import org.apache.spark.sql.catalyst.dsl.expressions._
-  import org.apache.spark.sql.catalyst.dsl.plans._
-
-  val star = UnresolvedAlias(UnresolvedStar(None))
-
-  test("test case insensitive") {
-    val result = OneRowRelation.select(1)
-    assert(result === parser.parsePlan("seLect 1"))
-    assert(result === parser.parsePlan("select 1"))
-    assert(result === parser.parsePlan("SELECT 1"))
-  }
-
-  test("test NOT operator with comparison operations") {
-    val parsed = parser.parsePlan("SELECT NOT TRUE > TRUE")
-    val expected = OneRowRelation.select(Not(GreaterThan(true, true)))
-    comparePlans(parsed, expected)
-  }
-
-  test("test Union Distinct operator") {
-    val parsed1 = parser.parsePlan(
-      "SELECT * FROM t0 UNION SELECT * FROM t1")
-    val parsed2 = parser.parsePlan(
-      "SELECT * FROM t0 UNION DISTINCT SELECT * FROM t1")
-    val expected = Distinct(Union(table("t0").select(star), table("t1").select(star)))
-      .as("u_1").select(star)
-    comparePlans(parsed1, expected)
-    comparePlans(parsed2, expected)
-  }
-
-  test("test Union All operator") {
-    val parsed = parser.parsePlan("SELECT * FROM t0 UNION ALL SELECT * FROM t1")
-    val expected = Union(table("t0").select(star), table("t1").select(star)).as("u_1").select(star)
-    comparePlans(parsed, expected)
-  }
-
-  test("support hive interval literal") {
-    def checkInterval(sql: String, result: CalendarInterval): Unit = {
-      val parsed = parser.parsePlan(sql)
-      val expected = OneRowRelation.select(Literal(result))
-      comparePlans(parsed, expected)
-    }
-
-    def checkYearMonth(lit: String): Unit = {
-      checkInterval(
-        s"SELECT INTERVAL '$lit' YEAR TO MONTH",
-        CalendarInterval.fromYearMonthString(lit))
-    }
-
-    def checkDayTime(lit: String): Unit = {
-      checkInterval(
-        s"SELECT INTERVAL '$lit' DAY TO SECOND",
-        CalendarInterval.fromDayTimeString(lit))
-    }
-
-    def checkSingleUnit(lit: String, unit: String): Unit = {
-      checkInterval(
-        s"SELECT INTERVAL '$lit' $unit",
-        CalendarInterval.fromSingleUnitString(unit, lit))
-    }
-
-    checkYearMonth("123-10")
-    checkYearMonth("496-0")
-    checkYearMonth("-2-3")
-    checkYearMonth("-123-0")
-
-    checkDayTime("99 11:22:33.123456789")
-    checkDayTime("-99 11:22:33.123456789")
-    checkDayTime("10 9:8:7.123456789")
-    checkDayTime("1 0:0:0")
-    checkDayTime("-1 0:0:0")
-    checkDayTime("1 0:0:1")
-
-    for (unit <- Seq("year", "month", "day", "hour", "minute", "second")) {
-      checkSingleUnit("7", unit)
-      checkSingleUnit("-7", unit)
-      checkSingleUnit("0", unit)
-    }
-
-    checkSingleUnit("13.123456789", "second")
-    checkSingleUnit("-13.123456789", "second")
-  }
-
-  test("support scientific notation") {
-    def assertRight(input: String, output: Double): Unit = {
-      val parsed = parser.parsePlan("SELECT " + input)
-      val expected = OneRowRelation.select(Literal(output))
-      comparePlans(parsed, expected)
-    }
-
-    assertRight("9.0e1", 90)
-    assertRight(".9e+2", 90)
-    assertRight("0.9e+2", 90)
-    assertRight("900e-1", 90)
-    assertRight("900.0E-1", 90)
-    assertRight("9.e+1", 90)
-
-    intercept[AnalysisException](parser.parsePlan("SELECT .e3"))
-  }
-
-  test("parse expressions") {
-    compareExpressions(
-      parser.parseExpression("prinln('hello', 'world')"),
-      UnresolvedFunction(
-        "prinln", Literal("hello") :: Literal("world") :: Nil, false))
-
-    compareExpressions(
-      parser.parseExpression("1 + r.r As q"),
-      Alias(Add(Literal(1), UnresolvedAttribute("r.r")), "q")())
-
-    compareExpressions(
-      parser.parseExpression("1 - f('o', o(bar))"),
-      Subtract(Literal(1),
-        UnresolvedFunction("f",
-          Literal("o") ::
-          UnresolvedFunction("o", UnresolvedAttribute("bar") :: Nil, false) ::
-          Nil, false)))
-
-    intercept[AnalysisException](parser.parseExpression("1 - f('o', o(bar)) hello * world"))
-  }
-
-  test("table identifier") {
-    assert(TableIdentifier("q") === parser.parseTableIdentifier("q"))
-    assert(TableIdentifier("q", Some("d")) === parser.parseTableIdentifier("d.q"))
-    intercept[AnalysisException](parser.parseTableIdentifier(""))
-    intercept[AnalysisException](parser.parseTableIdentifier("d.q.g"))
-  }
-
-  test("parse union/except/intersect") {
-    parser.parsePlan("select * from t1 union all select * from t2")
-    parser.parsePlan("select * from t1 union distinct select * from t2")
-    parser.parsePlan("select * from t1 union select * from t2")
-    parser.parsePlan("select * from t1 except select * from t2")
-    parser.parsePlan("select * from t1 intersect select * from t2")
-    parser.parsePlan("(select * from t1) union all (select * from t2)")
-    parser.parsePlan("(select * from t1) union distinct (select * from t2)")
-    parser.parsePlan("(select * from t1) union (select * from t2)")
-    parser.parsePlan("select * from ((select * from t1) union (select * from t2)) t")
-  }
-
-  test("window function: better support of parentheses") {
-    parser.parsePlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " +
-      "order by 2) from windowData")
-    parser.parsePlan("select sum(product + 1) over (partition by (1 + (product / 2)) " +
-      "order by 2) from windowData")
-    parser.parsePlan("select sum(product + 1) over (partition by ((product / 2) + 1) " +
-      "order by 2) from windowData")
-
-    parser.parsePlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " +
-      "from windowData")
-    parser.parsePlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " +
-      "from windowData")
-    parser.parsePlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " +
-      "from windowData")
-  }
-
-  test("very long AND/OR expression") {
-    val equals = (1 to 1000).map(x => s"$x == $x")
-    val expr = parser.parseExpression(equals.mkString(" AND "))
-    assert(expr.isInstanceOf[And])
-    assert(expr.collect( { case EqualTo(_, _) => true } ).size == 1000)
-
-    val expr2 = parser.parseExpression(equals.mkString(" OR "))
-    assert(expr2.isInstanceOf[Or])
-    assert(expr2.collect( { case EqualTo(_, _) => true } ).size == 1000)
-  }
-
-  test("subquery") {
-    parser.parsePlan("select (select max(b) from s) ss from t")
-    parser.parsePlan("select * from t where a = (select b from s)")
-    parser.parsePlan("select * from t group by g having a > (select b from s)")
-  }
-
-  test("using clause in JOIN") {
-    // Tests parsing of using clause for different join types.
-    parser.parsePlan("select * from t1 join t2 using (c1)")
-    parser.parsePlan("select * from t1 join t2 using (c1, c2)")
-    parser.parsePlan("select * from t1 left join t2 using (c1, c2)")
-    parser.parsePlan("select * from t1 right join t2 using (c1, c2)")
-    parser.parsePlan("select * from t1 full outer join t2 using (c1, c2)")
-    parser.parsePlan("select * from t1 join t2 using (c1) join t3 using (c2)")
-    // Tests errors
-    // (1) Empty using clause
-    // (2) Qualified columns in using
-    // (3) Both on and using clause
-    var error = intercept[AnalysisException](parser.parsePlan("select * from t1 join t2 using ()"))
-    assert(error.message.contains("cannot recognize input near ')'"))
-    error = intercept[AnalysisException](parser.parsePlan("select * from t1 join t2 using (t1.c1)"))
-    assert(error.message.contains("mismatched input '.'"))
-    error = intercept[AnalysisException](parser.parsePlan("select * from t1" +
-      " join t2 using (c1) on t1.c1 = t2.c1"))
-    assert(error.message.contains("missing EOF at 'on' near ')'"))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
index d9bd33c..07b89cb 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.sql.catalyst.parser
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.parser.ng.{CatalystSqlParser, ParseException}
 import org.apache.spark.sql.types._
 
 abstract class AbstractDataTypeParserSuite extends SparkFunSuite {

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
new file mode 100644
index 0000000..db96bfb
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.SparkFunSuite
+
+/**
+ * Test various parser errors.
+ */
+class ErrorParserSuite extends SparkFunSuite {
+  def intercept(sql: String, line: Int, startPosition: Int, messages: String*): Unit = {
+    val e = intercept[ParseException](CatalystSqlParser.parsePlan(sql))
+
+    // Check position.
+    assert(e.line.isDefined)
+    assert(e.line.get === line)
+    assert(e.startPosition.isDefined)
+    assert(e.startPosition.get === startPosition)
+
+    // Check messages.
+    val error = e.getMessage
+    messages.foreach { message =>
+      assert(error.contains(message))
+    }
+  }
+
+  test("no viable input") {
+    intercept("select from tbl", 1, 7, "no viable alternative at input", "-------^^^")
+    intercept("select\nfrom tbl", 2, 0, "no viable alternative at input", "^^^")
+    intercept("select ((r + 1) ", 1, 16, "no viable alternative at input", "----------------^^^")
+  }
+
+  test("extraneous input") {
+    intercept("select 1 1", 1, 9, "extraneous input '1' expecting", "---------^^^")
+    intercept("select *\nfrom r as q t", 2, 12, "extraneous input", "------------^^^")
+  }
+
+  test("mismatched input") {
+    intercept("select * from r order by q from t", 1, 27,
+      "mismatched input",
+      "---------------------------^^^")
+    intercept("select *\nfrom r\norder by q\nfrom t", 4, 0, "mismatched input", "^^^")
+  }
+
+  test("semantic errors") {
+    intercept("select *\nfrom r\norder by q\ncluster by q", 3, 0,
+      "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported",
+      "^^^")
+    intercept("select * from r where a in (select * from t)", 1, 24,
+      "IN with a Sub-query is currently not supported",
+      "------------------------^^^")
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org