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

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

[SPARK-14211][SQL] Remove ANTLR3 based parser

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

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

### How was this patch tested?

Existing unit tests.

cc rxin andrewor14 yhuai

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

Closes #12071 from hvanhovell/SPARK-14211.


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
deleted file mode 100644
index 916eb6a..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/IdentifiersParser.g
+++ /dev/null
@@ -1,184 +0,0 @@
-/**
-   Licensed to the Apache Software Foundation (ASF) under one or more 
-   contributor license agreements.  See the NOTICE file distributed with 
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with 
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/IdentifiersParser.g grammar.
-*/
-parser grammar IdentifiersParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//-----------------------------------------------------------------------------------
-
-// group by a,b
-groupByClause
-@init { gParent.pushMsg("group by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_GROUP KW_BY
-    expression
-    ( COMMA expression)*
-    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
-    (sets=KW_GROUPING KW_SETS 
-    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
-    -> {rollup != null}? ^(TOK_ROLLUP_GROUPBY expression+)
-    -> {cube != null}? ^(TOK_CUBE_GROUPBY expression+)
-    -> {sets != null}? ^(TOK_GROUPING_SETS expression+ groupingSetExpression+)
-    -> ^(TOK_GROUPBY expression+)
-    ;
-
-groupingSetExpression
-@init {gParent.pushMsg("grouping set expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   (LPAREN) => groupingSetExpressionMultiple 
-   |
-   groupingExpressionSingle
-   ;
-
-groupingSetExpressionMultiple
-@init {gParent.pushMsg("grouping set part expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   LPAREN 
-   expression? (COMMA expression)*
-   RPAREN
-   -> ^(TOK_GROUPING_SETS_EXPRESSION expression*)
-   ;
-
-groupingExpressionSingle
-@init { gParent.pushMsg("groupingExpression expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression -> ^(TOK_GROUPING_SETS_EXPRESSION expression)
-    ;
-
-havingClause
-@init { gParent.pushMsg("having clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_HAVING havingCondition -> ^(TOK_HAVING havingCondition)
-    ;
-
-havingCondition
-@init { gParent.pushMsg("having condition", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression
-    ;
-
-expressionsInParenthese
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> expression+
-    ;
-
-expressionsNotInParenthese
-    :
-    expression (COMMA expression)* -> expression+
-    ;
-
-columnRefOrderInParenthese
-    :
-    LPAREN columnRefOrder (COMMA columnRefOrder)* RPAREN -> columnRefOrder+
-    ;
-
-columnRefOrderNotInParenthese
-    :
-    columnRefOrder (COMMA columnRefOrder)* -> columnRefOrder+
-    ;
-    
-// order by a,b
-orderByClause
-@init { gParent.pushMsg("order by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_ORDER KW_BY columnRefOrder ( COMMA columnRefOrder)* -> ^(TOK_ORDERBY columnRefOrder+)
-    ;
-    
-clusterByClause
-@init { gParent.pushMsg("cluster by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CLUSTER KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_CLUSTERBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_CLUSTERBY expressionsNotInParenthese)
-    )
-    ;
-
-partitionByClause
-@init  { gParent.pushMsg("partition by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_PARTITION KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-distributeByClause
-@init { gParent.pushMsg("distribute by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_DISTRIBUTE KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-sortByClause
-@init { gParent.pushMsg("sort by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_SORT KW_BY
-    (
-    (LPAREN) => columnRefOrderInParenthese -> ^(TOK_SORTBY columnRefOrderInParenthese)
-    |
-    columnRefOrderNotInParenthese -> ^(TOK_SORTBY columnRefOrderNotInParenthese)
-    )
-    ;

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

http://git-wip-us.apache.org/repos/asf/spark/blob/a9b93e07/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
deleted file mode 100644
index f18b6ec..0000000
--- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g
+++ /dev/null
@@ -1,235 +0,0 @@
-/**
-   Licensed to the Apache Software Foundation (ASF) under one or more 
-   contributor license agreements.  See the NOTICE file distributed with 
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with 
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-
-   This file is an adaptation of Hive's org/apache/hadoop/hive/ql/SelectClauseParser.g grammar.
-*/
-parser grammar SelectClauseParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.displayRecognitionError(tokenNames, e);
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//----------------------- Rules for parsing selectClause -----------------------------
-// select a,b,c ...
-selectClause
-@init { gParent.pushMsg("select clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_SELECT hintClause? (((KW_ALL | dist=KW_DISTINCT)? selectList)
-                          | (transform=KW_TRANSFORM selectTrfmClause))
-     -> {$transform == null && $dist == null}? ^(TOK_SELECT hintClause? selectList)
-     -> {$transform == null && $dist != null}? ^(TOK_SELECTDI hintClause? selectList)
-     -> ^(TOK_SELECT hintClause? ^(TOK_SELEXPR selectTrfmClause) )
-    |
-    trfmClause  ->^(TOK_SELECT ^(TOK_SELEXPR trfmClause))
-    ;
-
-selectList
-@init { gParent.pushMsg("select list", state); }
-@after { gParent.popMsg(state); }
-    :
-    selectItem ( COMMA  selectItem )* -> selectItem+
-    ;
-
-selectTrfmClause
-@init { gParent.pushMsg("transform clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    LPAREN selectExpressionList RPAREN
-    inSerde=rowFormat inRec=recordWriter
-    KW_USING StringLiteral
-    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
-    outSerde=rowFormat outRec=recordReader
-    -> ^(TOK_TRANSFORM selectExpressionList $inSerde $inRec StringLiteral $outSerde $outRec aliasList? columnNameTypeList?)
-    ;
-
-hintClause
-@init { gParent.pushMsg("hint clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    DIVIDE STAR PLUS hintList STAR DIVIDE -> ^(TOK_HINTLIST hintList)
-    ;
-
-hintList
-@init { gParent.pushMsg("hint list", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintItem (COMMA hintItem)* -> hintItem+
-    ;
-
-hintItem
-@init { gParent.pushMsg("hint item", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintName (LPAREN hintArgs RPAREN)? -> ^(TOK_HINT hintName hintArgs?)
-    ;
-
-hintName
-@init { gParent.pushMsg("hint name", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_MAPJOIN -> TOK_MAPJOIN
-    | KW_STREAMTABLE -> TOK_STREAMTABLE
-    ;
-
-hintArgs
-@init { gParent.pushMsg("hint arguments", state); }
-@after { gParent.popMsg(state); }
-    :
-    hintArgName (COMMA hintArgName)* -> ^(TOK_HINTARGLIST hintArgName+)
-    ;
-
-hintArgName
-@init { gParent.pushMsg("hint argument name", state); }
-@after { gParent.popMsg(state); }
-    :
-    identifier
-    ;
-
-selectItem
-@init { gParent.pushMsg("selection target", state); }
-@after { gParent.popMsg(state); }
-    :
-    (tableAllColumns) => tableAllColumns -> ^(TOK_SELEXPR tableAllColumns)
-    |
-    namedExpression
-    ;
-
-namedExpression
-@init { gParent.pushMsg("select named expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    ( expression
-      ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?
-    ) -> ^(TOK_SELEXPR expression identifier*)
-    ;
-
-trfmClause
-@init { gParent.pushMsg("transform clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    (   KW_MAP    selectExpressionList
-      | KW_REDUCE selectExpressionList )
-    inSerde=rowFormat inRec=recordWriter
-    KW_USING StringLiteral
-    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
-    outSerde=rowFormat outRec=recordReader
-    -> ^(TOK_TRANSFORM selectExpressionList $inSerde $inRec StringLiteral $outSerde $outRec aliasList? columnNameTypeList?)
-    ;
-
-selectExpression
-@init { gParent.pushMsg("select expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    (tableAllColumns) => tableAllColumns
-    |
-    expression
-    ;
-
-selectExpressionList
-@init { gParent.pushMsg("select expression list", state); }
-@after { gParent.popMsg(state); }
-    :
-    selectExpression (COMMA selectExpression)* -> ^(TOK_EXPLIST selectExpression+)
-    ;
-
-//---------------------- Rules for windowing clauses -------------------------------
-window_clause 
-@init { gParent.pushMsg("window_clause", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_WINDOW window_defn (COMMA window_defn)* -> ^(KW_WINDOW window_defn+)
-;  
-
-window_defn 
-@init { gParent.pushMsg("window_defn", state); }
-@after { gParent.popMsg(state); } 
-:
-  Identifier KW_AS window_specification -> ^(TOK_WINDOWDEF Identifier window_specification)
-;  
-
-window_specification 
-@init { gParent.pushMsg("window_specification", state); }
-@after { gParent.popMsg(state); } 
-:
-  (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) -> ^(TOK_WINDOWSPEC Identifier? partitioningSpec? window_frame?)
-;
-
-window_frame :
- window_range_expression |
- window_value_expression
-;
-
-window_range_expression 
-@init { gParent.pushMsg("window_range_expression", state); }
-@after { gParent.popMsg(state); } 
-:
- KW_ROWS sb=window_frame_start_boundary -> ^(TOK_WINDOWRANGE $sb) |
- KW_ROWS KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary -> ^(TOK_WINDOWRANGE $s $end)
-;
-
-window_value_expression 
-@init { gParent.pushMsg("window_value_expression", state); }
-@after { gParent.popMsg(state); } 
-:
- KW_RANGE sb=window_frame_start_boundary -> ^(TOK_WINDOWVALUES $sb) |
- KW_RANGE KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary -> ^(TOK_WINDOWVALUES $s $end)
-;
-
-window_frame_start_boundary 
-@init { gParent.pushMsg("windowframestartboundary", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_UNBOUNDED KW_PRECEDING  -> ^(KW_PRECEDING KW_UNBOUNDED) | 
-  KW_CURRENT KW_ROW  -> ^(KW_CURRENT) |
-  Number KW_PRECEDING -> ^(KW_PRECEDING Number)
-;
-
-window_frame_boundary 
-@init { gParent.pushMsg("windowframeboundary", state); }
-@after { gParent.popMsg(state); } 
-:
-  KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING)  -> ^($r KW_UNBOUNDED) | 
-  KW_CURRENT KW_ROW  -> ^(KW_CURRENT) |
-  Number (d=KW_PRECEDING | d=KW_FOLLOWING ) -> ^($d Number)
-;   
-

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


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