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/01/06 20:16:56 UTC

[1/8] spark git commit: [SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Repository: spark
Updated Branches:
  refs/heads/master 3aa348822 -> ea489f14f


http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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 400f7f3..a2d2836 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
@@ -21,6 +21,7 @@ import scala.util.Try
 
 import org.scalatest.BeforeAndAfter
 
+import org.apache.spark.sql.catalyst.parser.ParseDriver
 import org.apache.spark.sql.{AnalysisException, QueryTest}
 import org.apache.spark.sql.catalyst.util.quietly
 import org.apache.spark.sql.hive.test.TestHiveSingleton
@@ -116,8 +117,9 @@ 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.parse(query, hiveContext.conf)
     def parseTree =
-      Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("<failed to parse>")
+      Try(quietly(ast.treeString)).getOrElse("<failed to parse>")
 
     test(name) {
       val error = intercept[AnalysisException] {
@@ -139,10 +141,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd
 
       val expectedStart = line.indexOf(token)
       val actualStart = error.startPosition.getOrElse {
-        fail(
-          s"start not returned for error on token $token\n" +
-            HiveQl.dumpTree(HiveQl.getAst(query))
-        )
+        fail(s"start not returned for error on token $token\n${ast.treeString}")
       }
       assert(expectedStart === actualStart,
        s"""Incorrect start position.


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g
deleted file mode 100644
index 69574d7..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g
+++ /dev/null
@@ -1,2457 +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.
-*/
-parser grammar SparkSqlParser;
-
-options
-{
-tokenVocab=SparkSqlLexer;
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-import SelectClauseParser, FromClauseParser, IdentifiersParser;
-
-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_JOIN;
-TOK_LEFTOUTERJOIN;
-TOK_RIGHTOUTERJOIN;
-TOK_FULLOUTERJOIN;
-TOK_UNIQUEJOIN;
-TOK_CROSSJOIN;
-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_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_DAY_LITERAL;
-TOK_INTERVAL_HOUR_LITERAL;
-TOK_INTERVAL_MINUTE_LITERAL;
-TOK_INTERVAL_SECOND_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_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_CHARSETLITERAL;
-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;
-}
-
-
-// Package headers
-@header {
-package org.apache.spark.sql.parser;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-}
-
-
-@members {
-  ArrayList<ParseError> errors = new ArrayList<ParseError>();
-  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");
-
-
-    // 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) {
-    errors.add(new ParseError(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) {
-    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 Configuration hiveConf;
-  public void setHiveConf(Configuration hiveConf) {
-    this.hiveConf = hiveConf;
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    if(hiveConf==null){
-      return false;
-    }
-    return !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS);
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
- reportError(e);
-  throw e;
-}
-}
-
-// starting rule
-statement
-	: explainStatement EOF
-	| execStatement 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
-    ;
-
-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
-    | 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?
-       | (LPAREN columnNameTypeList RPAREN)?
-         tableComment?
-         tablePartition?
-         tableBuckets?
-         tableSkewed?
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-         (KW_AS 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?)
-    ;
-
-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
-    ;
-
-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_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=primitiveType 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)
-    ;
-
-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]
-   :
-   (
-   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 b=simpleSelectStatement
-   -> {$setOpSelectStatement.tree != null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}?
-      ^(TOK_QUERY
-          ^(TOK_FROM
-            ^(TOK_SUBQUERY
-              ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b)
-              {adaptor.create(Identifier, generateUnionAlias())}
-             )
-          )
-          ^(TOK_INSERT
-             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-             ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
-          )
-       )
-   -> {$setOpSelectStatement.tree != null && $u.tree.getType()!=SparkSqlParser.TOK_UNIONDISTINCT}?
-      ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b)
-   -> {$setOpSelectStatement.tree == null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}?
-      ^(TOK_QUERY
-          ^(TOK_FROM
-            ^(TOK_SUBQUERY
-              ^(TOK_UNIONALL {$t} $b)
-              {adaptor.create(Identifier, generateUnionAlias())}
-             )
-           )
-          ^(TOK_INSERT
-            ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-            ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
-         )
-       )
-   -> ^(TOK_UNIONALL {$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
-*/

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java
deleted file mode 100644
index 35ecdc5..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java
+++ /dev/null
@@ -1,49 +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.parser;
-
-import org.antlr.runtime.RecognitionException;
-import org.antlr.runtime.Token;
-import org.antlr.runtime.TokenStream;
-import org.antlr.runtime.tree.CommonErrorNode;
-
-public class ASTErrorNode extends ASTNode {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-  CommonErrorNode delegate;
-
-  public ASTErrorNode(TokenStream input, Token start, Token stop,
-      RecognitionException e){
-    delegate = new CommonErrorNode(input,start,stop,e);
-  }
-
-  @Override
-  public boolean isNil() { return delegate.isNil(); }
-
-  @Override
-  public int getType() { return delegate.getType(); }
-
-  @Override
-  public String getText() { return delegate.getText(); }
-  @Override
-  public String toString() { return delegate.toString(); }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java
deleted file mode 100644
index 33d9322..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java
+++ /dev/null
@@ -1,245 +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.parser;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.antlr.runtime.Token;
-import org.antlr.runtime.tree.CommonTree;
-import org.antlr.runtime.tree.Tree;
-import org.apache.hadoop.hive.ql.lib.Node;
-
-public class ASTNode extends CommonTree implements Node, Serializable {
-  private static final long serialVersionUID = 1L;
-  private transient StringBuffer astStr;
-  private transient int startIndx = -1;
-  private transient int endIndx = -1;
-  private transient ASTNode rootNode;
-  private transient boolean isValidASTStr;
-
-  public ASTNode() {
-  }
-
-  /**
-   * Constructor.
-   *
-   * @param t
-   *          Token for the CommonTree Node
-   */
-  public ASTNode(Token t) {
-    super(t);
-  }
-
-  public ASTNode(ASTNode node) {
-    super(node);
-  }
-
-  @Override
-  public Tree dupNode() {
-    return new ASTNode(this);
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.hadoop.hive.ql.lib.Node#getChildren()
-   */
-  @Override
-  public ArrayList<Node> getChildren() {
-    if (super.getChildCount() == 0) {
-      return null;
-    }
-
-    ArrayList<Node> ret_vec = new ArrayList<Node>();
-    for (int i = 0; i < super.getChildCount(); ++i) {
-      ret_vec.add((Node) super.getChild(i));
-    }
-
-    return ret_vec;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.hadoop.hive.ql.lib.Node#getName()
-   */
-  @Override
-  public String getName() {
-    return (Integer.valueOf(super.getToken().getType())).toString();
-  }
-
-  public String dump() {
-    StringBuilder sb = new StringBuilder("\n");
-    dump(sb, "");
-    return sb.toString();
-  }
-
-  private StringBuilder dump(StringBuilder sb, String ws) {
-    sb.append(ws);
-    sb.append(toString());
-    sb.append("\n");
-
-    ArrayList<Node> children = getChildren();
-    if (children != null) {
-      for (Node node : getChildren()) {
-        if (node instanceof ASTNode) {
-          ((ASTNode) node).dump(sb, ws + "   ");
-        } else {
-          sb.append(ws);
-          sb.append("   NON-ASTNODE!!");
-          sb.append("\n");
-        }
-      }
-    }
-    return sb;
-  }
-
-  private ASTNode getRootNodeWithValidASTStr(boolean useMemoizedRoot) {
-    if (useMemoizedRoot && rootNode != null && rootNode.parent == null &&
-        rootNode.hasValidMemoizedString()) {
-      return rootNode;
-    }
-    ASTNode retNode = this;
-    while (retNode.parent != null) {
-      retNode = (ASTNode) retNode.parent;
-    }
-    rootNode=retNode;
-    if (!rootNode.isValidASTStr) {
-      rootNode.astStr = new StringBuffer();
-      rootNode.toStringTree(rootNode);
-      rootNode.isValidASTStr = true;
-    }
-    return retNode;
-  }
-
-  private boolean hasValidMemoizedString() {
-    return isValidASTStr && astStr != null;
-  }
-
-  private void resetRootInformation() {
-    // Reset the previously stored rootNode string
-    if (rootNode != null) {
-      rootNode.astStr = null;
-      rootNode.isValidASTStr = false;
-    }
-  }
-
-  private int getMemoizedStringLen() {
-    return astStr == null ? 0 : astStr.length();
-  }
-
-  private String getMemoizedSubString(int start, int end) {
-    return  (astStr == null || start < 0 || end > astStr.length() || start >= end) ? null :
-      astStr.subSequence(start, end).toString();
-  }
-
-  private void addtoMemoizedString(String string) {
-    if (astStr == null) {
-      astStr = new StringBuffer();
-    }
-    astStr.append(string);
-  }
-
-  @Override
-  public void setParent(Tree t) {
-    super.setParent(t);
-    resetRootInformation();
-  }
-
-  @Override
-  public void addChild(Tree t) {
-    super.addChild(t);
-    resetRootInformation();
-  }
-
-  @Override
-  public void addChildren(List kids) {
-    super.addChildren(kids);
-    resetRootInformation();
-  }
-
-  @Override
-  public void setChild(int i, Tree t) {
-    super.setChild(i, t);
-    resetRootInformation();
-  }
-
-  @Override
-  public void insertChild(int i, Object t) {
-    super.insertChild(i, t);
-    resetRootInformation();
-  }
-
-  @Override
-  public Object deleteChild(int i) {
-   Object ret = super.deleteChild(i);
-   resetRootInformation();
-   return ret;
-  }
-
-  @Override
-  public void replaceChildren(int startChildIndex, int stopChildIndex, Object t) {
-    super.replaceChildren(startChildIndex, stopChildIndex, t);
-    resetRootInformation();
-  }
-
-  @Override
-  public String toStringTree() {
-
-    // The root might have changed because of tree modifications.
-    // Compute the new root for this tree and set the astStr.
-    getRootNodeWithValidASTStr(true);
-
-    // If rootNotModified is false, then startIndx and endIndx will be stale.
-    if (startIndx >= 0 && endIndx <= rootNode.getMemoizedStringLen()) {
-      return rootNode.getMemoizedSubString(startIndx, endIndx);
-    }
-    return toStringTree(rootNode);
-  }
-
-  private String toStringTree(ASTNode rootNode) {
-    this.rootNode = rootNode;
-    startIndx = rootNode.getMemoizedStringLen();
-    // Leaf node
-    if ( children==null || children.size()==0 ) {
-      rootNode.addtoMemoizedString(this.toString());
-      endIndx =  rootNode.getMemoizedStringLen();
-      return this.toString();
-    }
-    if ( !isNil() ) {
-      rootNode.addtoMemoizedString("(");
-      rootNode.addtoMemoizedString(this.toString());
-      rootNode.addtoMemoizedString(" ");
-    }
-    for (int i = 0; children!=null && i < children.size(); i++) {
-      ASTNode t = (ASTNode)children.get(i);
-      if ( i>0 ) {
-        rootNode.addtoMemoizedString(" ");
-      }
-      t.toStringTree(rootNode);
-    }
-    if ( !isNil() ) {
-      rootNode.addtoMemoizedString(")");
-    }
-    endIndx =  rootNode.getMemoizedStringLen();
-    return rootNode.getMemoizedSubString(startIndx, endIndx);
-  }
-}


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
new file mode 100644
index 0000000..42bdf25
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala
@@ -0,0 +1,961 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst
+
+import java.sql.Date
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.Count
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.trees.CurrentOrigin
+import org.apache.spark.sql.catalyst.parser._
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.util.random.RandomSampler
+
+/**
+ * This class translates a HQL String to a Catalyst [[LogicalPlan]] or [[Expression]].
+ */
+private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) {
+  object Token {
+    def unapply(node: ASTNode): Some[(String, List[ASTNode])] = {
+      CurrentOrigin.setPosition(node.line, node.positionInLine)
+      node.pattern
+    }
+  }
+
+
+  /**
+   * Returns the AST for the given SQL string.
+   */
+  protected def getAst(sql: String): ASTNode = ParseDriver.parse(sql, conf)
+
+  /** Creates LogicalPlan for a given HiveQL string. */
+  def createPlan(sql: String): LogicalPlan = {
+    try {
+      createPlan(sql, ParseDriver.parse(sql, conf))
+    } 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
+             |${getAst(sql).treeString}
+             |$e
+             |${e.getStackTrace.head}
+          """.stripMargin)
+    }
+  }
+
+  protected def createPlan(sql: String, tree: ASTNode): LogicalPlan = nodeToPlan(tree)
+
+  def parseDdl(ddl: String): Seq[Attribute] = {
+    val tree = getAst(ddl)
+    assert(tree.text == "TOK_CREATETABLE", "Only CREATE TABLE supported.")
+    val tableOps = tree.children
+    val colList = tableOps
+      .find(_.text == "TOK_TABCOLLIST")
+      .getOrElse(sys.error("No columnList!"))
+
+    colList.children.map(nodeToAttribute)
+  }
+
+  protected def getClauses(
+      clauseNames: Seq[String],
+      nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
+    var remainingNodes = nodeList
+    val clauses = clauseNames.map { clauseName =>
+      val (matches, nonMatches) = remainingNodes.partition(_.text.toUpperCase == clauseName)
+      remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
+      matches.headOption
+    }
+
+    if (remainingNodes.nonEmpty) {
+      sys.error(
+        s"""Unhandled clauses: ${remainingNodes.map(_.treeString).mkString("\n")}.
+            |You are likely trying to use an unsupported Hive feature."""".stripMargin)
+    }
+    clauses
+  }
+
+  protected def getClause(clauseName: String, nodeList: Seq[ASTNode]): ASTNode =
+    getClauseOption(clauseName, nodeList).getOrElse(sys.error(
+      s"Expected clause $clauseName missing from ${nodeList.map(_.treeString).mkString("\n")}"))
+
+  protected def getClauseOption(clauseName: String, nodeList: Seq[ASTNode]): Option[ASTNode] = {
+    nodeList.filter { case ast: ASTNode => ast.text == clauseName } match {
+      case Seq(oneMatch) => Some(oneMatch)
+      case Seq() => None
+      case _ => sys.error(s"Found multiple instances of clause $clauseName")
+    }
+  }
+
+  protected def nodeToAttribute(node: ASTNode): Attribute = node match {
+    case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) =>
+      AttributeReference(colName, nodeToDataType(dataType), nullable = true)()
+    case _ =>
+      noParseRule("Attribute", node)
+  }
+
+  protected def nodeToDataType(node: ASTNode): DataType = node match {
+    case Token("TOK_DECIMAL", precision :: scale :: Nil) =>
+      DecimalType(precision.text.toInt, scale.text.toInt)
+    case Token("TOK_DECIMAL", precision :: Nil) =>
+      DecimalType(precision.text.toInt, 0)
+    case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT
+    case Token("TOK_BIGINT", Nil) => LongType
+    case Token("TOK_INT", Nil) => IntegerType
+    case Token("TOK_TINYINT", Nil) => ByteType
+    case Token("TOK_SMALLINT", Nil) => ShortType
+    case Token("TOK_BOOLEAN", Nil) => BooleanType
+    case Token("TOK_STRING", Nil) => StringType
+    case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType
+    case Token("TOK_FLOAT", Nil) => FloatType
+    case Token("TOK_DOUBLE", Nil) => DoubleType
+    case Token("TOK_DATE", Nil) => DateType
+    case Token("TOK_TIMESTAMP", Nil) => TimestampType
+    case Token("TOK_BINARY", Nil) => BinaryType
+    case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType))
+    case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) =>
+      StructType(fields.map(nodeToStructField))
+    case Token("TOK_MAP", keyType :: valueType :: Nil) =>
+      MapType(nodeToDataType(keyType), nodeToDataType(valueType))
+    case _ =>
+      noParseRule("DataType", node)
+  }
+
+  protected def nodeToStructField(node: ASTNode): StructField = node match {
+    case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: Nil) =>
+      StructField(fieldName, nodeToDataType(dataType), nullable = true)
+    case Token("TOK_TABCOL", Token(fieldName, Nil) :: dataType :: _ /* comment */:: Nil) =>
+      StructField(fieldName, nodeToDataType(dataType), nullable = true)
+    case _ =>
+      noParseRule("StructField", node)
+  }
+
+  protected def extractTableIdent(tableNameParts: ASTNode): TableIdentifier = {
+    tableNameParts.children.map {
+      case Token(part, Nil) => cleanIdentifier(part)
+    } match {
+      case Seq(tableOnly) => TableIdentifier(tableOnly)
+      case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName))
+      case other => sys.error("Hive only supports tables names like 'tableName' " +
+        s"or 'databaseName.tableName', found '$other'")
+    }
+  }
+
+  /**
+   * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2))
+   * is equivalent to
+   * 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 0: k1, bit 1: k2, and bit 2: k3), the grouping id of
+   * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 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 bitmasks: Seq[Int] = setASTs.map {
+      case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0
+      case Token("TOK_GROUPING_SETS_EXPRESSION", columns) =>
+        columns.foldLeft(0)((bitmap, col) => {
+          val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2)
+          bitmap | 1 << keyIndex.getOrElse(
+            throw new AnalysisException(s"${col.treeString} doesn't show up in the GROUP BY list"))
+        })
+      case _ => sys.error("Expect GROUPING SETS clause")
+    }
+
+    (keys, bitmasks)
+  }
+
+  protected def nodeToPlan(node: ASTNode): LogicalPlan = node match {
+    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[Subquery]
+              relation.alias -> relation
+            }
+            (Some(from.head), inserts, Some(cteRelations.toMap))
+          case Token("TOK_FROM", from) :: inserts =>
+            (Some(from.head), inserts, None)
+          case Token("TOK_INSERT", _) :: Nil =>
+            (None, queryArgs, None)
+        }
+
+      // Return one query for each insert clause.
+      val queries = insertClauses.map {
+        case Token("TOK_INSERT", singleInsert) =>
+          val (
+            intoClause ::
+              destClause ::
+              selectClause ::
+              selectDistinctClause ::
+              whereClause ::
+              groupByClause ::
+              rollupGroupByClause ::
+              cubeGroupByClause ::
+              groupingSetsClause ::
+              orderByClause ::
+              havingClause ::
+              sortByClause ::
+              clusterByClause ::
+              distributeByClause ::
+              limitClause ::
+              lateralViewClause ::
+              windowClause :: Nil) = {
+            getClauses(
+              Seq(
+                "TOK_INSERT_INTO",
+                "TOK_DESTINATION",
+                "TOK_SELECT",
+                "TOK_SELECTDI",
+                "TOK_WHERE",
+                "TOK_GROUPBY",
+                "TOK_ROLLUP_GROUPBY",
+                "TOK_CUBE_GROUPBY",
+                "TOK_GROUPING_SETS",
+                "TOK_ORDERBY",
+                "TOK_HAVING",
+                "TOK_SORTBY",
+                "TOK_CLUSTERBY",
+                "TOK_DISTRIBUTEBY",
+                "TOK_LIMIT",
+                "TOK_LATERAL_VIEW",
+                "WINDOW"),
+              singleInsert)
+          }
+
+          val relations = fromClause match {
+            case Some(f) => nodeToRelation(f)
+            case None => OneRowRelation
+          }
+
+          val withWhere = whereClause.map { whereNode =>
+            val Seq(whereExpr) = whereNode.children
+            Filter(nodeToExpr(whereExpr), relations)
+          }.getOrElse(relations)
+
+          val select = (selectClause orElse selectDistinctClause)
+            .getOrElse(sys.error("No select clause."))
+
+          val transformation = nodeToTransformation(select.children.head, withWhere)
+
+          val withLateralView = lateralViewClause.map { lv =>
+            nodeToGenerate(lv.children.head, outer = false, withWhere)
+          }.getOrElse(withWhere)
+
+          // The projection of the query can either be a normal projection, an aggregation
+          // (if there is a group by) or a script transformation.
+          val withProject: LogicalPlan = transformation.getOrElse {
+            val selectExpressions =
+              select.children.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_))
+            Seq(
+              groupByClause.map(e => e match {
+                case Token("TOK_GROUPBY", children) =>
+                  // Not a transformation so must be either project or aggregation.
+                  Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView)
+                case _ => sys.error("Expect GROUP BY")
+              }),
+              groupingSetsClause.map(e => e match {
+                case Token("TOK_GROUPING_SETS", children) =>
+                  val(groupByExprs, masks) = extractGroupingSet(children)
+                  GroupingSets(masks, groupByExprs, withLateralView, selectExpressions)
+                case _ => sys.error("Expect GROUPING SETS")
+              }),
+              rollupGroupByClause.map(e => e match {
+                case Token("TOK_ROLLUP_GROUPBY", children) =>
+                  Aggregate(
+                    Seq(Rollup(children.map(nodeToExpr))),
+                    selectExpressions,
+                    withLateralView)
+                case _ => sys.error("Expect WITH ROLLUP")
+              }),
+              cubeGroupByClause.map(e => e match {
+                case Token("TOK_CUBE_GROUPBY", children) =>
+                  Aggregate(
+                    Seq(Cube(children.map(nodeToExpr))),
+                    selectExpressions,
+                    withLateralView)
+                case _ => sys.error("Expect WITH CUBE")
+              }),
+              Some(Project(selectExpressions, withLateralView))).flatten.head
+          }
+
+          // Handle HAVING clause.
+          val withHaving = havingClause.map { h =>
+            val havingExpr = h.children match { case Seq(hexpr) => nodeToExpr(hexpr) }
+            // Note that we added a cast to boolean. If the expression itself is already boolean,
+            // the optimizer will get rid of the unnecessary cast.
+            Filter(Cast(havingExpr, BooleanType), withProject)
+          }.getOrElse(withProject)
+
+          // Handle SELECT DISTINCT
+          val withDistinct =
+            if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving
+
+          // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
+          val withSort =
+            (orderByClause, sortByClause, distributeByClause, clusterByClause) match {
+              case (Some(totalOrdering), None, None, None) =>
+                Sort(totalOrdering.children.map(nodeToSortOrder), global = true, withDistinct)
+              case (None, Some(perPartitionOrdering), None, None) =>
+                Sort(
+                  perPartitionOrdering.children.map(nodeToSortOrder),
+                  global = false, withDistinct)
+              case (None, None, Some(partitionExprs), None) =>
+                RepartitionByExpression(
+                  partitionExprs.children.map(nodeToExpr), withDistinct)
+              case (None, Some(perPartitionOrdering), Some(partitionExprs), None) =>
+                Sort(
+                  perPartitionOrdering.children.map(nodeToSortOrder), global = false,
+                  RepartitionByExpression(
+                    partitionExprs.children.map(nodeToExpr),
+                    withDistinct))
+              case (None, None, None, Some(clusterExprs)) =>
+                Sort(
+                  clusterExprs.children.map(nodeToExpr).map(SortOrder(_, Ascending)),
+                  global = false,
+                  RepartitionByExpression(
+                    clusterExprs.children.map(nodeToExpr),
+                    withDistinct))
+              case (None, None, None, None) => withDistinct
+              case _ => sys.error("Unsupported set of ordering / distribution clauses.")
+            }
+
+          val withLimit =
+            limitClause.map(l => nodeToExpr(l.children.head))
+              .map(Limit(_, withSort))
+              .getOrElse(withSort)
+
+          // Collect all window specifications defined in the WINDOW clause.
+          val windowDefinitions = windowClause.map(_.children.collect {
+            case Token("TOK_WINDOWDEF",
+            Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) =>
+              windowName -> nodesToWindowSpecification(spec)
+          }.toMap)
+          // Handle cases like
+          // window w1 as (partition by p_mfgr order by p_name
+          //               range between 2 preceding and 2 following),
+          //        w2 as w1
+          val resolvedCrossReference = windowDefinitions.map {
+            windowDefMap => windowDefMap.map {
+              case (windowName, WindowSpecReference(other)) =>
+                (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition])
+              case o => o.asInstanceOf[(String, WindowSpecDefinition)]
+            }
+          }
+
+          val withWindowDefinitions =
+            resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit)
+
+          // TOK_INSERT_INTO means to add files to the table.
+          // TOK_DESTINATION means to overwrite the table.
+          val resultDestination =
+            (intoClause orElse destClause).getOrElse(sys.error("No destination found."))
+          val overwrite = intoClause.isEmpty
+          nodeToDest(
+            resultDestination,
+            withWindowDefinitions,
+            overwrite)
+      }
+
+      // If there are multiple INSERTS just UNION them together into on query.
+      val query = queries.reduceLeft(Union)
+
+      // return With plan if there is CTE
+      cteRelations.map(With(query, _)).getOrElse(query)
+
+    // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT
+    case Token("TOK_UNIONALL", left :: right :: Nil) =>
+      Union(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) =>
+        Subquery(cleanIdentifier(alias), nodeToPlan(query))
+
+      case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) =>
+        nodeToGenerate(
+          selectClause,
+          outer = isOuter.nonEmpty,
+          nodeToRelation(relationClause))
+
+      /* All relations, possibly with aliases or sampling clauses. */
+      case Token("TOK_TABREF", clauses) =>
+        // If the last clause is not a token then it's the alias of the table.
+        val (nonAliasClauses, aliasClause) =
+          if (clauses.last.text.startsWith("TOK")) {
+            (clauses, None)
+          } else {
+            (clauses.dropRight(1), Some(clauses.last))
+          }
+
+        val (Some(tableNameParts) ::
+          splitSampleClause ::
+          bucketSampleClause :: Nil) = {
+          getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"),
+            nonAliasClauses)
+        }
+
+        val tableIdent = extractTableIdent(tableNameParts)
+        val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) }
+        val relation = UnresolvedRelation(tableIdent, alias)
+
+        // Apply sampling if requested.
+        (bucketSampleClause orElse splitSampleClause).map {
+          case Token("TOK_TABLESPLITSAMPLE",
+          Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) =>
+            Limit(Literal(count.toInt), relation)
+          case Token("TOK_TABLESPLITSAMPLE",
+          Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) =>
+            // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
+            // function takes X PERCENT as the input and the range of X is [0, 100], we need to
+            // adjust the fraction.
+            require(
+              fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon)
+                && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon),
+              s"Sampling fraction ($fraction) must be on interval [0, 100]")
+            Sample(0.0, fraction.toDouble / 100, withReplacement = false,
+              (math.random * 1000).toInt,
+              relation)
+          case Token("TOK_TABLEBUCKETSAMPLE",
+          Token(numerator, Nil) ::
+            Token(denominator, Nil) :: Nil) =>
+            val fraction = numerator.toDouble / denominator.toDouble
+            Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation)
+          case a =>
+            noParseRule("Sampling", a)
+        }.getOrElse(relation)
+
+      case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) =>
+        if (!(other.size <= 1)) {
+          sys.error(s"Unsupported join operation: $other")
+        }
+
+        val joinType = joinToken match {
+          case "TOK_JOIN" => Inner
+          case "TOK_CROSSJOIN" => Inner
+          case "TOK_RIGHTOUTERJOIN" => RightOuter
+          case "TOK_LEFTOUTERJOIN" => LeftOuter
+          case "TOK_FULLOUTERJOIN" => FullOuter
+          case "TOK_LEFTSEMIJOIN" => LeftSemi
+          case "TOK_UNIQUEJOIN" => noParseRule("Unique Join", node)
+          case "TOK_ANTIJOIN" => noParseRule("Anti Join", node)
+        }
+        Join(nodeToRelation(relation1),
+          nodeToRelation(relation2),
+          joinType,
+          other.headOption.map(nodeToExpr))
+
+      case _ =>
+        noParseRule("Relation", node)
+    }
+  }
+
+  protected def nodeToSortOrder(node: ASTNode): SortOrder = node match {
+    case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) =>
+      SortOrder(nodeToExpr(sortExpr), Ascending)
+    case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) =>
+      SortOrder(nodeToExpr(sortExpr), Descending)
+    case _ =>
+      noParseRule("SortOrder", node)
+  }
+
+  val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r
+  protected def nodeToDest(
+      node: ASTNode,
+      query: LogicalPlan,
+      overwrite: Boolean): LogicalPlan = node match {
+    case Token(destinationToken(),
+    Token("TOK_DIR",
+    Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) =>
+      query
+
+    case Token(destinationToken(),
+    Token("TOK_TAB",
+    tableArgs) :: Nil) =>
+      val Some(tableNameParts) :: partitionClause :: Nil =
+        getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs)
+
+      val tableIdent = extractTableIdent(tableNameParts)
+
+      val partitionKeys = partitionClause.map(_.children.map {
+        // Parse partitions. We also make keys case insensitive.
+        case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
+          cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value))
+        case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) =>
+          cleanIdentifier(key.toLowerCase) -> None
+      }.toMap).getOrElse(Map.empty)
+
+      InsertIntoTable(
+        UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = false)
+
+    case Token(destinationToken(),
+    Token("TOK_TAB",
+    tableArgs) ::
+      Token("TOK_IFNOTEXISTS",
+      ifNotExists) :: Nil) =>
+      val Some(tableNameParts) :: partitionClause :: Nil =
+        getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs)
+
+      val tableIdent = extractTableIdent(tableNameParts)
+
+      val partitionKeys = partitionClause.map(_.children.map {
+        // Parse partitions. We also make keys case insensitive.
+        case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
+          cleanIdentifier(key.toLowerCase) -> Some(unquoteString(value))
+        case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) =>
+          cleanIdentifier(key.toLowerCase) -> None
+      }.toMap).getOrElse(Map.empty)
+
+      InsertIntoTable(
+        UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, ifNotExists = true)
+
+    case _ =>
+      noParseRule("Destination", node)
+  }
+
+  protected def selExprNodeToExpr(node: ASTNode): Option[Expression] = node match {
+    case Token("TOK_SELEXPR", e :: Nil) =>
+      Some(nodeToExpr(e))
+
+    case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) =>
+      Some(Alias(nodeToExpr(e), cleanIdentifier(alias))())
+
+    case Token("TOK_SELEXPR", e :: aliasChildren) =>
+      val aliasNames = aliasChildren.collect {
+        case Token(name, Nil) => cleanIdentifier(name)
+      }
+      Some(MultiAlias(nodeToExpr(e), aliasNames))
+
+    /* Hints are ignored */
+    case Token("TOK_HINTLIST", _) => None
+
+    case _ =>
+      noParseRule("Select", node)
+  }
+
+  protected val escapedIdentifier = "`([^`]+)`".r
+  protected val doubleQuotedString = "\"([^\"]+)\"".r
+  protected val singleQuotedString = "'([^']+)'".r
+
+  protected def unquoteString(str: String) = str match {
+    case singleQuotedString(s) => s
+    case doubleQuotedString(s) => s
+    case other => other
+  }
+
+  /** Strips backticks from ident if present */
+  protected def cleanIdentifier(ident: String): String = ident match {
+    case escapedIdentifier(i) => i
+    case plainIdent => plainIdent
+  }
+
+  val numericAstTypes = Seq(
+    SparkSqlParser.Number,
+    SparkSqlParser.TinyintLiteral,
+    SparkSqlParser.SmallintLiteral,
+    SparkSqlParser.BigintLiteral,
+    SparkSqlParser.DecimalLiteral)
+
+  /* Case insensitive matches */
+  val COUNT = "(?i)COUNT".r
+  val SUM = "(?i)SUM".r
+  val AND = "(?i)AND".r
+  val OR = "(?i)OR".r
+  val NOT = "(?i)NOT".r
+  val TRUE = "(?i)TRUE".r
+  val FALSE = "(?i)FALSE".r
+  val LIKE = "(?i)LIKE".r
+  val RLIKE = "(?i)RLIKE".r
+  val REGEXP = "(?i)REGEXP".r
+  val IN = "(?i)IN".r
+  val DIV = "(?i)DIV".r
+  val BETWEEN = "(?i)BETWEEN".r
+  val WHEN = "(?i)WHEN".r
+  val CASE = "(?i)CASE".r
+
+  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(attr))
+      }
+
+    /* 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", Token(name, Nil) :: Nil) :: Nil) =>
+      UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(name)))
+
+    /* Aggregate Functions */
+    case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) =>
+      Count(args.map(nodeToExpr)).toAggregateExpression(isDistinct = true)
+    case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) =>
+      Count(Literal(1)).toAggregateExpression()
+
+    /* Casts */
+    case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), StringType)
+    case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), StringType)
+    case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), StringType)
+    case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), IntegerType)
+    case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), LongType)
+    case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), FloatType)
+    case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), DoubleType)
+    case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), ShortType)
+    case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), ByteType)
+    case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), BinaryType)
+    case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), BooleanType)
+    case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: scale :: nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, scale.text.toInt))
+    case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), DecimalType(precision.text.toInt, 0))
+    case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), DecimalType.USER_DEFAULT)
+    case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), TimestampType)
+    case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) =>
+      Cast(nodeToExpr(arg), DateType)
+
+    /* Arithmetic */
+    case Token("+", child :: Nil) => nodeToExpr(child)
+    case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child))
+    case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child))
+    case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right))
+    case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right))
+    case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right))
+    case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right))
+    case Token(DIV(), left :: right:: Nil) =>
+      Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType)
+    case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right))
+    case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right))
+    case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right))
+    case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right))
+
+    /* Comparisons */
+    case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right))
+    case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right))
+    case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right))
+    case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right)))
+    case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right)))
+    case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right))
+    case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right))
+    case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right))
+    case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right))
+    case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right))
+    case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right))
+    case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right))
+    case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) =>
+      IsNotNull(nodeToExpr(child))
+    case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) =>
+      IsNull(nodeToExpr(child))
+    case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) =>
+      In(nodeToExpr(value), list.map(nodeToExpr))
+    case Token("TOK_FUNCTION",
+    Token(BETWEEN(), Nil) ::
+      kw ::
+      target ::
+      minValue ::
+      maxValue :: Nil) =>
+
+      val targetExpression = nodeToExpr(target)
+      val betweenExpr =
+        And(
+          GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)),
+          LessThanOrEqual(targetExpression, nodeToExpr(maxValue)))
+      kw match {
+        case Token("KW_FALSE", Nil) => betweenExpr
+        case Token("KW_TRUE", Nil) => Not(betweenExpr)
+      }
+
+    /* Boolean Logic */
+    case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right))
+    case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right))
+    case Token(NOT(), child :: Nil) => Not(nodeToExpr(child))
+    case Token("!", child :: Nil) => Not(nodeToExpr(child))
+
+    /* Case statements */
+    case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) =>
+      CaseWhen(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)
+
+    // This code is adapted from
+    // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223
+    case ast: ASTNode if numericAstTypes contains ast.tokenType =>
+      var v: Literal = null
+      try {
+        if (ast.text.endsWith("L")) {
+          // Literal bigint.
+          v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toLong, LongType)
+        } else if (ast.text.endsWith("S")) {
+          // Literal smallint.
+          v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toShort, ShortType)
+        } else if (ast.text.endsWith("Y")) {
+          // Literal tinyint.
+          v = Literal.create(ast.text.substring(0, ast.text.length() - 1).toByte, ByteType)
+        } else if (ast.text.endsWith("BD") || ast.text.endsWith("D")) {
+          // Literal decimal
+          val strVal = ast.text.stripSuffix("D").stripSuffix("B")
+          v = Literal(Decimal(strVal))
+        } else {
+          v = Literal.create(ast.text.toDouble, DoubleType)
+          v = Literal.create(ast.text.toLong, LongType)
+          v = Literal.create(ast.text.toInt, IntegerType)
+        }
+      } catch {
+        case nfe: NumberFormatException => // Do nothing
+      }
+
+      if (v == null) {
+        sys.error(s"Failed to parse number '${ast.text}'.")
+      } else {
+        v
+      }
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.StringLiteral =>
+      Literal(ParseUtils.unescapeSQLString(ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_DATELITERAL =>
+      Literal(Date.valueOf(ast.text.substring(1, ast.text.length - 1)))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_CHARSETLITERAL =>
+      Literal(ParseUtils.charSetString(ast.children.head.text, ast.children(1).text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL =>
+      Literal(CalendarInterval.fromYearMonthString(ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL =>
+      Literal(CalendarInterval.fromDayTimeString(ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_YEAR_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("year", ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_MONTH_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("month", ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_DAY_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("day", ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_HOUR_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("hour", ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_MINUTE_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("minute", ast.text))
+
+    case ast: ASTNode if ast.tokenType == SparkSqlParser.TOK_INTERVAL_SECOND_LITERAL =>
+      Literal(CalendarInterval.fromSingleUnitString("second", ast.text))
+
+    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 = getClause("TOK_TABALIAS", clauses).children.head.text
+
+    val generator = clauses.head match {
+      case Token("TOK_FUNCTION", Token(explode(), Nil) :: childNode :: Nil) =>
+        Explode(nodeToExpr(childNode))
+      case Token("TOK_FUNCTION", Token(jsonTuple(), Nil) :: children) =>
+        JsonTuple(children.map(nodeToExpr))
+      case other =>
+        nodeToGenerator(other)
+    }
+
+    val attributes = clauses.collect {
+      case Token(a, Nil) => UnresolvedAttribute(a.toLowerCase)
+    }
+
+    Generate(generator, join = true, outer = outer, Some(alias.toLowerCase), attributes, child)
+  }
+
+  protected def nodeToGenerator(node: ASTNode): Generator = noParseRule("Generator", node)
+
+  protected def noParseRule(msg: String, node: ASTNode): Nothing = throw new NotImplementedError(
+    s"[$msg]: No parse rules for ASTNode type: ${node.tokenType}, tree:\n${node.treeString}")
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
new file mode 100644
index 0000000..ec5e710
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ASTNode.scala
@@ -0,0 +1,93 @@
+/*
+ * 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 = children.size
+
+  /** tuple used in pattern matching. */
+  val pattern = 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(Some(line), Some(positionInLine))
+
+  /** Source text. */
+  lazy val source = stream.toString(startIndex, stopIndex)
+
+  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/ea489f14/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
new file mode 100644
index 0000000..0e93af8
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -0,0 +1,156 @@
+/*
+ * 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._
+import org.antlr.runtime.tree.CommonTree
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.AnalysisException
+
+/**
+ * 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
+ */
+object ParseDriver extends Logging {
+  def parse(command: String, conf: ParserConf): ASTNode = {
+    logInfo(s"Parsing command: $command")
+
+    // Setup error collection.
+    val reporter = new ParseErrorReporter()
+
+    // Create lexer.
+    val lexer = new SparkSqlLexer(new ANTLRNoCaseStringStream(command))
+    val tokens = new TokenRewriteStream(lexer)
+    lexer.configure(conf, reporter)
+
+    // Create the parser.
+    val parser = new SparkSqlParser(tokens)
+    parser.configure(conf, reporter)
+
+    try {
+      val result = parser.statement()
+
+      // Check errors.
+      reporter.checkForErrors()
+
+      // Return the AST node from the result.
+      logInfo(s"Parse completed.")
+
+      // Find the non null token tree in the result.
+      def nonNullToken(tree: CommonTree): CommonTree = {
+        if (tree.token != null || tree.getChildCount == 0) tree
+        else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree])
+      }
+      val tree = nonNullToken(result.getTree)
+
+      // 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)
+      }
+      createASTNode(tree)
+    }
+    catch {
+      case e: RecognitionException =>
+        logInfo(s"Parse failed.")
+        reporter.throwError(e)
+    }
+  }
+}
+
+/**
+ * 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 ANTLRStringStream(input) {
+  override def LA(i: Int): Int = {
+    val la = super.LA(i)
+    if (la == 0 || la == CharStream.EOF) la
+    else Character.toUpperCase(la)
+  }
+}
+
+/**
+ * Utility used by the Parser and the Lexer for error collection and reporting.
+ */
+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)
+  }
+
+  def checkForErrors(): Unit = {
+    if (errors.nonEmpty) {
+      val first = errors.head
+      val e = first.re
+      throwError(e.line, e.charPositionInLine, first.buildMessage().toString, errors.tail)
+    }
+  }
+
+  def throwError(e: RecognitionException): Nothing = {
+    throwError(e.line, e.charPositionInLine, e.toString, errors)
+  }
+
+  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))
+  }
+}
+
+/**
+ * Error collected during the parsing process.
+ *
+ * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseError
+ */
+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))
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
new file mode 100644
index 0000000..ce449b1
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserConf.scala
@@ -0,0 +1,26 @@
+/*
+ * 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/ea489f14/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
index b58a373..26c00dc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
@@ -25,6 +25,7 @@ import scala.collection.JavaConverters._
 import org.apache.parquet.hadoop.ParquetOutputCommitter
 
 import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.parser.ParserConf
 
 ////////////////////////////////////////////////////////////////////////////////////////////////////
 // This file defines the configuration options for Spark SQL.
@@ -451,6 +452,19 @@ private[spark] 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.")
+
   object Deprecated {
     val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
     val EXTERNAL_SORT = "spark.sql.planner.externalSort"
@@ -471,7 +485,7 @@ private[spark] object SQLConf {
  *
  * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads).
  */
-private[sql] class SQLConf extends Serializable with CatalystConf {
+private[sql] class SQLConf extends Serializable with CatalystConf with ParserConf {
   import SQLConf._
 
   /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */
@@ -569,6 +583,10 @@ private[sql] class SQLConf extends Serializable with CatalystConf {
 
   private[spark] def runSQLOnFile: Boolean = getConf(RUN_SQL_ON_FILES)
 
+  def supportQuotedId: Boolean = getConf(PARSER_SUPPORT_QUOTEDID)
+
+  def supportSQL11ReservedKeywords: Boolean = getConf(PARSER_SUPPORT_SQL11_RESERVED_KEYWORDS)
+
   /** ********************** SQLConf functionality methods ************ */
 
   /** Set Spark SQL configuration properties. */

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
new file mode 100644
index 0000000..a322688
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution
+
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
+import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
+
+private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) {
+  /** Check if a command should not be explained. */
+  protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command
+
+  protected override def nodeToPlan(node: ASTNode): LogicalPlan = {
+    node match {
+      // 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_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) :: Nil) =>
+              nameParts match {
+                case Token(".", dbName :: tableName :: 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 = extractTableIdent(nameParts)
+                  datasources.DescribeCommand(
+                    UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined)
+                case Token(".", dbName :: tableName :: colName :: Nil) =>
+                  // It is describing a column with the format like "describe db.table column".
+                  nodeToDescribeFallback(node)
+                case tableName =>
+                  // It is describing a table with the format like "describe table".
+                  datasources.DescribeCommand(
+                    UnresolvedRelation(TableIdentifier(tableName.text), None),
+                    isExtended = extended.isDefined)
+              }
+            // All other cases.
+            case _ => nodeToDescribeFallback(node)
+          }
+        }
+
+      case _ =>
+        super.nodeToPlan(node)
+    }
+  }
+
+  protected def nodeToDescribeFallback(node: ASTNode): LogicalPlan = noParseRule("Describe", node)
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/pom.xml
----------------------------------------------------------------------
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index ffabb92..cd0c2ae 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -262,26 +262,6 @@
         </executions>
       </plugin>
 
-
-      <plugin>
-        <groupId>org.antlr</groupId>
-        <artifactId>antlr3-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>antlr</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <sourceDirectory>${basedir}/src/main/antlr3</sourceDirectory>
-          <includes>
-            <include>**/SparkSqlLexer.g</include>
-            <include>**/SparkSqlParser.g</include>
-          </includes>
-        </configuration>
-      </plugin>
-
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g
deleted file mode 100644
index e4a80f0..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g
+++ /dev/null
@@ -1,330 +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.
-*/
-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.errors.add(new ParseError(gParent, e, tokenNames));
-  }
-  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 ( KW_ON! expression {$joinToken.start.getType() != COMMA}? )? )*
-    | uniqueJoinToken^ uniqueJoinSource (COMMA! uniqueJoinSource)+
-    ;
-
-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
-    | 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_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
-    | (LPAREN) => LPAREN joinSource RPAREN -> joinSource
-    | fromSource0
-    ;
-
-
-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); }
-    :
-    db=identifier DOT tab=identifier
-    -> ^(TOK_TABNAME $db $tab)
-    |
-    tab=identifier
-    -> ^(TOK_TABNAME $tab)
-    ;
-
-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+))
-    ;
-
-//-----------------------------------------------------------------------------------


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
index 31d82eb..bf3fe12 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -17,41 +17,30 @@
 
 package org.apache.spark.sql.hive
 
-import java.sql.Date
 import java.util.Locale
 
 import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
 
+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.{Context, ErrorMsg}
-import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry}
-import org.apache.hadoop.hive.ql.lib.Node
-import org.apache.hadoop.hive.ql.parse.SemanticException
-import org.apache.hadoop.hive.ql.plan.PlanUtils
+import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo}
+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.Logging
-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._
-import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.parser._
+import org.apache.spark.sql.catalyst.parser.ParseUtils._
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.execution.ExplainCommand
-import org.apache.spark.sql.execution.datasources.DescribeCommand
-import org.apache.spark.sql.hive.HiveShim._
+import org.apache.spark.sql.execution.SparkQl
+import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper
 import org.apache.spark.sql.hive.client._
-import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema}
-import org.apache.spark.sql.parser._
+import org.apache.spark.sql.hive.execution.{HiveNativeCommand, AnalyzeTable, DropTable, HiveScriptIOSchema}
 import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
-import org.apache.spark.util.random.RandomSampler
+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
@@ -71,7 +60,7 @@ private[hive] case class CreateTableAsSelect(
   override def output: Seq[Attribute] = Seq.empty[Attribute]
   override lazy val resolved: Boolean =
     tableDesc.specifiedDatabase.isDefined &&
-    tableDesc.schema.size > 0 &&
+    tableDesc.schema.nonEmpty &&
     tableDesc.serde.isDefined &&
     tableDesc.inputFormat.isDefined &&
     tableDesc.outputFormat.isDefined &&
@@ -89,7 +78,7 @@ private[hive] case class CreateViewAsSelect(
 }
 
 /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */
-private[hive] object HiveQl extends Logging {
+private[hive] object HiveQl extends SparkQl with Logging {
   protected val nativeCommands = Seq(
     "TOK_ALTERDATABASE_OWNER",
     "TOK_ALTERDATABASE_PROPERTIES",
@@ -181,103 +170,6 @@ private[hive] object HiveQl extends Logging {
   protected val hqlParser = new ExtendedHiveQlParser
 
   /**
-   * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations
-   * similar to [[catalyst.trees.TreeNode]].
-   *
-   * Note that this should be considered very experimental and is not indented as a replacement
-   * for TreeNode.  Primarily it should be noted ASTNodes are not immutable and do not appear to
-   * have clean copy semantics.  Therefore, users of this class should take care when
-   * copying/modifying trees that might be used elsewhere.
-   */
-  implicit class TransformableNode(n: ASTNode) {
-    /**
-     * Returns a copy of this node where `rule` has been recursively applied to it and all of its
-     * children.  When `rule` does not apply to a given node it is left unchanged.
-     * @param rule the function use to transform this nodes children
-     */
-    def transform(rule: PartialFunction[ASTNode, ASTNode]): ASTNode = {
-      try {
-        val afterRule = rule.applyOrElse(n, identity[ASTNode])
-        afterRule.withChildren(
-          nilIfEmpty(afterRule.getChildren)
-            .asInstanceOf[Seq[ASTNode]]
-            .map(ast => Option(ast).map(_.transform(rule)).orNull))
-      } catch {
-        case e: Exception =>
-          logError(dumpTree(n).toString)
-          throw e
-      }
-    }
-
-    /**
-     * Returns a scala.Seq equivalent to [s] or Nil if [s] is null.
-     */
-    private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] =
-      Option(s).map(_.asScala).getOrElse(Nil)
-
-    /**
-     * Returns this ASTNode with the text changed to `newText`.
-     */
-    def withText(newText: String): ASTNode = {
-      n.token.asInstanceOf[org.antlr.runtime.CommonToken].setText(newText)
-      n
-    }
-
-    /**
-     * Returns this ASTNode with the children changed to `newChildren`.
-     */
-    def withChildren(newChildren: Seq[ASTNode]): ASTNode = {
-      (1 to n.getChildCount).foreach(_ => n.deleteChild(0))
-      newChildren.foreach(n.addChild(_))
-      n
-    }
-
-    /**
-     * Throws an error if this is not equal to other.
-     *
-     * Right now this function only checks the name, type, text and children of the node
-     * for equality.
-     */
-    def checkEquals(other: ASTNode): Unit = {
-      def check(field: String, f: ASTNode => Any): Unit = if (f(n) != f(other)) {
-        sys.error(s"$field does not match for trees. " +
-          s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}")
-      }
-      check("name", _.getName)
-      check("type", _.getType)
-      check("text", _.getText)
-      check("numChildren", n => nilIfEmpty(n.getChildren).size)
-
-      val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]]
-      val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]]
-      leftChildren zip rightChildren foreach {
-        case (l, r) => l checkEquals r
-      }
-    }
-  }
-
-  /**
-   * Returns the AST for the given SQL string.
-   */
-  def getAst(sql: String): ASTNode = {
-    /*
-     * Context has to be passed in hive0.13.1.
-     * Otherwise, there will be Null pointer exception,
-     * when retrieving properties form HiveConf.
-     */
-    val hContext = createContext()
-    val node = getAst(sql, hContext)
-    hContext.clear()
-    node
-  }
-
-  private def createContext(): Context = new Context(hiveConf)
-
-  private def getAst(sql: String, context: Context) =
-    ParseUtils.findRootNonNullToken(
-        (new ParseDriver).parse(sql, context))
-
-  /**
    * Returns the HiveConf
    */
   private[this] def hiveConf: HiveConf = {
@@ -296,226 +188,16 @@ private[hive] object HiveQl extends Logging {
   /** Returns a LogicalPlan for a given HiveQL string. */
   def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql)
 
-  val errorRegEx = "line (\\d+):(\\d+) (.*)".r
-
-  /** Creates LogicalPlan for a given HiveQL string. */
-  def createPlan(sql: String): LogicalPlan = {
-    try {
-      val context = createContext()
-      val tree = getAst(sql, context)
-      val plan = if (nativeCommands contains tree.getText) {
-        HiveNativeCommand(sql)
-      } else {
-        nodeToPlan(tree, context) match {
-          case NativePlaceholder => HiveNativeCommand(sql)
-          case other => other
-        }
-      }
-      context.clear()
-      plan
-    } catch {
-      case pe: ParseException =>
-        pe.getMessage match {
-          case errorRegEx(line, start, message) =>
-            throw new AnalysisException(message, Some(line.toInt), Some(start.toInt))
-          case otherMessage =>
-            throw new AnalysisException(otherMessage)
-        }
-      case e: MatchError => throw e
-      case e: Exception =>
-        throw new AnalysisException(e.getMessage)
-      case e: NotImplementedError =>
-        throw new AnalysisException(
-          s"""
-            |Unsupported language features in query: $sql
-            |${dumpTree(getAst(sql))}
-            |$e
-            |${e.getStackTrace.head}
-          """.stripMargin)
-    }
-  }
-
-  def parseDdl(ddl: String): Seq[Attribute] = {
-    val tree =
-      try {
-        ParseUtils.findRootNonNullToken(
-          (new ParseDriver)
-            .parse(ddl, null /* no context required for parsing alone */))
-      } catch {
-        case pe: org.apache.hadoop.hive.ql.parse.ParseException =>
-          throw new RuntimeException(s"Failed to parse ddl: '$ddl'", pe)
-      }
-    assert(tree.asInstanceOf[ASTNode].getText == "TOK_CREATETABLE", "Only CREATE TABLE supported.")
-    val tableOps = tree.getChildren
-    val colList =
-      tableOps.asScala
-        .find(_.asInstanceOf[ASTNode].getText == "TOK_TABCOLLIST")
-        .getOrElse(sys.error("No columnList!")).getChildren
-
-    colList.asScala.map(nodeToAttribute)
-  }
-
-  /** Extractor for matching Hive's AST Tokens. */
-  private[hive] case class Token(name: String, children: Seq[ASTNode]) extends Node {
-    def getName(): String = name
-    def getChildren(): java.util.List[Node] = {
-      val col = new java.util.ArrayList[Node](children.size)
-      children.foreach(col.add(_))
-      col
-    }
-  }
-  object Token {
-    /** @return matches of the form (tokenName, children). */
-    def unapply(t: Any): Option[(String, Seq[ASTNode])] = t match {
-      case t: ASTNode =>
-        CurrentOrigin.setPosition(t.getLine, t.getCharPositionInLine)
-        Some((t.getText,
-          Option(t.getChildren).map(_.asScala.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]]))
-      case t: Token => Some((t.name, t.children))
-      case _ => None
-    }
-  }
-
-  protected def getClauses(
-      clauseNames: Seq[String],
-      nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = {
-    var remainingNodes = nodeList
-    val clauses = clauseNames.map { clauseName =>
-      val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName)
-      remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil)
-      matches.headOption
-    }
-
-    if (remainingNodes.nonEmpty) {
-      sys.error(
-        s"""Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}.
-           |You are likely trying to use an unsupported Hive feature."""".stripMargin)
-    }
-    clauses
-  }
-
-  def getClause(clauseName: String, nodeList: Seq[Node]): Node =
-    getClauseOption(clauseName, nodeList).getOrElse(sys.error(
-      s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}"))
-
-  def getClauseOption(clauseName: String, nodeList: Seq[Node]): Option[Node] = {
-    nodeList.filter { case ast: ASTNode => ast.getText == clauseName } match {
-      case Seq(oneMatch) => Some(oneMatch)
-      case Seq() => None
-      case _ => sys.error(s"Found multiple instances of clause $clauseName")
-    }
-  }
-
-  protected def nodeToAttribute(node: Node): Attribute = node match {
-    case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) =>
-      AttributeReference(colName, nodeToDataType(dataType), true)()
-
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
-  }
-
-  protected def nodeToDataType(node: Node): DataType = node match {
-    case Token("TOK_DECIMAL", precision :: scale :: Nil) =>
-      DecimalType(precision.getText.toInt, scale.getText.toInt)
-    case Token("TOK_DECIMAL", precision :: Nil) =>
-      DecimalType(precision.getText.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_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 a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for DataType:\n ${dumpTree(a).toString} ")
-  }
-
-  protected def nodeToStructField(node: Node): StructField = node match {
-    case Token("TOK_TABCOL",
-           Token(fieldName, Nil) ::
-           dataType :: Nil) =>
-      StructField(fieldName, nodeToDataType(dataType), nullable = true)
-    case Token("TOK_TABCOL",
-           Token(fieldName, Nil) ::
-             dataType ::
-             _ /* comment */:: Nil) =>
-      StructField(fieldName, nodeToDataType(dataType), nullable = true)
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ")
-  }
-
-  protected def extractTableIdent(tableNameParts: Node): TableIdentifier = {
-    tableNameParts.getChildren.asScala.map {
-      case Token(part, Nil) => cleanIdentifier(part)
-    } match {
-      case Seq(tableOnly) => TableIdentifier(tableOnly)
-      case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName))
-      case other => sys.error("Hive only supports tables names like 'tableName' " +
-        s"or 'databaseName.tableName', found '$other'")
-    }
-  }
-
-  /**
-   * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2))
-   * is equivalent to
-   * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2
-   * Check the following link for details.
-   *
-https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup
-   *
-   * The bitmask denotes the grouping expressions validity for a grouping set,
-   * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive)
-   * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of
-   * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively.
-   */
-  protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = {
-    val (keyASTs, setASTs) = children.partition( n => n match {
-        case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets
-        case _ => true // grouping keys
-      })
-
-    val keys = keyASTs.map(nodeToExpr).toSeq
-    val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap
-
-    val bitmasks: Seq[Int] = setASTs.map(set => set match {
-      case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0
-      case Token("TOK_GROUPING_SETS_EXPRESSION", children) =>
-        children.foldLeft(0)((bitmap, col) => {
-          val colString = col.asInstanceOf[ASTNode].toStringTree()
-          require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list")
-          bitmap | 1 << keyMap(colString)
-        })
-      case _ => sys.error("Expect GROUPING SETS clause")
-    })
-
-    (keys, bitmasks)
-  }
-
-  protected def getProperties(node: Node): Seq[(String, String)] = node match {
+  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))
+          unquoteString(key) -> unquoteString(value)
       }
   }
 
   private def createView(
       view: ASTNode,
-      context: Context,
       viewNameParts: ASTNode,
       query: ASTNode,
       schema: Seq[HiveColumn],
@@ -524,8 +206,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
       replace: Boolean): CreateViewAsSelect = {
     val TableIdentifier(viewName, dbName) = extractTableIdent(viewNameParts)
 
-    val originalText = context.getTokenRewriteStream
-      .toString(query.getTokenStartIndex, query.getTokenStopIndex)
+    val originalText = query.source
 
     val tableDesc = HiveTable(
       specifiedDatabase = dbName,
@@ -544,104 +225,67 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
     // 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 = context.getTokenRewriteStream
-      .toString(view.getTokenStartIndex, view.getTokenStopIndex)
-    CreateViewAsSelect(tableDesc, nodeToPlan(query, context), allowExist, replace, sql)
+    val sql = view.source
+    CreateViewAsSelect(tableDesc, nodeToPlan(query), allowExist, replace, sql)
   }
 
-  protected def nodeToPlan(node: ASTNode, context: Context): LogicalPlan = node match {
-    // 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 tableNmae 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)
+  protected override def createPlan(
+      sql: String,
+      node: ASTNode): LogicalPlan = {
+    if (nativeCommands.contains(node.text)) {
+      HiveNativeCommand(sql)
+    } else {
+      nodeToPlan(node) match {
+        case NativePlaceholder => HiveNativeCommand(sql)
+        case plan => plan
       }
-    // Just fake explain for any of the native commands.
-    case Token("TOK_EXPLAIN", explainArgs)
-      if noExplainCommands.contains(explainArgs.head.getText) =>
-      ExplainCommand(OneRowRelation)
-    case Token("TOK_EXPLAIN", explainArgs)
-      if "TOK_CREATETABLE" == explainArgs.head.getText =>
-      val Some(crtTbl) :: _ :: extended :: Nil =
-        getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs)
-      ExplainCommand(
-        nodeToPlan(crtTbl, context),
-        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, context),
-        extended = extended.isDefined)
-
-    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.
-        NativePlaceholder
-      } else {
-        tableType match {
-          case Token("TOK_TABTYPE", Token("TOK_TABNAME", nameParts :: Nil) :: Nil) => {
-            nameParts match {
-              case Token(".", dbName :: tableName :: 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 = extractTableIdent(nameParts)
-                DescribeCommand(
-                  UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined)
-              case Token(".", dbName :: tableName :: colName :: Nil) =>
-                // It is describing a column with the format like "describe db.table column".
-                NativePlaceholder
-              case tableName =>
-                // It is describing a table with the format like "describe table".
-                DescribeCommand(
-                  UnresolvedRelation(TableIdentifier(tableName.getText), None),
-                  isExtended = extended.isDefined)
-            }
-          }
-          // All other cases.
-          case _ => NativePlaceholder
+    }
+  }
+
+  protected override def isNoExplainCommand(command: String): Boolean =
+    noExplainCommands.contains(command)
+
+  protected override def nodeToPlan(node: ASTNode): LogicalPlan = {
+    node match {
+      // 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 tableNmae 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(viewNameParts) :: maybeQuery :: ignores =
-        getClauses(Seq(
-          "TOK_TABNAME",
-          "TOK_QUERY",
-          "TOK_ALTERVIEW_ADDPARTS",
-          "TOK_ALTERVIEW_DROPPARTS",
-          "TOK_ALTERVIEW_PROPERTIES",
-          "TOK_ALTERVIEW_RENAME"), children)
+      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, context, viewNameParts, query, Nil, Map(), false, true)
-      }.getOrElse(NativePlaceholder)
+        // 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)
+      case view @ Token("TOK_CREATEVIEW", children)
         if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty =>
-      val Seq(
+        val Seq(
         Some(viewNameParts),
         Some(query),
         maybeComment,
@@ -650,1224 +294,466 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
         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 =>
-          SemanticAnalyzer.getColumns(cols, true).asScala.map { field =>
+        ) = 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.
-            HiveColumn(field.getName, null, field.getComment)
-          }
-        }.getOrElse(Seq.empty[HiveColumn])
-
-        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 = SemanticAnalyzer.unescapeSQLString(child.getText)
-            if (comment ne null) {
-              properties += ("comment" -> comment)
-            }
-        }
-
-        createView(view, context, 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 +:
-          ignores) =
-        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(tblName, dbName) = extractTableIdent(tableNameParts)
-
-      // TODO add bucket support
-      var tableDesc: HiveTable = HiveTable(
-        specifiedDatabase = dbName,
-        name = tblName,
-        schema = Seq.empty[HiveColumn],
-        partitionColumns = Seq.empty[HiveColumn],
-        properties = Map[String, String](),
-        serdeProperties = Map[String, String](),
-        tableType = if (externalTable.isDefined) ExternalTable else ManagedTable,
-        location = None,
-        inputFormat = None,
-        outputFormat = None,
-        serde = None,
-        viewText = None)
-
-      // 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"))
-      }
+            nodeToColumns(cols, lowerCase = true).map(_.copy(hiveType = null))
+          }.getOrElse(Seq.empty[HiveColumn])
 
-      hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f)))
-      hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f)))
-      hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f)))
+          val properties = scala.collection.mutable.Map.empty[String, String]
 
-      children.collect {
-        case list @ Token("TOK_TABCOLLIST", _) =>
-          val cols = SemanticAnalyzer.getColumns(list, true)
-          if (cols != null) {
-            tableDesc = tableDesc.copy(
-              schema = cols.asScala.map { field =>
-                HiveColumn(field.getName, field.getType, field.getComment)
-              })
+          maybeProperties.foreach {
+            case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
+              properties ++= getProperties(list)
           }
-        case Token("TOK_TABLECOMMENT", child :: Nil) =>
-          val comment = SemanticAnalyzer.unescapeSQLString(child.getText)
-          // TODO support the sql text
-          tableDesc = tableDesc.copy(viewText = Option(comment))
-        case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) =>
-          val cols = SemanticAnalyzer.getColumns(list(0), false)
-          if (cols != null) {
-            tableDesc = tableDesc.copy(
-              partitionColumns = cols.asScala.map { field =>
-                HiveColumn(field.getName, field.getType, field.getComment)
-              })
-          }
-        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 = SemanticAnalyzer.unescapeSQLString (rowChild1.getText())
-              serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim)
-              serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim)
-              if (rowChild2.length > 1) {
-                val fieldEscape = SemanticAnalyzer.unescapeSQLString (rowChild2(0).getText)
-                serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape)
-              }
-            case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) =>
-              val collItemDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
-              serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim)
-            case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) =>
-              val mapKeyDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
-              serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim)
-            case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) =>
-              val lineDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
-              if (!(lineDelim == "\n") && !(lineDelim == "10")) {
-                throw new AnalysisException(
-                  SemanticAnalyzer.generateErrorMessage(
-                    rowChild,
-                    ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg))
-              }
-              serdeParams.put(serdeConstants.LINE_DELIM, lineDelim)
-            case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) =>
-              val nullFormat = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
-              // TODO support the nullFormat
-            case _ => assert(false)
-          }
-          tableDesc = tableDesc.copy(
-            serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala)
-        case Token("TOK_TABLELOCATION", child :: Nil) =>
-          var location = SemanticAnalyzer.unescapeSQLString(child.getText)
-          location = SemanticAnalyzer.relativeToAbsolutePath(hiveConf, location)
-          tableDesc = tableDesc.copy(location = Option(location))
-        case Token("TOK_TABLESERIALIZER", child :: Nil) =>
-          tableDesc = tableDesc.copy(
-            serde = Option(SemanticAnalyzer.unescapeSQLString(child.getChild(0).getText)))
-          if (child.getChildCount == 2) {
-            val serdeParams = new java.util.HashMap[String, String]()
-            SemanticAnalyzer.readProps(
-              (child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams)
-            tableDesc = tableDesc.copy(
-              serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala)
-          }
-        case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) =>
-          child.getText().toLowerCase(Locale.ENGLISH) match {
-            case "orc" =>
-              tableDesc = tableDesc.copy(
-                inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"),
-                outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))
-              if (tableDesc.serde.isEmpty) {
-                tableDesc = tableDesc.copy(
-                  serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))
-              }
-
-            case "parquet" =>
-              tableDesc = tableDesc.copy(
-                inputFormat =
-                  Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
-                outputFormat =
-                  Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
-              if (tableDesc.serde.isEmpty) {
-                tableDesc = tableDesc.copy(
-                  serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
-              }
-
-            case "rcfile" =>
-              tableDesc = tableDesc.copy(
-                inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"),
-                outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
-              if (tableDesc.serde.isEmpty) {
-                tableDesc = tableDesc.copy(
-                  serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))
-              }
 
-            case "textfile" =>
-              tableDesc = tableDesc.copy(
-                inputFormat =
-                  Option("org.apache.hadoop.mapred.TextInputFormat"),
-                outputFormat =
-                  Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
-
-            case "sequencefile" =>
-              tableDesc = tableDesc.copy(
-                inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"),
-                outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))
-
-            case "avro" =>
-              tableDesc = tableDesc.copy(
-                inputFormat =
-                  Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"),
-                outputFormat =
-                  Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"))
-              if (tableDesc.serde.isEmpty) {
-                tableDesc = tableDesc.copy(
-                  serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))
+          maybeComment.foreach {
+            case Token("TOK_TABLECOMMENT", child :: Nil) =>
+              val comment = unescapeSQLString(child.text)
+              if (comment ne null) {
+                properties += ("comment" -> comment)
               }
-
-            case _ =>
-              throw new SemanticException(
-                s"Unrecognized file format in STORED AS clause: ${child.getText}")
-          }
-
-        case Token("TOK_TABLESERIALIZER",
-               Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) =>
-          tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName)))
-
-          otherProps match {
-            case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil =>
-              tableDesc = tableDesc.copy(
-                serdeProperties = tableDesc.serdeProperties ++ getProperties(list))
-            case Nil =>
           }
 
-        case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
-          tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list))
-        case list @ Token("TOK_TABLEFILEFORMAT", children) =>
-          tableDesc = tableDesc.copy(
-            inputFormat =
-              Option(SemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)),
-            outputFormat =
-              Option(SemanticAnalyzer.unescapeSQLString(list.getChild(1).getText)))
-        case Token("TOK_STORAGEHANDLER", _) =>
-          throw new AnalysisException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg())
-        case _ => // Unsupport features
-      }
-
-      CreateTableAsSelect(tableDesc, nodeToPlan(query, context), allowExisting != None)
-
-    // 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 Token("TOK_QUERY", queryArgs)
-        if Seq("TOK_CTE", "TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) =>
-
-      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, context).asInstanceOf[Subquery]
-              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)
+          createView(view, viewNameParts, query, schema, properties.toMap,
+            allowExisting.isDefined, replace.isDefined)
         }
 
-      // Return one query for each insert clause.
-      val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) =>
+      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 (
-            intoClause ::
-            destClause ::
-            selectClause ::
-            selectDistinctClause ::
-            whereClause ::
-            groupByClause ::
-            rollupGroupByClause ::
-            cubeGroupByClause ::
-            groupingSetsClause ::
-            orderByClause ::
-            havingClause ::
-            sortByClause ::
-            clusterByClause ::
-            distributeByClause ::
-            limitClause ::
-            lateralViewClause ::
-            windowClause :: Nil) = {
+          Some(tableNameParts) ::
+            _ /* likeTable */ ::
+            externalTable ::
+            Some(query) ::
+            allowExisting +:
+              _) =
           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)
+              "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(tblName, dbName) = extractTableIdent(tableNameParts)
+
+        // TODO add bucket support
+        var tableDesc: HiveTable = HiveTable(
+          specifiedDatabase = dbName,
+          name = tblName,
+          schema = Seq.empty[HiveColumn],
+          partitionColumns = Seq.empty[HiveColumn],
+          properties = Map[String, String](),
+          serdeProperties = Map[String, String](),
+          tableType = if (externalTable.isDefined) ExternalTable else ManagedTable,
+          location = None,
+          inputFormat = None,
+          outputFormat = None,
+          serde = None,
+          viewText = None)
+
+        // 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"))
         }
 
-        val relations = fromClause match {
-          case Some(f) => nodeToRelation(f, context)
-          case None => OneRowRelation
-        }
+        hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f)))
+        hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f)))
+        hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f)))
 
-        val withWhere = whereClause.map { whereNode =>
-          val Seq(whereExpr) = whereNode.getChildren.asScala
-          Filter(nodeToExpr(whereExpr), relations)
-        }.getOrElse(relations)
-
-        val select =
-          (selectClause orElse selectDistinctClause).getOrElse(sys.error("No select clause."))
-
-        // Script transformations are expressed as a select clause with a single expression of type
-        // TOK_TRANSFORM
-        val transformation = select.getChildren.iterator().next() 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(name, StringType)() },
-                  false)
-              case Token("TOK_TABCOLLIST", attributes) :: Nil =>
-                (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) =>
-                  AttributeReference(name, nodeToDataType(dataType))() }, false)
-              case Nil =>
-                (List(AttributeReference("key", StringType)(),
-                  AttributeReference("value", StringType)()), true)
+        children.collect {
+          case list @ Token("TOK_TABCOLLIST", _) =>
+            val cols = nodeToColumns(list, lowerCase = true)
+            if (cols != null) {
+              tableDesc = tableDesc.copy(schema = cols)
             }
-
-            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)
+          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)
                 }
-                (rowFormat, None, Nil, false)
-
-              case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil =>
-                (Nil, Some(SemanticAnalyzer.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) =>
-                    (SemanticAnalyzer.unescapeSQLString(name),
-                      SemanticAnalyzer.unescapeSQLString(value))
+              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")
                 }
-
-                // SPARK-10310: Special cases LazySimpleSerDe
-                // TODO Fully supports user-defined record reader/writer classes
-                val unescapedSerDeClass = SemanticAnalyzer.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 = SemanticAnalyzer.unescapeSQLString(script)
-
-            // TODO Adds support for user-defined record reader/writer classes
-            val recordReaderClass = if (useDefaultRecordReader) {
-              Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER))
-            } else {
-              None
+                serdeParams.put(serdeConstants.LINE_DELIM, lineDelim)
+              case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) =>
+                val nullFormat = unescapeSQLString(rowChild.text)
+              // TODO support the nullFormat
+              case _ => assert(false)
             }
-
-            val recordWriterClass = if (useDefaultRecordWriter) {
-              Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDWRITER))
-            } else {
-              None
+            tableDesc = tableDesc.copy(
+              serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala)
+          case Token("TOK_TABLELOCATION", child :: Nil) =>
+            val location = EximUtil.relativeToAbsolutePath(hiveConf, unescapeSQLString(child.text))
+            tableDesc = tableDesc.copy(location = Option(location))
+          case Token("TOK_TABLESERIALIZER", child :: Nil) =>
+            tableDesc = tableDesc.copy(
+              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.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams)
             }
+          case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) =>
+            child.text.toLowerCase(Locale.ENGLISH) match {
+              case "orc" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"),
+                  outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))
+                if (tableDesc.serde.isEmpty) {
+                  tableDesc = tableDesc.copy(
+                    serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))
+                }
 
-            val schema = HiveScriptIOSchema(
-              inRowFormat, outRowFormat,
-              inSerdeClass, outSerdeClass,
-              inSerdeProps, outSerdeProps,
-              recordReaderClass, recordWriterClass,
-              schemaLess)
-
-            Some(
-              logical.ScriptTransformation(
-                inputExprs.map(nodeToExpr),
-                unescapedScript,
-                output,
-                withWhere, schema))
-          case _ => None
-        }
-
-        val withLateralView = lateralViewClause.map { lv =>
-          val Token("TOK_SELECT",
-          Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.iterator().next()
-
-          val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next()
-            .asInstanceOf[ASTNode].getText
-
-          val (generator, attributes) = nodesToGenerator(clauses)
-            Generate(
-              generator,
-              join = true,
-              outer = false,
-              Some(alias.toLowerCase),
-              attributes.map(UnresolvedAttribute(_)),
-              withWhere)
-        }.getOrElse(withWhere)
-
-        // The projection of the query can either be a normal projection, an aggregation
-        // (if there is a group by) or a script transformation.
-        val withProject: LogicalPlan = transformation.getOrElse {
-          val selectExpressions =
-            select.getChildren.asScala.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_))
-          Seq(
-            groupByClause.map(e => e match {
-              case Token("TOK_GROUPBY", children) =>
-                // Not a transformation so must be either project or aggregation.
-                Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView)
-              case _ => sys.error("Expect GROUP BY")
-            }),
-            groupingSetsClause.map(e => e match {
-              case Token("TOK_GROUPING_SETS", children) =>
-                val(groupByExprs, masks) = extractGroupingSet(children)
-                GroupingSets(masks, groupByExprs, withLateralView, selectExpressions)
-              case _ => sys.error("Expect GROUPING SETS")
-            }),
-            rollupGroupByClause.map(e => e match {
-              case Token("TOK_ROLLUP_GROUPBY", children) =>
-                Aggregate(Seq(Rollup(children.map(nodeToExpr))), selectExpressions, withLateralView)
-              case _ => sys.error("Expect WITH ROLLUP")
-            }),
-            cubeGroupByClause.map(e => e match {
-              case Token("TOK_CUBE_GROUPBY", children) =>
-                Aggregate(Seq(Cube(children.map(nodeToExpr))), selectExpressions, withLateralView)
-              case _ => sys.error("Expect WITH CUBE")
-            }),
-            Some(Project(selectExpressions, withLateralView))).flatten.head
-        }
+              case "parquet" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat =
+                    Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+                  outputFormat =
+                    Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
+                if (tableDesc.serde.isEmpty) {
+                  tableDesc = tableDesc.copy(
+                    serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
+                }
 
-        // Handle HAVING clause.
-        val withHaving = havingClause.map { h =>
-          val havingExpr = h.getChildren.asScala 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.getChildren.asScala.map(nodeToSortOrder), true, withDistinct)
-            case (None, Some(perPartitionOrdering), None, None) =>
-              Sort(
-                perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder),
-                false, withDistinct)
-            case (None, None, Some(partitionExprs), None) =>
-              RepartitionByExpression(
-                partitionExprs.getChildren.asScala.map(nodeToExpr), withDistinct)
-            case (None, Some(perPartitionOrdering), Some(partitionExprs), None) =>
-              Sort(
-                perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), false,
-                RepartitionByExpression(
-                  partitionExprs.getChildren.asScala.map(nodeToExpr),
-                  withDistinct))
-            case (None, None, None, Some(clusterExprs)) =>
-              Sort(
-                clusterExprs.getChildren.asScala.map(nodeToExpr).map(SortOrder(_, Ascending)),
-                false,
-                RepartitionByExpression(
-                  clusterExprs.getChildren.asScala.map(nodeToExpr),
-                  withDistinct))
-            case (None, None, None, None) => withDistinct
-            case _ => sys.error("Unsupported set of ordering / distribution clauses.")
-          }
+              case "rcfile" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"),
+                  outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
+                if (tableDesc.serde.isEmpty) {
+                  tableDesc = tableDesc.copy(serde =
+                    Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))
+                }
 
-        val withLimit =
-          limitClause.map(l => nodeToExpr(l.getChildren.iterator().next()))
-            .map(Limit(_, withSort))
-            .getOrElse(withSort)
-
-        // Collect all window specifications defined in the WINDOW clause.
-        val windowDefinitions = windowClause.map(_.getChildren.asScala.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)]
-          }
-        }
+              case "textfile" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat =
+                    Option("org.apache.hadoop.mapred.TextInputFormat"),
+                  outputFormat =
+                    Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
 
-        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)
-      }
+              case "sequencefile" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"),
+                  outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))
 
-      // If there are multiple INSERTS just UNION them together into on query.
-      val query = queries.reduceLeft(Union)
+              case "avro" =>
+                tableDesc = tableDesc.copy(
+                  inputFormat =
+                    Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"),
+                  outputFormat =
+                    Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"))
+                if (tableDesc.serde.isEmpty) {
+                  tableDesc = tableDesc.copy(
+                    serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))
+                }
 
-      // return With plan if there is CTE
-      cteRelations.map(With(query, _)).getOrElse(query)
+              case _ =>
+                throw new AnalysisException(
+                  s"Unrecognized file format in STORED AS clause: ${child.text}")
+            }
 
-    // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT
-    case Token("TOK_UNIONALL", left :: right :: Nil) =>
-      Union(nodeToPlan(left, context), nodeToPlan(right, context))
+          case Token("TOK_TABLESERIALIZER",
+          Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) =>
+            tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName)))
 
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for $node:\n ${dumpTree(a).toString} ")
-  }
+            otherProps match {
+              case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil =>
+                tableDesc = tableDesc.copy(
+                  serdeProperties = tableDesc.serdeProperties ++ getProperties(list))
+              case _ =>
+            }
 
-  val allJoinTokens = "(TOK_.*JOIN)".r
-  val laterViewToken = "TOK_LATERAL_VIEW(.*)".r
-  def nodeToRelation(node: Node, context: Context): LogicalPlan = node match {
-    case Token("TOK_SUBQUERY",
-           query :: Token(alias, Nil) :: Nil) =>
-      Subquery(cleanIdentifier(alias), nodeToPlan(query, context))
-
-    case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) =>
-      val Token("TOK_SELECT",
-            Token("TOK_SELEXPR", clauses) :: Nil) = selectClause
-
-      val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next()
-        .asInstanceOf[ASTNode].getText
-
-      val (generator, attributes) = nodesToGenerator(clauses)
-        Generate(
-          generator,
-          join = true,
-          outer = isOuter.nonEmpty,
-          Some(alias.toLowerCase),
-          attributes.map(UnresolvedAttribute(_)),
-          nodeToRelation(relationClause, context))
-
-    /* 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.getText.startsWith("TOK")) {
-          (clauses, None)
-        } else {
-          (clauses.dropRight(1), Some(clauses.last))
+          case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
+            tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list))
+          case list @ Token("TOK_TABLEFILEFORMAT", _) =>
+            tableDesc = tableDesc.copy(
+              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 _ => // Unsupport features
         }
 
-      val (Some(tableNameParts) ::
-          splitSampleClause ::
-          bucketSampleClause :: Nil) = {
-        getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"),
-          nonAliasClauses)
-      }
-
-      val tableIdent = extractTableIdent(tableNameParts)
-      val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) }
-      val relation = UnresolvedRelation(tableIdent, alias)
-
-      // Apply sampling if requested.
-      (bucketSampleClause orElse splitSampleClause).map {
-        case Token("TOK_TABLESPLITSAMPLE",
-               Token("TOK_ROWCOUNT", Nil) ::
-               Token(count, Nil) :: Nil) =>
-          Limit(Literal(count.toInt), relation)
-        case Token("TOK_TABLESPLITSAMPLE",
-               Token("TOK_PERCENT", Nil) ::
-               Token(fraction, Nil) :: Nil) =>
-          // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
-          // function takes X PERCENT as the input and the range of X is [0, 100], we need to
-          // adjust the fraction.
-          require(
-            fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon)
-              && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon),
-            s"Sampling fraction ($fraction) must be on interval [0, 100]")
-          Sample(0.0, fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt,
-            relation)
-        case Token("TOK_TABLEBUCKETSAMPLE",
-               Token(numerator, Nil) ::
-               Token(denominator, Nil) :: Nil) =>
-          val fraction = numerator.toDouble / denominator.toDouble
-          Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation)
-        case a: ASTNode =>
-          throw new NotImplementedError(
-            s"""No parse rules for sampling clause: ${a.getType}, text: ${a.getText} :
-           |${dumpTree(a).toString}" +
-         """.stripMargin)
-      }.getOrElse(relation)
-
-    case Token("TOK_UNIQUEJOIN", joinArgs) =>
-      val tableOrdinals =
-        joinArgs.zipWithIndex.filter {
-          case (arg, i) => arg.getText == "TOK_TABREF"
-        }.map(_._2)
-
-      val isPreserved = tableOrdinals.map(i => (i - 1 < 0) || joinArgs(i - 1).getText == "PRESERVE")
-      val tables = tableOrdinals.map(i => nodeToRelation(joinArgs(i), context))
-      val joinExpressions =
-        tableOrdinals.map(i => joinArgs(i + 1).getChildren.asScala.map(nodeToExpr))
-
-      val joinConditions = joinExpressions.sliding(2).map {
-        case Seq(c1, c2) =>
-          val predicates = (c1, c2).zipped.map { case (e1, e2) => EqualTo(e1, e2): Expression }
-          predicates.reduceLeft(And)
-      }.toBuffer
-
-      val joinType = isPreserved.sliding(2).map {
-        case Seq(true, true) => FullOuter
-        case Seq(true, false) => LeftOuter
-        case Seq(false, true) => RightOuter
-        case Seq(false, false) => Inner
-      }.toBuffer
-
-      val joinedTables = tables.reduceLeft(Join(_, _, Inner, None))
-
-      // Must be transform down.
-      val joinedResult = joinedTables transform {
-        case j: Join =>
-          j.copy(
-            condition = Some(joinConditions.remove(joinConditions.length - 1)),
-            joinType = joinType.remove(joinType.length - 1))
-      }
-
-      val groups = joinExpressions.head.indices.map(i => Coalesce(joinExpressions.map(_(i))))
-
-      // Unique join is not really the same as an outer join so we must group together results where
-      // the joinExpressions are the same, taking the First of each value is only okay because the
-      // user of a unique join is implicitly promising that there is only one result.
-      // TODO: This doesn't actually work since [[Star]] is not a valid aggregate expression.
-      // instead we should figure out how important supporting this feature is and whether it is
-      // worth the number of hacks that will be required to implement it.  Namely, we need to add
-      // some sort of mapped star expansion that would expand all child output row to be similarly
-      // named output expressions where some aggregate expression has been applied (i.e. First).
-      // Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult)
-      throw new UnsupportedOperationException
-
-    case Token(allJoinTokens(joinToken),
-           relation1 ::
-           relation2 :: other) =>
-      if (!(other.size <= 1)) {
-        sys.error(s"Unsupported join operation: $other")
-      }
-
-      val joinType = joinToken match {
-        case "TOK_JOIN" => Inner
-        case "TOK_CROSSJOIN" => Inner
-        case "TOK_RIGHTOUTERJOIN" => RightOuter
-        case "TOK_LEFTOUTERJOIN" => LeftOuter
-        case "TOK_FULLOUTERJOIN" => FullOuter
-        case "TOK_LEFTSEMIJOIN" => LeftSemi
-        case "TOK_ANTIJOIN" => throw new NotImplementedError("Anti join not supported")
-      }
-      Join(nodeToRelation(relation1, context),
-        nodeToRelation(relation2, context),
-        joinType,
-        other.headOption.map(nodeToExpr))
-
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
-  }
+        CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting.isDefined)
 
-  def nodeToSortOrder(node: Node): SortOrder = node match {
-    case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) =>
-      SortOrder(nodeToExpr(sortExpr), Ascending)
-    case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) =>
-      SortOrder(nodeToExpr(sortExpr), Descending)
+      // If its not a "CTAS" like above then take it as a native command
+      case Token("TOK_CREATETABLE", _) =>
+        NativePlaceholder
 
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
-  }
+      // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]"
+      case Token("TOK_TRUNCATETABLE", Token("TOK_TABLE_PARTITION", table) :: Nil) =>
+        NativePlaceholder
 
-  val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r
-  protected def nodeToDest(
-      node: Node,
-      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(_.getChildren.asScala.map {
-        // Parse partitions. We also make keys case insensitive.
-        case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
-          cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value))
-        case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) =>
-          cleanIdentifier(key.toLowerCase) -> None
-      }.toMap).getOrElse(Map.empty)
-
-      InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, 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(_.getChildren.asScala.map {
-        // Parse partitions. We also make keys case insensitive.
-        case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
-          cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value))
-        case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) =>
-          cleanIdentifier(key.toLowerCase) -> None
-      }.toMap).getOrElse(Map.empty)
-
-      InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true)
-
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for ${a.getName}:" +
-          s"\n ${dumpTree(a).toString} ")
+      case _ =>
+        super.nodeToPlan(node)
+    }
   }
 
-  protected def selExprNodeToExpr(node: Node): 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) =>
-      var aliasNames = ArrayBuffer[String]()
-      aliasChildren.foreach { _ match {
-        case Token(name, Nil) => aliasNames += cleanIdentifier(name)
+  protected override def nodeToDescribeFallback(node: ASTNode): LogicalPlan = NativePlaceholder
+
+  protected override def nodeToTransformation(
+      node: ASTNode,
+      child: LogicalPlan): Option[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(name, StringType)() },
+            false)
+        case Token("TOK_TABCOLLIST", attributes) :: Nil =>
+          (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) =>
+            AttributeReference(name, nodeToDataType(dataType))() }, false)
+        case Nil =>
+          (List(AttributeReference("key", StringType)(),
+            AttributeReference("value", StringType)()), true)
         case _ =>
-        }
+          noParseRule("Transform", node)
       }
-      Some(MultiAlias(nodeToExpr(e), aliasNames))
-
-    /* Hints are ignored */
-    case Token("TOK_HINTLIST", _) => None
 
-    case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for ${a.getName }:" +
-          s"\n ${dumpTree(a).toString } ")
-  }
-
-  protected val escapedIdentifier = "`([^`]+)`".r
-  protected val doubleQuotedString = "\"([^\"]+)\"".r
-  protected val singleQuotedString = "'([^']+)'".r
+      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)
 
-  protected def unquoteString(str: String) = str match {
-    case singleQuotedString(s) => s
-    case doubleQuotedString(s) => s
-    case other => other
-  }
+        case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil =>
+          (Nil, Some(unescapeSQLString(serdeClass)), Nil, false)
 
-  /** Strips backticks from ident if present */
-  protected def cleanIdentifier(ident: String): String = ident match {
-    case escapedIdentifier(i) => i
-    case plainIdent => plainIdent
-  }
+        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))
+          }
 
-  val numericAstTypes = Seq(
-    SparkSqlParser.Number,
-    SparkSqlParser.TinyintLiteral,
-    SparkSqlParser.SmallintLiteral,
-    SparkSqlParser.BigintLiteral,
-    SparkSqlParser.DecimalLiteral)
-
-  /* Case insensitive matches */
-  val COUNT = "(?i)COUNT".r
-  val SUM = "(?i)SUM".r
-  val AND = "(?i)AND".r
-  val OR = "(?i)OR".r
-  val NOT = "(?i)NOT".r
-  val TRUE = "(?i)TRUE".r
-  val FALSE = "(?i)FALSE".r
-  val LIKE = "(?i)LIKE".r
-  val RLIKE = "(?i)RLIKE".r
-  val REGEXP = "(?i)REGEXP".r
-  val IN = "(?i)IN".r
-  val DIV = "(?i)DIV".r
-  val BETWEEN = "(?i)BETWEEN".r
-  val WHEN = "(?i)WHEN".r
-  val CASE = "(?i)CASE".r
-
-  protected def nodeToExpr(node: Node): 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(attr))
+          // 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)
       }
 
-    /* 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", Token(name, Nil) :: Nil) :: Nil) =>
-      UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(name)))
-
-    /* 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.getText.toInt, scale.getText.toInt))
-    case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) =>
-      Cast(nodeToExpr(arg), DecimalType(precision.getText.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)
-      }
+      val (inRowFormat, inSerdeClass, inSerdeProps, useDefaultRecordReader) =
+        matchSerDe(inputSerdeClause)
 
-    /* Boolean Logic */
-    case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right))
-    case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right))
-    case Token(NOT(), child :: Nil) => Not(nodeToExpr(child))
-    case Token("!", child :: Nil) => Not(nodeToExpr(child))
-
-    /* Case statements */
-    case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) =>
-      CaseWhen(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(name, args :+ Token("TOK_WINDOWSPEC", spec)) =>
-      val function = nodeToExpr(Token(name, args))
-      nodesToWindowSpecification(spec) match {
-        case reference: WindowSpecReference =>
-          UnresolvedWindowExpression(function, reference)
-        case definition: WindowSpecDefinition =>
-          WindowExpression(function, definition)
-      }
+      val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) =
+        matchSerDe(outputSerdeClause)
 
-    /* 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 => SemanticAnalyzer.unescapeSQLString(s.getText)).mkString)
-
-    // This code is adapted from
-    // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223
-    case ast: ASTNode if numericAstTypes contains ast.getType =>
-      var v: Literal = null
-      try {
-        if (ast.getText.endsWith("L")) {
-          // Literal bigint.
-          v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType)
-        } else if (ast.getText.endsWith("S")) {
-          // Literal smallint.
-          v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType)
-        } else if (ast.getText.endsWith("Y")) {
-          // Literal tinyint.
-          v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType)
-        } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) {
-          // Literal decimal
-          val strVal = ast.getText.stripSuffix("D").stripSuffix("B")
-          v = Literal(Decimal(strVal))
-        } else {
-          v = Literal.create(ast.getText.toDouble, DoubleType)
-          v = Literal.create(ast.getText.toLong, LongType)
-          v = Literal.create(ast.getText.toInt, IntegerType)
-        }
-      } catch {
-        case nfe: NumberFormatException => // Do nothing
-      }
+      val unescapedScript = unescapeSQLString(script)
 
-      if (v == null) {
-        sys.error(s"Failed to parse number '${ast.getText}'.")
+      // TODO Adds support for user-defined record reader/writer classes
+      val recordReaderClass = if (useDefaultRecordReader) {
+        Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER))
       } else {
-        v
+        None
       }
 
-    case ast: ASTNode if ast.getType == SparkSqlParser.StringLiteral =>
-      Literal(SemanticAnalyzer.unescapeSQLString(ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_DATELITERAL =>
-      Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1)))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_CHARSETLITERAL =>
-      Literal(SemanticAnalyzer.charSetString(ast.getChild(0).getText, ast.getChild(1).getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL =>
-      Literal(CalendarInterval.fromYearMonthString(ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL =>
-      Literal(CalendarInterval.fromDayTimeString(ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_LITERAL =>
-      Literal(CalendarInterval.fromSingleUnitString("year", ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MONTH_LITERAL =>
-      Literal(CalendarInterval.fromSingleUnitString("month", ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_LITERAL =>
-      Literal(CalendarInterval.fromSingleUnitString("day", ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_HOUR_LITERAL =>
-      Literal(CalendarInterval.fromSingleUnitString("hour", ast.getText))
-
-    case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MINUTE_LITERAL =>
-      Literal(CalendarInterval.fromSingleUnitString("minute", ast.getText))
-
-    case ast: ASTNode if ast.g

<TRUNCATED>

---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
[SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

This PR moves a major part of the new SQL parser to Catalyst. This is a prelude to start using this parser for all of our SQL parsing. The following key changes have been made:

The ANTLR Parser & Supporting classes have been moved to the Catalyst project. They are now part of the ```org.apache.spark.sql.catalyst.parser``` package. These classes contained quite a bit of code that was originally from the Hive project, I have added aknowledgements whenever this applied. All Hive dependencies have been factored out. I have also taken this chance to clean-up the ```ASTNode``` class, and to improve the error handling.

The HiveQl object that provides the functionality to convert an AST into a LogicalPlan has been refactored into three different classes, one for every SQL sub-project:
- ```CatalystQl```: This implements Query and Expression parsing functionality.
- ```SparkQl```: This is a subclass of CatalystQL and provides SQL/Core only functionality such as Explain and Describe.
- ```HiveQl```: This is a subclass of ```SparkQl``` and this adds Hive-only functionality to the parser such as Analyze, Drop, Views, CTAS & Transforms. This class still depends on Hive.

cc rxin

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

Closes #10583 from hvanhovell/SPARK-12575.


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

Branch: refs/heads/master
Commit: ea489f14f11b2fdfb44c86634d2e2c2167b6ea18
Parents: 3aa3488
Author: Herman van Hovell <hv...@questtec.nl>
Authored: Wed Jan 6 11:16:53 2016 -0800
Committer: Reynold Xin <rx...@databricks.com>
Committed: Wed Jan 6 11:16:53 2016 -0800

----------------------------------------------------------------------
 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 +-
 pom.xml                                         |    6 +
 project/SparkBuild.scala                        |  104 +-
 sql/catalyst/pom.xml                            |   22 +
 .../sql/catalyst/parser/FromClauseParser.g      |  332 +++
 .../sql/catalyst/parser/IdentifiersParser.g     |  699 +++++
 .../sql/catalyst/parser/SelectClauseParser.g    |  228 ++
 .../spark/sql/catalyst/parser/SparkSqlLexer.g   |  485 ++++
 .../spark/sql/catalyst/parser/SparkSqlParser.g  | 2462 ++++++++++++++++++
 .../spark/sql/catalyst/parser/ParseUtils.java   |  162 ++
 .../apache/spark/sql/catalyst/CatalystQl.scala  |  961 +++++++
 .../spark/sql/catalyst/parser/ASTNode.scala     |   93 +
 .../spark/sql/catalyst/parser/ParseDriver.scala |  156 ++
 .../spark/sql/catalyst/parser/ParserConf.scala  |   26 +
 .../scala/org/apache/spark/sql/SQLConf.scala    |   20 +-
 .../apache/spark/sql/execution/SparkQl.scala    |   84 +
 sql/hive/pom.xml                                |   20 -
 .../apache/spark/sql/parser/FromClauseParser.g  |  330 ---
 .../apache/spark/sql/parser/IdentifiersParser.g |  697 -----
 .../spark/sql/parser/SelectClauseParser.g       |  226 --
 .../org/apache/spark/sql/parser/SparkSqlLexer.g |  474 ----
 .../apache/spark/sql/parser/SparkSqlParser.g    | 2457 -----------------
 .../apache/spark/sql/parser/ASTErrorNode.java   |   49 -
 .../org/apache/spark/sql/parser/ASTNode.java    |  245 --
 .../apache/spark/sql/parser/ParseDriver.java    |  213 --
 .../org/apache/spark/sql/parser/ParseError.java |   54 -
 .../apache/spark/sql/parser/ParseException.java |   51 -
 .../org/apache/spark/sql/parser/ParseUtils.java |   96 -
 .../spark/sql/parser/SemanticAnalyzer.java      |  406 ---
 .../org/apache/spark/sql/hive/HiveQl.scala      | 2050 ++++-----------
 .../spark/sql/hive/ErrorPositionSuite.scala     |    9 +-
 34 files changed, 6267 insertions(+), 6966 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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 44727f9..e4373f7 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -5,8 +5,7 @@ activation-1.1.jar
 akka-actor_2.10-2.3.11.jar
 akka-remote_2.10-2.3.11.jar
 akka-slf4j_2.10-2.3.11.jar
-antlr-2.7.7.jar
-antlr-runtime-3.4.jar
+antlr-runtime-3.5.2.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
 arpack_combined_all-0.1.jar
@@ -179,7 +178,6 @@ spire_2.10-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
 tachyon-client-0.8.2.jar
 tachyon-underfs-hdfs-0.8.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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 6014d50..7478181 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -5,8 +5,7 @@ activation-1.1.1.jar
 akka-actor_2.10-2.3.11.jar
 akka-remote_2.10-2.3.11.jar
 akka-slf4j_2.10-2.3.11.jar
-antlr-2.7.7.jar
-antlr-runtime-3.4.jar
+antlr-runtime-3.5.2.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
 arpack_combined_all-0.1.jar
@@ -170,7 +169,6 @@ spire_2.10-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
 tachyon-client-0.8.2.jar
 tachyon-underfs-hdfs-0.8.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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 f56e6f4..faffb8b 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -5,8 +5,7 @@ activation-1.1.1.jar
 akka-actor_2.10-2.3.11.jar
 akka-remote_2.10-2.3.11.jar
 akka-slf4j_2.10-2.3.11.jar
-antlr-2.7.7.jar
-antlr-runtime-3.4.jar
+antlr-runtime-3.5.2.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
 arpack_combined_all-0.1.jar
@@ -171,7 +170,6 @@ spire_2.10-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
 tachyon-client-0.8.2.jar
 tachyon-underfs-hdfs-0.8.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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 e374844..e703c7a 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -5,8 +5,7 @@ activation-1.1.1.jar
 akka-actor_2.10-2.3.11.jar
 akka-remote_2.10-2.3.11.jar
 akka-slf4j_2.10-2.3.11.jar
-antlr-2.7.7.jar
-antlr-runtime-3.4.jar
+antlr-runtime-3.5.2.jar
 aopalliance-1.0.jar
 apache-log4j-extras-1.2.17.jar
 apacheds-i18n-2.0.0-M15.jar
@@ -177,7 +176,6 @@ spire_2.10-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
 tachyon-client-0.8.2.jar
 tachyon-underfs-hdfs-0.8.2.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d0ac1eb..e414a8b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -183,6 +183,7 @@
     <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>
 
     <test.java.home>${java.home}</test.java.home>
     <test.exclude.tags></test.exclude.tags>
@@ -1843,6 +1844,11 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr-runtime</artifactId>
+        <version>${antlr.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index af1d36c..5d4f19a 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -247,6 +247,9 @@ object SparkBuild extends PomBuild {
   /* Enable unidoc only for the root spark project */
   enable(Unidoc.settings)(spark)
 
+  /* Catalyst ANTLR generation settings */
+  enable(Catalyst.settings)(catalyst)
+
   /* Spark SQL Core console settings */
   enable(SQL.settings)(sql)
 
@@ -357,6 +360,58 @@ object OldDeps {
   )
 }
 
+object Catalyst {
+  lazy val settings = Seq(
+    // 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
+
+      // 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
+      if (antlr.getNumErrors > 0) {
+        log.error("ANTLR: Caught %d build errors.".format(antlr.getNumErrors))
+      }
+
+      // 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
+  )
+}
+
 object SQL {
   lazy val settings = Seq(
     initialCommands in console :=
@@ -414,54 +469,7 @@ object Hive {
     // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
     // in order to generate golden files.  This is only required for developers who are adding new
     // new query tests.
-    fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") },
-    // 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
-
-      // 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
-      if (antlr.getNumErrors > 0) {
-        log.error("ANTLR: Caught %d build errors.".format(antlr.getNumErrors))
-      }
-
-      // 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
+    fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") }
   )
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/catalyst/pom.xml
----------------------------------------------------------------------
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index cfa520b..76ca3f3 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -71,6 +71,10 @@
       <groupId>org.codehaus.janino</groupId>
       <artifactId>janino</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr-runtime</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
@@ -103,6 +107,24 @@
           </execution>
         </executions>
       </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>
     </plugins>
   </build>
   <profiles>

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
new file mode 100644
index 0000000..ba6cfc6
--- /dev/null
+++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/FromClauseParser.g
@@ -0,0 +1,332 @@
+/**
+   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 ( KW_ON! expression {$joinToken.start.getType() != COMMA}? )? )*
+    | uniqueJoinToken^ uniqueJoinSource (COMMA! uniqueJoinSource)+
+    ;
+
+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
+    | 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_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
+    | (LPAREN) => LPAREN joinSource RPAREN -> joinSource
+    | fromSource0
+    ;
+
+
+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); }
+    :
+    db=identifier DOT tab=identifier
+    -> ^(TOK_TABNAME $db $tab)
+    |
+    tab=identifier
+    -> ^(TOK_TABNAME $tab)
+    ;
+
+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/ea489f14/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
new file mode 100644
index 0000000..86c6bd6
--- /dev/null
+++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
@@ -0,0 +1,699 @@
+/**
+   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)
+    )
+    ;
+
+// 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
+    | DecimalLiteral
+    | charSetStringLiteral
+    | booleanValue
+    ;
+
+stringLiteralSequence
+    :
+    StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+)
+    ;
+
+charSetStringLiteral
+@init { gParent.pushMsg("character string literal", state); }
+@after { gParent.popMsg(state); }
+    :
+    csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral)
+    ;
+
+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 StringLiteral qualifiers=intervalQualifiers ->
+    {
+      adaptor.create($qualifiers.tree.token.getType(), $StringLiteral.text)
+    }
+    ;
+
+intervalQualifiers
+    :
+    KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL
+    | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL
+    | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL
+    | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL
+    | KW_DAY -> TOK_INTERVAL_DAY_LITERAL
+    | KW_HOUR -> TOK_INTERVAL_HOUR_LITERAL
+    | KW_MINUTE -> TOK_INTERVAL_MINUTE_LITERAL
+    | KW_SECOND -> TOK_INTERVAL_SECOND_LITERAL
+    ;
+
+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! 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^)* precedenceFieldExpression
+    ;
+
+precedenceUnarySuffixExpression
+    : 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)*
+    ;
+
+
+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
+    ;
+
+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); }
+    : db=identifier DOT fn=identifier
+    -> Identifier[$db.text + "." + $fn.text]
+    |
+    identifier
+    ;
+
+principalIdentifier
+@init { gParent.pushMsg("identifier for principal spec", state); }
+@after { gParent.popMsg(state); }
+    : identifier
+    | QuotedIdentifier
+    ;
+
+//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
+;
+
+//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/ea489f14/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
new file mode 100644
index 0000000..2d2bafb
--- /dev/null
+++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
@@ -0,0 +1,228 @@
+/**
+   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)
+    |
+    ( 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/ea489f14/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
new file mode 100644
index 0000000..e01e710
--- /dev/null
+++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlLexer.g
@@ -0,0 +1,485 @@
+/**
+   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_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';
+KW_MONTH: 'MONTH';
+KW_DAY: 'DAY';
+KW_HOUR: 'HOUR';
+KW_MINUTE: 'MINUTE';
+KW_SECOND: 'SECOND';
+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';
+
+// 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
+    :
+    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
+    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
+    )+
+    ;
+
+CharSetLiteral
+    :
+    StringLiteral
+    | '0' 'X' (HexDigit|Digit)+
+    ;
+
+BigintLiteral
+    :
+    (Digit)+ 'L'
+    ;
+
+SmallintLiteral
+    :
+    (Digit)+ 'S'
+    ;
+
+TinyintLiteral
+    :
+    (Digit)+ 'Y'
+    ;
+
+DecimalLiteral
+    :
+    Number 'B' 'D'
+    ;
+
+ByteLengthLiteral
+    :
+    (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
+    ;
+
+Number
+    :
+    (Digit)+ ( DOT (Digit)* (Exponent)? | 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) (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().substring(1, getText().length() -1 ).replaceAll("``", "`")); }
+    ;
+
+CharSetName
+    :
+    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
+    ;
+
+WS  :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}
+    ;
+
+COMMENT
+  : '--' (~('\n'|'\r'))*
+    { $channel=HIDDEN; }
+  ;
+


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
deleted file mode 100644
index 9f1e168..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
+++ /dev/null
@@ -1,697 +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.
-*/
-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.errors.add(new ParseError(gParent, e, tokenNames));
-  }
-  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)
-    )
-    ;
-
-// 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
-    | DecimalLiteral
-    | charSetStringLiteral
-    | booleanValue
-    ;
-
-stringLiteralSequence
-    :
-    StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+)
-    ;
-
-charSetStringLiteral
-@init { gParent.pushMsg("character string literal", state); }
-@after { gParent.popMsg(state); }
-    :
-    csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral)
-    ;
-
-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 StringLiteral qualifiers=intervalQualifiers ->
-    {
-      adaptor.create($qualifiers.tree.token.getType(), $StringLiteral.text)
-    }
-    ;
-
-intervalQualifiers
-    :
-    KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL
-    | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL
-    | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL
-    | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL
-    | KW_DAY -> TOK_INTERVAL_DAY_LITERAL
-    | KW_HOUR -> TOK_INTERVAL_HOUR_LITERAL
-    | KW_MINUTE -> TOK_INTERVAL_MINUTE_LITERAL
-    | KW_SECOND -> TOK_INTERVAL_SECOND_LITERAL
-    ;
-
-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! 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^)* precedenceFieldExpression
-    ;
-
-precedenceUnarySuffixExpression
-    : 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)*
-    ;
-
-
-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
-    ;
-
-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); }
-    : db=identifier DOT fn=identifier
-    -> Identifier[$db.text + "." + $fn.text]
-    |
-    identifier
-    ;
-
-principalIdentifier
-@init { gParent.pushMsg("identifier for principal spec", state); }
-@after { gParent.popMsg(state); }
-    : identifier
-    | QuotedIdentifier
-    ;
-
-//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
-;
-
-//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/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g
deleted file mode 100644
index 48bc8b0..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g
+++ /dev/null
@@ -1,226 +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.
-*/
-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.errors.add(new ParseError(gParent, e, tokenNames));
-  }
-  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)
-    |
-    ( 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/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g
deleted file mode 100644
index ee1b898..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g
+++ /dev/null
@@ -1,474 +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.
-*/
-lexer grammar SparkSqlLexer;
-
-@lexer::header {
-package org.apache.spark.sql.parser;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-}
-
-@lexer::members {
-  private Configuration hiveConf;
-  
-  public void setHiveConf(Configuration hiveConf) {
-    this.hiveConf = hiveConf;
-  }
-  
-  protected boolean allowQuotedId() {
-    String supportedQIds = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT);
-    return !"none".equals(supportedQIds);
-  }
-}
-
-// 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_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';
-KW_MONTH: 'MONTH';
-KW_DAY: 'DAY';
-KW_HOUR: 'HOUR';
-KW_MINUTE: 'MINUTE';
-KW_SECOND: 'SECOND';
-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';
-
-// 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
-    :
-    ( '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
-    | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"'
-    )+
-    ;
-
-CharSetLiteral
-    :
-    StringLiteral
-    | '0' 'X' (HexDigit|Digit)+
-    ;
-
-BigintLiteral
-    :
-    (Digit)+ 'L'
-    ;
-
-SmallintLiteral
-    :
-    (Digit)+ 'S'
-    ;
-
-TinyintLiteral
-    :
-    (Digit)+ 'Y'
-    ;
-
-DecimalLiteral
-    :
-    Number 'B' 'D'
-    ;
-
-ByteLengthLiteral
-    :
-    (Digit)+ ('b' | 'B' | 'k' | 'K' | 'm' | 'M' | 'g' | 'G')
-    ;
-
-Number
-    :
-    (Digit)+ ( DOT (Digit)* (Exponent)? | 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) (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().substring(1, getText().length() -1 ).replaceAll("``", "`")); }
-    ;
-
-CharSetName
-    :
-    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
-    ;
-
-WS  :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}
-    ;
-
-COMMENT
-  : '--' (~('\n'|'\r'))*
-    { $channel=HIDDEN; }
-  ;
-


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java
deleted file mode 100644
index c77198b..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java
+++ /dev/null
@@ -1,213 +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.parser;
-
-import java.util.ArrayList;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.NoViableAltException;
-import org.antlr.runtime.RecognitionException;
-import org.antlr.runtime.Token;
-import org.antlr.runtime.TokenRewriteStream;
-import org.antlr.runtime.TokenStream;
-import org.antlr.runtime.tree.CommonTree;
-import org.antlr.runtime.tree.CommonTreeAdaptor;
-import org.antlr.runtime.tree.TreeAdaptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.Context;
-
-/**
- * ParseDriver.
- *
- */
-public class ParseDriver {
-
-  private static final Logger LOG = LoggerFactory.getLogger("hive.ql.parse.ParseDriver");
-
-  /**
-   * ANTLRNoCaseStringStream.
-   *
-   */
-  //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.
-  public class ANTLRNoCaseStringStream extends ANTLRStringStream {
-
-    public ANTLRNoCaseStringStream(String input) {
-      super(input);
-    }
-
-    @Override
-    public int LA(int i) {
-
-      int returnChar = super.LA(i);
-      if (returnChar == CharStream.EOF) {
-        return returnChar;
-      } else if (returnChar == 0) {
-        return returnChar;
-      }
-
-      return Character.toUpperCase((char) returnChar);
-    }
-  }
-
-  /**
-   * HiveLexerX.
-   *
-   */
-  public class HiveLexerX extends SparkSqlLexer {
-
-    private final ArrayList<ParseError> errors;
-
-    public HiveLexerX(CharStream input) {
-      super(input);
-      errors = new ArrayList<ParseError>();
-    }
-
-    @Override
-    public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-      errors.add(new ParseError(this, e, tokenNames));
-    }
-
-    @Override
-    public String getErrorMessage(RecognitionException e, String[] tokenNames) {
-      String msg = null;
-
-      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 = "character " + getCharErrorDisplay(e.c) + " not supported here";
-      } else {
-        msg = super.getErrorMessage(e, tokenNames);
-      }
-
-      return msg;
-    }
-
-    public ArrayList<ParseError> getErrors() {
-      return errors;
-    }
-
-  }
-
-  /**
-   * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
-   * so that the graph walking algorithms and the rules framework defined in
-   * ql.lib can be used with the AST Nodes.
-   */
-  public static final TreeAdaptor adaptor = new CommonTreeAdaptor() {
-    /**
-     * Creates an ASTNode for the given token. The ASTNode is a wrapper around
-     * antlr's CommonTree class that implements the Node interface.
-     *
-     * @param payload
-     *          The token.
-     * @return Object (which is actually an ASTNode) for the token.
-     */
-    @Override
-    public Object create(Token payload) {
-      return new ASTNode(payload);
-    }
-
-    @Override
-    public Object dupNode(Object t) {
-
-      return create(((CommonTree)t).token);
-    };
-
-    @Override
-    public Object errorNode(TokenStream input, Token start, Token stop, RecognitionException e) {
-      return new ASTErrorNode(input, start, stop, e);
-    };
-  };
-
-  public ASTNode parse(String command) throws ParseException {
-    return parse(command, null);
-  }
-  
-  public ASTNode parse(String command, Context ctx) 
-      throws ParseException {
-    return parse(command, ctx, true);
-  }
-
-  /**
-   * Parses a command, optionally assigning the parser's token stream to the
-   * given context.
-   *
-   * @param command
-   *          command to parse
-   *
-   * @param ctx
-   *          context with which to associate this parser's token stream, or
-   *          null if either no context is available or the context already has
-   *          an existing stream
-   *
-   * @return parsed AST
-   */
-  public ASTNode parse(String command, Context ctx, boolean setTokenRewriteStream) 
-      throws ParseException {
-    LOG.info("Parsing command: " + command);
-
-    HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
-    TokenRewriteStream tokens = new TokenRewriteStream(lexer);
-    if (ctx != null) {
-      if ( setTokenRewriteStream) {
-        ctx.setTokenRewriteStream(tokens);
-      }
-      lexer.setHiveConf(ctx.getConf());
-    }
-    SparkSqlParser parser = new SparkSqlParser(tokens);
-    if (ctx != null) {
-      parser.setHiveConf(ctx.getConf());
-    }
-    parser.setTreeAdaptor(adaptor);
-    SparkSqlParser.statement_return r = null;
-    try {
-      r = parser.statement();
-    } catch (RecognitionException e) {
-      e.printStackTrace();
-      throw new ParseException(parser.errors);
-    }
-
-    if (lexer.getErrors().size() == 0 && parser.errors.size() == 0) {
-      LOG.info("Parse Completed");
-    } else if (lexer.getErrors().size() != 0) {
-      throw new ParseException(lexer.getErrors());
-    } else {
-      throw new ParseException(parser.errors);
-    }
-
-    ASTNode tree = (ASTNode) r.getTree();
-    tree.setUnknownTokenBoundaries();
-    return tree;
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java
deleted file mode 100644
index b47bcfb..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java
+++ /dev/null
@@ -1,54 +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.parser;
-
-import org.antlr.runtime.BaseRecognizer;
-import org.antlr.runtime.RecognitionException;
-
-/**
- *
- */
-public class ParseError {
-  private final BaseRecognizer br;
-  private final RecognitionException re;
-  private final String[] tokenNames;
-
-  ParseError(BaseRecognizer br, RecognitionException re, String[] tokenNames) {
-    this.br = br;
-    this.re = re;
-    this.tokenNames = tokenNames;
-  }
-
-  BaseRecognizer getBaseRecognizer() {
-    return br;
-  }
-
-  RecognitionException getRecognitionException() {
-    return re;
-  }
-
-  String[] getTokenNames() {
-    return tokenNames;
-  }
-
-  String getMessage() {
-    return br.getErrorHeader(re) + " " + br.getErrorMessage(re, tokenNames);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java
deleted file mode 100644
index fff891c..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java
+++ /dev/null
@@ -1,51 +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.parser;
-
-import java.util.ArrayList;
-
-/**
- * ParseException.
- *
- */
-public class ParseException extends Exception {
-
-  private static final long serialVersionUID = 1L;
-  ArrayList<ParseError> errors;
-
-  public ParseException(ArrayList<ParseError> errors) {
-    super();
-    this.errors = errors;
-  }
-
-  @Override
-  public String getMessage() {
-
-    StringBuilder sb = new StringBuilder();
-    for (ParseError err : errors) {
-      if (sb.length() > 0) {
-        sb.append('\n');
-      }
-      sb.append(err.getMessage());
-    }
-
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java
deleted file mode 100644
index a5c2998..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java
+++ /dev/null
@@ -1,96 +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.parser;
-
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-
-
-/**
- * Library of utility functions used in the parse code.
- *
- */
-public final class ParseUtils {
-  /**
-   * Performs a descent of the leftmost branch of a tree, stopping when either a
-   * node with a non-null token is found or the leaf level is encountered.
-   *
-   * @param tree
-   *          candidate node from which to start searching
-   *
-   * @return node at which descent stopped
-   */
-  public static ASTNode findRootNonNullToken(ASTNode tree) {
-    while ((tree.getToken() == null) && (tree.getChildCount() > 0)) {
-      tree = (org.apache.spark.sql.parser.ASTNode) tree.getChild(0);
-    }
-    return tree;
-  }
-
-  private ParseUtils() {
-    // prevent instantiation
-  }
-
-  public static VarcharTypeInfo getVarcharTypeInfo(ASTNode node)
-      throws SemanticException {
-    if (node.getChildCount() != 1) {
-      throw new SemanticException("Bad params for type varchar");
-    }
-
-    String lengthStr = node.getChild(0).getText();
-    return TypeInfoFactory.getVarcharTypeInfo(Integer.valueOf(lengthStr));
-  }
-
-  public static CharTypeInfo getCharTypeInfo(ASTNode node)
-      throws SemanticException {
-    if (node.getChildCount() != 1) {
-      throw new SemanticException("Bad params for type char");
-    }
-
-    String lengthStr = node.getChild(0).getText();
-    return TypeInfoFactory.getCharTypeInfo(Integer.valueOf(lengthStr));
-  }
-
-  public static DecimalTypeInfo getDecimalTypeTypeInfo(ASTNode node)
-      throws SemanticException {
-    if (node.getChildCount() > 2) {
-        throw new SemanticException("Bad params for type decimal");
-      }
-
-      int precision = HiveDecimal.USER_DEFAULT_PRECISION;
-      int scale = HiveDecimal.USER_DEFAULT_SCALE;
-
-      if (node.getChildCount() >= 1) {
-        String precStr = node.getChild(0).getText();
-        precision = Integer.valueOf(precStr);
-      }
-
-      if (node.getChildCount() == 2) {
-        String scaleStr = node.getChild(1).getText();
-        scale = Integer.valueOf(scaleStr);
-      }
-
-      return TypeInfoFactory.getDecimalTypeInfo(precision, scale);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java b/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java
deleted file mode 100644
index 4b2015e..0000000
--- a/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java
+++ /dev/null
@@ -1,406 +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.parser;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.antlr.runtime.tree.Tree;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.ErrorMsg;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-
-/**
- * SemanticAnalyzer.
- *
- */
-public abstract class SemanticAnalyzer {
-  public static String charSetString(String charSetName, String charSetString)
-      throws SemanticException {
-    try {
-      // The character set name starts with a _, so strip that
-      charSetName = charSetName.substring(1);
-      if (charSetString.charAt(0) == '\'') {
-        return new String(unescapeSQLString(charSetString).getBytes(),
-            charSetName);
-      } else // hex input is also supported
-      {
-        assert charSetString.charAt(0) == '0';
-        assert charSetString.charAt(1) == 'x';
-        charSetString = charSetString.substring(2);
-
-        byte[] bArray = new byte[charSetString.length() / 2];
-        int j = 0;
-        for (int i = 0; i < charSetString.length(); i += 2) {
-          int val = Character.digit(charSetString.charAt(i), 16) * 16
-              + Character.digit(charSetString.charAt(i + 1), 16);
-          if (val > 127) {
-            val = val - 256;
-          }
-          bArray[j++] = (byte)val;
-        }
-
-        String res = new String(bArray, charSetName);
-        return res;
-      }
-    } catch (UnsupportedEncodingException e) {
-      throw new SemanticException(e);
-    }
-  }
-
-  /**
-   * Remove the encapsulating "`" pair from the identifier. We allow users to
-   * use "`" to escape identifier for table names, column names and aliases, in
-   * case that coincide with Hive language keywords.
-   */
-  public static String unescapeIdentifier(String val) {
-    if (val == null) {
-      return null;
-    }
-    if (val.charAt(0) == '`' && val.charAt(val.length() - 1) == '`') {
-      val = val.substring(1, val.length() - 1);
-    }
-    return val;
-  }
-
-  /**
-   * Converts parsed key/value properties pairs into a map.
-   *
-   * @param prop ASTNode parent of the key/value pairs
-   *
-   * @param mapProp property map which receives the mappings
-   */
-  public static void readProps(
-    ASTNode prop, Map<String, String> mapProp) {
-
-    for (int propChild = 0; propChild < prop.getChildCount(); propChild++) {
-      String key = unescapeSQLString(prop.getChild(propChild).getChild(0)
-          .getText());
-      String value = null;
-      if (prop.getChild(propChild).getChild(1) != null) {
-        value = unescapeSQLString(prop.getChild(propChild).getChild(1).getText());
-      }
-      mapProp.put(key, value);
-    }
-  }
-
-  private static final int[] multiplier = new int[] {1000, 100, 10, 1};
-
-  @SuppressWarnings("nls")
-  public static String unescapeSQLString(String b) {
-    Character enclosure = null;
-
-    // Some of the strings can be passed in as unicode. For example, the
-    // delimiter can be passed in as \002 - So, we first check if the
-    // string is a unicode number, else go back to the old behavior
-    StringBuilder sb = new StringBuilder(b.length());
-    for (int i = 0; i < b.length(); i++) {
-
-      char currentChar = b.charAt(i);
-      if (enclosure == null) {
-        if (currentChar == '\'' || b.charAt(i) == '\"') {
-          enclosure = currentChar;
-        }
-        // ignore all other chars outside the enclosure
-        continue;
-      }
-
-      if (enclosure.equals(currentChar)) {
-        enclosure = null;
-        continue;
-      }
-
-      if (currentChar == '\\' && (i + 6 < b.length()) && b.charAt(i + 1) == 'u') {
-        int code = 0;
-        int base = i + 2;
-        for (int j = 0; j < 4; j++) {
-          int digit = Character.digit(b.charAt(j + base), 16);
-          code += digit * multiplier[j];
-        }
-        sb.append((char)code);
-        i += 5;
-        continue;
-      }
-
-      if (currentChar == '\\' && (i + 4 < b.length())) {
-        char i1 = b.charAt(i + 1);
-        char i2 = b.charAt(i + 2);
-        char i3 = b.charAt(i + 3);
-        if ((i1 >= '0' && i1 <= '1') && (i2 >= '0' && i2 <= '7')
-            && (i3 >= '0' && i3 <= '7')) {
-          byte bVal = (byte) ((i3 - '0') + ((i2 - '0') * 8) + ((i1 - '0') * 8 * 8));
-          byte[] bValArr = new byte[1];
-          bValArr[0] = bVal;
-          String tmp = new String(bValArr);
-          sb.append(tmp);
-          i += 3;
-          continue;
-        }
-      }
-
-      if (currentChar == '\\' && (i + 2 < b.length())) {
-        char n = b.charAt(i + 1);
-        switch (n) {
-        case '0':
-          sb.append("\0");
-          break;
-        case '\'':
-          sb.append("'");
-          break;
-        case '"':
-          sb.append("\"");
-          break;
-        case 'b':
-          sb.append("\b");
-          break;
-        case 'n':
-          sb.append("\n");
-          break;
-        case 'r':
-          sb.append("\r");
-          break;
-        case 't':
-          sb.append("\t");
-          break;
-        case 'Z':
-          sb.append("\u001A");
-          break;
-        case '\\':
-          sb.append("\\");
-          break;
-        // The following 2 lines are exactly what MySQL does TODO: why do we do this?
-        case '%':
-          sb.append("\\%");
-          break;
-        case '_':
-          sb.append("\\_");
-          break;
-        default:
-          sb.append(n);
-        }
-        i++;
-      } else {
-        sb.append(currentChar);
-      }
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Get the list of FieldSchema out of the ASTNode.
-   */
-  public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase) throws SemanticException {
-    List<FieldSchema> colList = new ArrayList<FieldSchema>();
-    int numCh = ast.getChildCount();
-    for (int i = 0; i < numCh; i++) {
-      FieldSchema col = new FieldSchema();
-      ASTNode child = (ASTNode) ast.getChild(i);
-      Tree grandChild = child.getChild(0);
-      if(grandChild != null) {
-        String name = grandChild.getText();
-        if(lowerCase) {
-          name = name.toLowerCase();
-        }
-        // child 0 is the name of the column
-        col.setName(unescapeIdentifier(name));
-        // child 1 is the type of the column
-        ASTNode typeChild = (ASTNode) (child.getChild(1));
-        col.setType(getTypeStringFromAST(typeChild));
-
-        // child 2 is the optional comment of the column
-        if (child.getChildCount() == 3) {
-          col.setComment(unescapeSQLString(child.getChild(2).getText()));
-        }
-      }
-      colList.add(col);
-    }
-    return colList;
-  }
-
-  protected static String getTypeStringFromAST(ASTNode typeNode)
-      throws SemanticException {
-    switch (typeNode.getType()) {
-    case SparkSqlParser.TOK_LIST:
-      return serdeConstants.LIST_TYPE_NAME + "<"
-          + getTypeStringFromAST((ASTNode) typeNode.getChild(0)) + ">";
-    case SparkSqlParser.TOK_MAP:
-      return serdeConstants.MAP_TYPE_NAME + "<"
-          + getTypeStringFromAST((ASTNode) typeNode.getChild(0)) + ","
-          + getTypeStringFromAST((ASTNode) typeNode.getChild(1)) + ">";
-    case SparkSqlParser.TOK_STRUCT:
-      return getStructTypeStringFromAST(typeNode);
-    case SparkSqlParser.TOK_UNIONTYPE:
-      return getUnionTypeStringFromAST(typeNode);
-    default:
-      return getTypeName(typeNode);
-    }
-  }
-
-  private static String getStructTypeStringFromAST(ASTNode typeNode)
-      throws SemanticException {
-    String typeStr = serdeConstants.STRUCT_TYPE_NAME + "<";
-    typeNode = (ASTNode) typeNode.getChild(0);
-    int children = typeNode.getChildCount();
-    if (children <= 0) {
-      throw new SemanticException("empty struct not allowed.");
-    }
-    StringBuilder buffer = new StringBuilder(typeStr);
-    for (int i = 0; i < children; i++) {
-      ASTNode child = (ASTNode) typeNode.getChild(i);
-      buffer.append(unescapeIdentifier(child.getChild(0).getText())).append(":");
-      buffer.append(getTypeStringFromAST((ASTNode) child.getChild(1)));
-      if (i < children - 1) {
-        buffer.append(",");
-      }
-    }
-
-    buffer.append(">");
-    return buffer.toString();
-  }
-
-  private static String getUnionTypeStringFromAST(ASTNode typeNode)
-      throws SemanticException {
-    String typeStr = serdeConstants.UNION_TYPE_NAME + "<";
-    typeNode = (ASTNode) typeNode.getChild(0);
-    int children = typeNode.getChildCount();
-    if (children <= 0) {
-      throw new SemanticException("empty union not allowed.");
-    }
-    StringBuilder buffer = new StringBuilder(typeStr);
-    for (int i = 0; i < children; i++) {
-      buffer.append(getTypeStringFromAST((ASTNode) typeNode.getChild(i)));
-      if (i < children - 1) {
-        buffer.append(",");
-      }
-    }
-    buffer.append(">");
-    typeStr = buffer.toString();
-    return typeStr;
-  }
-
-  public static String getAstNodeText(ASTNode tree) {
-    return tree.getChildCount() == 0?tree.getText() :
-        getAstNodeText((ASTNode)tree.getChild(tree.getChildCount() - 1));
-  }
-
-  public static String generateErrorMessage(ASTNode ast, String message) {
-    StringBuilder sb = new StringBuilder();
-    if (ast == null) {
-      sb.append(message).append(". Cannot tell the position of null AST.");
-      return sb.toString();
-    }
-    sb.append(ast.getLine());
-    sb.append(":");
-    sb.append(ast.getCharPositionInLine());
-    sb.append(" ");
-    sb.append(message);
-    sb.append(". Error encountered near token '");
-    sb.append(getAstNodeText(ast));
-    sb.append("'");
-    return sb.toString();
-  }
-
-  private static final Map<Integer, String> TokenToTypeName = new HashMap<Integer, String>();
-
-  static {
-    TokenToTypeName.put(SparkSqlParser.TOK_BOOLEAN, serdeConstants.BOOLEAN_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_TINYINT, serdeConstants.TINYINT_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_SMALLINT, serdeConstants.SMALLINT_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_INT, serdeConstants.INT_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_BIGINT, serdeConstants.BIGINT_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_FLOAT, serdeConstants.FLOAT_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_DOUBLE, serdeConstants.DOUBLE_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_STRING, serdeConstants.STRING_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_CHAR, serdeConstants.CHAR_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_VARCHAR, serdeConstants.VARCHAR_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_BINARY, serdeConstants.BINARY_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_DATE, serdeConstants.DATE_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_DATETIME, serdeConstants.DATETIME_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_INTERVAL_YEAR_MONTH, serdeConstants.INTERVAL_YEAR_MONTH_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_INTERVAL_DAY_TIME, serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME);
-    TokenToTypeName.put(SparkSqlParser.TOK_DECIMAL, serdeConstants.DECIMAL_TYPE_NAME);
-  }
-
-  public static String getTypeName(ASTNode node) throws SemanticException {
-    int token = node.getType();
-    String typeName;
-
-    // datetime type isn't currently supported
-    if (token == SparkSqlParser.TOK_DATETIME) {
-      throw new SemanticException(ErrorMsg.UNSUPPORTED_TYPE.getMsg());
-    }
-
-    switch (token) {
-      case SparkSqlParser.TOK_CHAR:
-        CharTypeInfo charTypeInfo = ParseUtils.getCharTypeInfo(node);
-        typeName = charTypeInfo.getQualifiedName();
-        break;
-      case SparkSqlParser.TOK_VARCHAR:
-        VarcharTypeInfo varcharTypeInfo = ParseUtils.getVarcharTypeInfo(node);
-        typeName = varcharTypeInfo.getQualifiedName();
-        break;
-      case SparkSqlParser.TOK_DECIMAL:
-        DecimalTypeInfo decTypeInfo = ParseUtils.getDecimalTypeTypeInfo(node);
-        typeName = decTypeInfo.getQualifiedName();
-        break;
-      default:
-        typeName = TokenToTypeName.get(token);
-    }
-    return typeName;
-  }
-
-  public static String relativeToAbsolutePath(HiveConf conf, String location) throws SemanticException {
-    boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE);
-    if (testMode) {
-      URI uri = new Path(location).toUri();
-      String scheme = uri.getScheme();
-      String authority = uri.getAuthority();
-      String path = uri.getPath();
-      if (!path.startsWith("/")) {
-        path = (new Path(System.getProperty("test.tmp.dir"),
-            path)).toUri().getPath();
-      }
-      if (StringUtils.isEmpty(scheme)) {
-        scheme = "pfile";
-      }
-      try {
-        uri = new URI(scheme, authority, path, null, null);
-      } catch (URISyntaxException e) {
-        throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(), e);
-      }
-      return uri.toString();
-    } else {
-      //no-op for non-test mode for now
-      return location;
-    }
-  }
-}


---------------------------------------------------------------------
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-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst

Posted by rx...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/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
new file mode 100644
index 0000000..98b4679
--- /dev/null
+++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g
@@ -0,0 +1,2462 @@
+/**
+   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;
+
+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_JOIN;
+TOK_LEFTOUTERJOIN;
+TOK_RIGHTOUTERJOIN;
+TOK_FULLOUTERJOIN;
+TOK_UNIQUEJOIN;
+TOK_CROSSJOIN;
+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_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_DAY_LITERAL;
+TOK_INTERVAL_HOUR_LITERAL;
+TOK_INTERVAL_MINUTE_LITERAL;
+TOK_INTERVAL_SECOND_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_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_CHARSETLITERAL;
+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;
+}
+
+
+// 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");
+
+
+    // 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) {
+    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
+	;
+
+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
+    ;
+
+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
+    | 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?
+       | (LPAREN columnNameTypeList RPAREN)?
+         tableComment?
+         tablePartition?
+         tableBuckets?
+         tableSkewed?
+         tableRowFormat?
+         tableFileFormat?
+         tableLocation?
+         tablePropertiesPrefixed?
+         (KW_AS 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?)
+    ;
+
+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
+    ;
+
+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_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=primitiveType 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)
+    ;
+
+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]
+   :
+   (
+   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 b=simpleSelectStatement
+   -> {$setOpSelectStatement.tree != null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}?
+      ^(TOK_QUERY
+          ^(TOK_FROM
+            ^(TOK_SUBQUERY
+              ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b)
+              {adaptor.create(Identifier, generateUnionAlias())}
+             )
+          )
+          ^(TOK_INSERT
+             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+             ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
+          )
+       )
+   -> {$setOpSelectStatement.tree != null && $u.tree.getType()!=SparkSqlParser.TOK_UNIONDISTINCT}?
+      ^(TOK_UNIONALL {$setOpSelectStatement.tree} $b)
+   -> {$setOpSelectStatement.tree == null && $u.tree.getType()==SparkSqlParser.TOK_UNIONDISTINCT}?
+      ^(TOK_QUERY
+          ^(TOK_FROM
+            ^(TOK_SUBQUERY
+              ^(TOK_UNIONALL {$t} $b)
+              {adaptor.create(Identifier, generateUnionAlias())}
+             )
+           )
+          ^(TOK_INSERT
+            ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+            ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
+         )
+       )
+   -> ^(TOK_UNIONALL {$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
+*/

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java
new file mode 100644
index 0000000..5bc87b6
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java
@@ -0,0 +1,162 @@
+/**
+ * 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.io.UnsupportedEncodingException;
+
+/**
+ * A couple of utility methods that help with parsing ASTs.
+ *
+ * Both methods in this class were take from the SemanticAnalyzer in Hive:
+ * ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+ */
+public final class ParseUtils {
+  private ParseUtils() {
+    super();
+  }
+
+  public static String charSetString(String charSetName, String charSetString)
+          throws UnsupportedEncodingException {
+    // The character set name starts with a _, so strip that
+    charSetName = charSetName.substring(1);
+    if (charSetString.charAt(0) == '\'') {
+      return new String(unescapeSQLString(charSetString).getBytes(), charSetName);
+    } else // hex input is also supported
+    {
+      assert charSetString.charAt(0) == '0';
+      assert charSetString.charAt(1) == 'x';
+      charSetString = charSetString.substring(2);
+
+      byte[] bArray = new byte[charSetString.length() / 2];
+      int j = 0;
+      for (int i = 0; i < charSetString.length(); i += 2) {
+        int val = Character.digit(charSetString.charAt(i), 16) * 16
+                + Character.digit(charSetString.charAt(i + 1), 16);
+        if (val > 127) {
+          val = val - 256;
+        }
+        bArray[j++] = (byte)val;
+      }
+
+      return new String(bArray, charSetName);
+    }
+  }
+
+  private static final int[] multiplier = new int[] {1000, 100, 10, 1};
+
+  @SuppressWarnings("nls")
+  public static String unescapeSQLString(String b) {
+    Character enclosure = null;
+
+    // Some of the strings can be passed in as unicode. For example, the
+    // delimiter can be passed in as \002 - So, we first check if the
+    // string is a unicode number, else go back to the old behavior
+    StringBuilder sb = new StringBuilder(b.length());
+    for (int i = 0; i < b.length(); i++) {
+
+      char currentChar = b.charAt(i);
+      if (enclosure == null) {
+        if (currentChar == '\'' || b.charAt(i) == '\"') {
+          enclosure = currentChar;
+        }
+        // ignore all other chars outside the enclosure
+        continue;
+      }
+
+      if (enclosure.equals(currentChar)) {
+        enclosure = null;
+        continue;
+      }
+
+      if (currentChar == '\\' && (i + 6 < b.length()) && b.charAt(i + 1) == 'u') {
+        int code = 0;
+        int base = i + 2;
+        for (int j = 0; j < 4; j++) {
+          int digit = Character.digit(b.charAt(j + base), 16);
+          code += digit * multiplier[j];
+        }
+        sb.append((char)code);
+        i += 5;
+        continue;
+      }
+
+      if (currentChar == '\\' && (i + 4 < b.length())) {
+        char i1 = b.charAt(i + 1);
+        char i2 = b.charAt(i + 2);
+        char i3 = b.charAt(i + 3);
+        if ((i1 >= '0' && i1 <= '1') && (i2 >= '0' && i2 <= '7')
+            && (i3 >= '0' && i3 <= '7')) {
+          byte bVal = (byte) ((i3 - '0') + ((i2 - '0') * 8) + ((i1 - '0') * 8 * 8));
+          byte[] bValArr = new byte[1];
+          bValArr[0] = bVal;
+          String tmp = new String(bValArr);
+          sb.append(tmp);
+          i += 3;
+          continue;
+        }
+      }
+
+      if (currentChar == '\\' && (i + 2 < b.length())) {
+        char n = b.charAt(i + 1);
+        switch (n) {
+        case '0':
+          sb.append("\0");
+          break;
+        case '\'':
+          sb.append("'");
+          break;
+        case '"':
+          sb.append("\"");
+          break;
+        case 'b':
+          sb.append("\b");
+          break;
+        case 'n':
+          sb.append("\n");
+          break;
+        case 'r':
+          sb.append("\r");
+          break;
+        case 't':
+          sb.append("\t");
+          break;
+        case 'Z':
+          sb.append("\u001A");
+          break;
+        case '\\':
+          sb.append("\\");
+          break;
+        // The following 2 lines are exactly what MySQL does TODO: why do we do this?
+        case '%':
+          sb.append("\\%");
+          break;
+        case '_':
+          sb.append("\\_");
+          break;
+        default:
+          sb.append(n);
+        }
+        i++;
+      } else {
+        sb.append(currentChar);
+      }
+    }
+    return sb.toString();
+  }
+}


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