aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--pom.xml5
-rw-r--r--project/SparkBuild.scala2
-rw-r--r--project/plugins.sbt4
-rw-r--r--sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala10
-rw-r--r--sql/hive/pom.xml22
-rw-r--r--sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g330
-rw-r--r--sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g697
-rw-r--r--sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g226
-rw-r--r--sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g474
-rw-r--r--sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g2457
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java49
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java245
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java213
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java54
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java51
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java96
-rw-r--r--sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java406
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala133
18 files changed, 5402 insertions, 72 deletions
diff --git a/pom.xml b/pom.xml
index 284c219519..73ba8d555a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1951,6 +1951,11 @@
</compilerArgs>
</configuration>
</plugin>
+ <plugin>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr3-maven-plugin</artifactId>
+ <version>3.5.2</version>
+ </plugin>
<!-- Surefire runs all Java tests -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index c3d53f835f..df21d3eb63 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -415,7 +415,7 @@ object Hive {
// 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") }
- )
+ ) ++ sbtantlr.SbtAntlrPlugin.antlrSettings
}
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 5e23224cf8..f172dc9c1f 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -4,6 +4,8 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release
resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/"
+resolvers += "stefri" at "http://stefri.github.io/repo/releases"
+
addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2")
addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0")
@@ -24,6 +26,8 @@ addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2")
addSbtPlugin("io.spray" % "sbt-revolver" % "0.7.2")
+addSbtPlugin("com.github.stefri" % "sbt-antlr" % "0.5.3")
+
libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3"
libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3"
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 2d0d7b8af3..2b0e48dbfc 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -308,7 +308,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
// The difference between the double numbers generated by Hive and Spark
// can be ignored (e.g., 0.6633880657639323 and 0.6633880657639322)
- "udaf_corr"
+ "udaf_corr",
+
+ // Feature removed in HIVE-11145
+ "alter_partition_protect_mode",
+ "drop_partitions_ignore_protection",
+ "protectmode"
)
/**
@@ -328,7 +333,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"alter_index",
"alter_merge_2",
"alter_partition_format_loc",
- "alter_partition_protect_mode",
"alter_partition_with_whitelist",
"alter_rename_partition",
"alter_table_serde",
@@ -460,7 +464,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"drop_partitions_filter",
"drop_partitions_filter2",
"drop_partitions_filter3",
- "drop_partitions_ignore_protection",
"drop_table",
"drop_table2",
"drop_table_removes_partition_dirs",
@@ -778,7 +781,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"ppr_pushdown2",
"ppr_pushdown3",
"progress_1",
- "protectmode",
"push_or",
"query_with_semi",
"quote1",
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index e9885f6682..ffabb92179 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -232,6 +232,7 @@
<configuration>
<sources>
<source>v${hive.version.short}/src/main/scala</source>
+ <source>${project.build.directory/generated-sources/antlr</source>
</sources>
</configuration>
</execution>
@@ -260,6 +261,27 @@
</execution>
</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>
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
new file mode 100644
index 0000000000..e4a80f0ce8
--- /dev/null
+++ b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/FromClauseParser.g
@@ -0,0 +1,330 @@
+/**
+ 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+))
+ ;
+
+//-----------------------------------------------------------------------------------
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
new file mode 100644
index 0000000000..5c3d7ef866
--- /dev/null
+++ b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
@@ -0,0 +1,697 @@
+/**
+ 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+)?)
+ ;
+
+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
+ ;
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
new file mode 100644
index 0000000000..48bc8b0a30
--- /dev/null
+++ b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SelectClauseParser.g
@@ -0,0 +1,226 @@
+/**
+ 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)
+;
+
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
new file mode 100644
index 0000000000..ee1b8989b5
--- /dev/null
+++ b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlLexer.g
@@ -0,0 +1,474 @@
+/**
+ 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; }
+ ;
+
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
new file mode 100644
index 0000000000..69574d713d
--- /dev/null
+++ b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/SparkSqlParser.g
@@ -0,0 +1,2457 @@
+/**
+ 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
+*/
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
new file mode 100644
index 0000000000..35ecdc5ad1
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTErrorNode.java
@@ -0,0 +1,49 @@
+/**
+ * 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(); }
+}
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
new file mode 100644
index 0000000000..33d9322b62
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ASTNode.java
@@ -0,0 +1,245 @@
+/**
+ * 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);
+ }
+}
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
new file mode 100644
index 0000000000..c77198b087
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseDriver.java
@@ -0,0 +1,213 @@
+/**
+ * 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;
+ }
+}
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
new file mode 100644
index 0000000000..b47bcfb291
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseError.java
@@ -0,0 +1,54 @@
+/**
+ * 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);
+ }
+
+}
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
new file mode 100644
index 0000000000..fff891ced5
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseException.java
@@ -0,0 +1,51 @@
+/**
+ * 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();
+ }
+
+}
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
new file mode 100644
index 0000000000..a5c2998f86
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/ParseUtils.java
@@ -0,0 +1,96 @@
+/**
+ * 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);
+ }
+
+}
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
new file mode 100644
index 0000000000..4b2015e0df
--- /dev/null
+++ b/sql/hive/src/main/java/org/apache/spark/sql/parser/SemanticAnalyzer.java
@@ -0,0 +1,406 @@
+/**
+ * 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;
+ }
+ }
+}
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 0e89928cb6..b1d841d1b5 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
@@ -27,28 +27,28 @@ import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry}
import org.apache.hadoop.hive.ql.lib.Node
-import org.apache.hadoop.hive.ql.parse._
+import org.apache.hadoop.hive.ql.parse.SemanticException
import org.apache.hadoop.hive.ql.plan.PlanUtils
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.ql.{Context, ErrorMsg}
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, catalyst}
+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.{logical, _}
import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.plans.{logical, _}
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.catalyst.TableIdentifier
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.hive.client._
import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema}
+import org.apache.spark.sql.parser._
import org.apache.spark.sql.types._
+import org.apache.spark.sql.{AnalysisException, catalyst}
import org.apache.spark.unsafe.types.CalendarInterval
import org.apache.spark.util.random.RandomSampler
@@ -227,7 +227,7 @@ private[hive] object HiveQl extends Logging {
*/
def withChildren(newChildren: Seq[ASTNode]): ASTNode = {
(1 to n.getChildCount).foreach(_ => n.deleteChild(0))
- n.addChildren(newChildren.asJava)
+ newChildren.foreach(n.addChild(_))
n
}
@@ -273,7 +273,8 @@ private[hive] object HiveQl extends Logging {
private def createContext(): Context = new Context(hiveConf)
private def getAst(sql: String, context: Context) =
- ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, context))
+ ParseUtils.findRootNonNullToken(
+ (new ParseDriver).parse(sql, context))
/**
* Returns the HiveConf
@@ -312,7 +313,7 @@ private[hive] object HiveQl extends Logging {
context.clear()
plan
} catch {
- case pe: org.apache.hadoop.hive.ql.parse.ParseException =>
+ case pe: ParseException =>
pe.getMessage match {
case errorRegEx(line, start, message) =>
throw new AnalysisException(message, Some(line.toInt), Some(start.toInt))
@@ -337,7 +338,8 @@ private[hive] object HiveQl extends Logging {
val tree =
try {
ParseUtils.findRootNonNullToken(
- (new ParseDriver).parse(ddl, null /* no context required for parsing alone */))
+ (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)
@@ -598,12 +600,12 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
NativePlaceholder
} else {
tableType match {
- case Token("TOK_TABTYPE", nameParts) if nameParts.size == 1 => {
- nameParts.head 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.head)
+ val tableIdent = extractTableIdent(nameParts)
DescribeCommand(
UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined)
case Token(".", dbName :: tableName :: colName :: Nil) =>
@@ -662,7 +664,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
NativePlaceholder
} else {
val schema = maybeColumns.map { cols =>
- BaseSemanticAnalyzer.getColumns(cols, true).asScala.map { field =>
+ SemanticAnalyzer.getColumns(cols, true).asScala.map { field =>
// 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)
@@ -678,7 +680,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
maybeComment.foreach {
case Token("TOK_TABLECOMMENT", child :: Nil) =>
- val comment = BaseSemanticAnalyzer.unescapeSQLString(child.getText)
+ val comment = SemanticAnalyzer.unescapeSQLString(child.getText)
if (comment ne null) {
properties += ("comment" -> comment)
}
@@ -750,7 +752,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
children.collect {
case list @ Token("TOK_TABCOLLIST", _) =>
- val cols = BaseSemanticAnalyzer.getColumns(list, true)
+ val cols = SemanticAnalyzer.getColumns(list, true)
if (cols != null) {
tableDesc = tableDesc.copy(
schema = cols.asScala.map { field =>
@@ -758,11 +760,11 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
})
}
case Token("TOK_TABLECOMMENT", child :: Nil) =>
- val comment = BaseSemanticAnalyzer.unescapeSQLString(child.getText)
+ 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 = BaseSemanticAnalyzer.getColumns(list(0), false)
+ val cols = SemanticAnalyzer.getColumns(list(0), false)
if (cols != null) {
tableDesc = tableDesc.copy(
partitionColumns = cols.asScala.map { field =>
@@ -773,21 +775,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
val serdeParams = new java.util.HashMap[String, String]()
child match {
case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) =>
- val fieldDelim = BaseSemanticAnalyzer.unescapeSQLString (rowChild1.getText())
+ val fieldDelim = SemanticAnalyzer.unescapeSQLString (rowChild1.getText())
serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim)
serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim)
if (rowChild2.length > 1) {
- val fieldEscape = BaseSemanticAnalyzer.unescapeSQLString (rowChild2(0).getText)
+ val fieldEscape = SemanticAnalyzer.unescapeSQLString (rowChild2(0).getText)
serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape)
}
case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) =>
- val collItemDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText)
+ val collItemDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim)
case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) =>
- val mapKeyDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText)
+ val mapKeyDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim)
case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) =>
- val lineDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText)
+ val lineDelim = SemanticAnalyzer.unescapeSQLString(rowChild.getText)
if (!(lineDelim == "\n") && !(lineDelim == "10")) {
throw new AnalysisException(
SemanticAnalyzer.generateErrorMessage(
@@ -796,22 +798,22 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
}
serdeParams.put(serdeConstants.LINE_DELIM, lineDelim)
case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) =>
- val nullFormat = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText)
+ 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 = BaseSemanticAnalyzer.unescapeSQLString(child.getText)
- location = EximUtil.relativeToAbsolutePath(hiveConf, location)
+ 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(BaseSemanticAnalyzer.unescapeSQLString(child.getChild(0).getText)))
+ serde = Option(SemanticAnalyzer.unescapeSQLString(child.getChild(0).getText)))
if (child.getChildCount == 2) {
val serdeParams = new java.util.HashMap[String, String]()
- BaseSemanticAnalyzer.readProps(
+ SemanticAnalyzer.readProps(
(child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams)
tableDesc = tableDesc.copy(
serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala)
@@ -891,9 +893,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
case list @ Token("TOK_TABLEFILEFORMAT", children) =>
tableDesc = tableDesc.copy(
inputFormat =
- Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)),
+ Option(SemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)),
outputFormat =
- Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(1).getText)))
+ Option(SemanticAnalyzer.unescapeSQLString(list.getChild(1).getText)))
case Token("TOK_STORAGEHANDLER", _) =>
throw new AnalysisException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg())
case _ => // Unsupport features
@@ -909,24 +911,20 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
Token("TOK_TABLE_PARTITION", table) :: Nil) => NativePlaceholder
case Token("TOK_QUERY", queryArgs)
- if Seq("TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) =>
+ if Seq("TOK_CTE", "TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) =>
val (fromClause: Option[ASTNode], insertClauses, cteRelations) =
queryArgs match {
- case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses =>
- // check if has CTE
- insertClauses.last match {
- case Token("TOK_CTE", cteClauses) =>
- val cteRelations = cteClauses.map(node => {
- val relation = nodeToRelation(node, context).asInstanceOf[Subquery]
- (relation.alias, relation)
- }).toMap
- (Some(args.head), insertClauses.init, Some(cteRelations))
-
- case _ => (Some(args.head), insertClauses, None)
+ 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
}
-
- case Token("TOK_INSERT", _) :: Nil => (None, queryArgs, None)
+ (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.
@@ -1025,20 +1023,20 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
(rowFormat, None, Nil, false)
case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil =>
- (Nil, Some(BaseSemanticAnalyzer.unescapeSQLString(serdeClass)), Nil, false)
+ (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) =>
- (BaseSemanticAnalyzer.unescapeSQLString(name),
- BaseSemanticAnalyzer.unescapeSQLString(value))
+ (SemanticAnalyzer.unescapeSQLString(name),
+ SemanticAnalyzer.unescapeSQLString(value))
}
// SPARK-10310: Special cases LazySimpleSerDe
// TODO Fully supports user-defined record reader/writer classes
- val unescapedSerDeClass = BaseSemanticAnalyzer.unescapeSQLString(serdeClass)
+ val unescapedSerDeClass = SemanticAnalyzer.unescapeSQLString(serdeClass)
val useDefaultRecordReaderWriter =
unescapedSerDeClass == classOf[LazySimpleSerDe].getCanonicalName
(Nil, Some(unescapedSerDeClass), serdeProps, useDefaultRecordReaderWriter)
@@ -1055,7 +1053,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) =
matchSerDe(outputSerdeClause)
- val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script)
+ val unescapedScript = SemanticAnalyzer.unescapeSQLString(script)
// TODO Adds support for user-defined record reader/writer classes
val recordReaderClass = if (useDefaultRecordReader) {
@@ -1361,6 +1359,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
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),
@@ -1475,11 +1474,11 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
}
val numericAstTypes = Seq(
- HiveParser.Number,
- HiveParser.TinyintLiteral,
- HiveParser.SmallintLiteral,
- HiveParser.BigintLiteral,
- HiveParser.DecimalLiteral)
+ SparkSqlParser.Number,
+ SparkSqlParser.TinyintLiteral,
+ SparkSqlParser.SmallintLiteral,
+ SparkSqlParser.BigintLiteral,
+ SparkSqlParser.DecimalLiteral)
/* Case insensitive matches */
val COUNT = "(?i)COUNT".r
@@ -1649,7 +1648,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
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 => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString)
+ 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
@@ -1684,37 +1683,37 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
v
}
- case ast: ASTNode if ast.getType == HiveParser.StringLiteral =>
- Literal(BaseSemanticAnalyzer.unescapeSQLString(ast.getText))
+ case ast: ASTNode if ast.getType == SparkSqlParser.StringLiteral =>
+ Literal(SemanticAnalyzer.unescapeSQLString(ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_DATELITERAL =>
+ 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 == HiveParser.TOK_CHARSETLITERAL =>
- Literal(BaseSemanticAnalyzer.charSetString(ast.getChild(0).getText, ast.getChild(1).getText))
+ 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 == HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_MONTH_LITERAL =>
Literal(CalendarInterval.fromYearMonthString(ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_TIME_LITERAL =>
Literal(CalendarInterval.fromDayTimeString(ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_YEAR_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_YEAR_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("year", ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_MONTH_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MONTH_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("month", ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_DAY_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_DAY_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("day", ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_HOUR_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_HOUR_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("hour", ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_MINUTE_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_MINUTE_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("minute", ast.getText))
- case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_SECOND_LITERAL =>
+ case ast: ASTNode if ast.getType == SparkSqlParser.TOK_INTERVAL_SECOND_LITERAL =>
Literal(CalendarInterval.fromSingleUnitString("second", ast.getText))
case a: ASTNode =>