You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/01/06 20:17:00 UTC

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

http://git-wip-us.apache.org/repos/asf/spark/blob/ea489f14/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g b/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
deleted file mode 100644
index 9f1e168..0000000
--- a/sql/hive/src/main/antlr3/org/apache/spark/sql/parser/IdentifiersParser.g
+++ /dev/null
@@ -1,697 +0,0 @@
-/**
-   Licensed to the Apache Software Foundation (ASF) under one or more 
-   contributor license agreements.  See the NOTICE file distributed with 
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with 
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-*/
-parser grammar IdentifiersParser;
-
-options
-{
-output=AST;
-ASTLabelType=CommonTree;
-backtrack=false;
-k=3;
-}
-
-@members {
-  @Override
-  public Object recoverFromMismatchedSet(IntStream input,
-      RecognitionException re, BitSet follow) throws RecognitionException {
-    throw re;
-  }
-  @Override
-  public void displayRecognitionError(String[] tokenNames,
-      RecognitionException e) {
-    gParent.errors.add(new ParseError(gParent, e, tokenNames));
-  }
-  protected boolean useSQL11ReservedKeywordsForIdentifier() {
-    return gParent.useSQL11ReservedKeywordsForIdentifier();
-  }
-}
-
-@rulecatch {
-catch (RecognitionException e) {
-  throw e;
-}
-}
-
-//-----------------------------------------------------------------------------------
-
-// group by a,b
-groupByClause
-@init { gParent.pushMsg("group by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_GROUP KW_BY
-    expression
-    ( COMMA expression)*
-    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
-    (sets=KW_GROUPING KW_SETS 
-    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
-    -> {rollup != null}? ^(TOK_ROLLUP_GROUPBY expression+)
-    -> {cube != null}? ^(TOK_CUBE_GROUPBY expression+)
-    -> {sets != null}? ^(TOK_GROUPING_SETS expression+ groupingSetExpression+)
-    -> ^(TOK_GROUPBY expression+)
-    ;
-
-groupingSetExpression
-@init {gParent.pushMsg("grouping set expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   (LPAREN) => groupingSetExpressionMultiple 
-   |
-   groupingExpressionSingle
-   ;
-
-groupingSetExpressionMultiple
-@init {gParent.pushMsg("grouping set part expression", state); }
-@after {gParent.popMsg(state); }
-   :
-   LPAREN 
-   expression? (COMMA expression)*
-   RPAREN
-   -> ^(TOK_GROUPING_SETS_EXPRESSION expression*)
-   ;
-
-groupingExpressionSingle
-@init { gParent.pushMsg("groupingExpression expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression -> ^(TOK_GROUPING_SETS_EXPRESSION expression)
-    ;
-
-havingClause
-@init { gParent.pushMsg("having clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_HAVING havingCondition -> ^(TOK_HAVING havingCondition)
-    ;
-
-havingCondition
-@init { gParent.pushMsg("having condition", state); }
-@after { gParent.popMsg(state); }
-    :
-    expression
-    ;
-
-expressionsInParenthese
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> expression+
-    ;
-
-expressionsNotInParenthese
-    :
-    expression (COMMA expression)* -> expression+
-    ;
-
-columnRefOrderInParenthese
-    :
-    LPAREN columnRefOrder (COMMA columnRefOrder)* RPAREN -> columnRefOrder+
-    ;
-
-columnRefOrderNotInParenthese
-    :
-    columnRefOrder (COMMA columnRefOrder)* -> columnRefOrder+
-    ;
-    
-// order by a,b
-orderByClause
-@init { gParent.pushMsg("order by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_ORDER KW_BY columnRefOrder ( COMMA columnRefOrder)* -> ^(TOK_ORDERBY columnRefOrder+)
-    ;
-    
-clusterByClause
-@init { gParent.pushMsg("cluster by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CLUSTER KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_CLUSTERBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_CLUSTERBY expressionsNotInParenthese)
-    )
-    ;
-
-partitionByClause
-@init  { gParent.pushMsg("partition by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_PARTITION KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-distributeByClause
-@init { gParent.pushMsg("distribute by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_DISTRIBUTE KW_BY
-    (
-    (LPAREN) => expressionsInParenthese -> ^(TOK_DISTRIBUTEBY expressionsInParenthese)
-    |
-    expressionsNotInParenthese -> ^(TOK_DISTRIBUTEBY expressionsNotInParenthese)
-    )
-    ;
-
-sortByClause
-@init { gParent.pushMsg("sort by clause", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_SORT KW_BY
-    (
-    (LPAREN) => columnRefOrderInParenthese -> ^(TOK_SORTBY columnRefOrderInParenthese)
-    |
-    columnRefOrderNotInParenthese -> ^(TOK_SORTBY columnRefOrderNotInParenthese)
-    )
-    ;
-
-// fun(par1, par2, par3)
-function
-@init { gParent.pushMsg("function specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    functionName
-    LPAREN
-      (
-        (STAR) => (star=STAR)
-        | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)?
-      )
-    RPAREN (KW_OVER ws=window_specification)?
-           -> {$star != null}? ^(TOK_FUNCTIONSTAR functionName $ws?)
-           -> {$dist == null}? ^(TOK_FUNCTION functionName (selectExpression+)? $ws?)
-                            -> ^(TOK_FUNCTIONDI functionName (selectExpression+)? $ws?)
-    ;
-
-functionName
-@init { gParent.pushMsg("function name", state); }
-@after { gParent.popMsg(state); }
-    : // Keyword IF is also a function name
-    (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE) => (KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE)
-    | 
-    (functionIdentifier) => functionIdentifier
-    |
-    {!useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsCastFunctionName -> Identifier[$sql11ReservedKeywordsUsedAsCastFunctionName.text]
-    ;
-
-castExpression
-@init { gParent.pushMsg("cast expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CAST
-    LPAREN
-          expression
-          KW_AS
-          primitiveType
-    RPAREN -> ^(TOK_FUNCTION primitiveType expression)
-    ;
-
-caseExpression
-@init { gParent.pushMsg("case expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CASE expression
-    (KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END -> ^(TOK_FUNCTION KW_CASE expression*)
-    ;
-
-whenExpression
-@init { gParent.pushMsg("case expression", state); }
-@after { gParent.popMsg(state); }
-    :
-    KW_CASE
-     ( KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END -> ^(TOK_FUNCTION KW_WHEN expression*)
-    ;
-
-constant
-@init { gParent.pushMsg("constant", state); }
-@after { gParent.popMsg(state); }
-    :
-    Number
-    | dateLiteral
-    | timestampLiteral
-    | intervalLiteral
-    | StringLiteral
-    | stringLiteralSequence
-    | BigintLiteral
-    | SmallintLiteral
-    | TinyintLiteral
-    | DecimalLiteral
-    | charSetStringLiteral
-    | booleanValue
-    ;
-
-stringLiteralSequence
-    :
-    StringLiteral StringLiteral+ -> ^(TOK_STRINGLITERALSEQUENCE StringLiteral StringLiteral+)
-    ;
-
-charSetStringLiteral
-@init { gParent.pushMsg("character string literal", state); }
-@after { gParent.popMsg(state); }
-    :
-    csName=CharSetName csLiteral=CharSetLiteral -> ^(TOK_CHARSETLITERAL $csName $csLiteral)
-    ;
-
-dateLiteral
-    :
-    KW_DATE StringLiteral ->
-    {
-      // Create DateLiteral token, but with the text of the string value
-      // This makes the dateLiteral more consistent with the other type literals.
-      adaptor.create(TOK_DATELITERAL, $StringLiteral.text)
-    }
-    |
-    KW_CURRENT_DATE -> ^(TOK_FUNCTION KW_CURRENT_DATE)
-    ;
-
-timestampLiteral
-    :
-    KW_TIMESTAMP StringLiteral ->
-    {
-      adaptor.create(TOK_TIMESTAMPLITERAL, $StringLiteral.text)
-    }
-    |
-    KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP)
-    ;
-
-intervalLiteral
-    :
-    KW_INTERVAL StringLiteral qualifiers=intervalQualifiers ->
-    {
-      adaptor.create($qualifiers.tree.token.getType(), $StringLiteral.text)
-    }
-    ;
-
-intervalQualifiers
-    :
-    KW_YEAR KW_TO KW_MONTH -> TOK_INTERVAL_YEAR_MONTH_LITERAL
-    | KW_DAY KW_TO KW_SECOND -> TOK_INTERVAL_DAY_TIME_LITERAL
-    | KW_YEAR -> TOK_INTERVAL_YEAR_LITERAL
-    | KW_MONTH -> TOK_INTERVAL_MONTH_LITERAL
-    | KW_DAY -> TOK_INTERVAL_DAY_LITERAL
-    | KW_HOUR -> TOK_INTERVAL_HOUR_LITERAL
-    | KW_MINUTE -> TOK_INTERVAL_MINUTE_LITERAL
-    | KW_SECOND -> TOK_INTERVAL_SECOND_LITERAL
-    ;
-
-expression
-@init { gParent.pushMsg("expression specification", state); }
-@after { gParent.popMsg(state); }
-    :
-    precedenceOrExpression
-    ;
-
-atomExpression
-    :
-    (KW_NULL) => KW_NULL -> TOK_NULL
-    | (constant) => constant
-    | castExpression
-    | caseExpression
-    | whenExpression
-    | (functionName LPAREN) => function
-    | tableOrColumn
-    | LPAREN! expression RPAREN!
-    ;
-
-
-precedenceFieldExpression
-    :
-    atomExpression ((LSQUARE^ expression RSQUARE!) | (DOT^ identifier))*
-    ;
-
-precedenceUnaryOperator
-    :
-    PLUS | MINUS | TILDE
-    ;
-
-nullCondition
-    :
-    KW_NULL -> ^(TOK_ISNULL)
-    | KW_NOT KW_NULL -> ^(TOK_ISNOTNULL)
-    ;
-
-precedenceUnaryPrefixExpression
-    :
-    (precedenceUnaryOperator^)* precedenceFieldExpression
-    ;
-
-precedenceUnarySuffixExpression
-    : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)?
-    -> {$a != null}? ^(TOK_FUNCTION nullCondition precedenceUnaryPrefixExpression)
-    -> precedenceUnaryPrefixExpression
-    ;
-
-
-precedenceBitwiseXorOperator
-    :
-    BITWISEXOR
-    ;
-
-precedenceBitwiseXorExpression
-    :
-    precedenceUnarySuffixExpression (precedenceBitwiseXorOperator^ precedenceUnarySuffixExpression)*
-    ;
-
-
-precedenceStarOperator
-    :
-    STAR | DIVIDE | MOD | DIV
-    ;
-
-precedenceStarExpression
-    :
-    precedenceBitwiseXorExpression (precedenceStarOperator^ precedenceBitwiseXorExpression)*
-    ;
-
-
-precedencePlusOperator
-    :
-    PLUS | MINUS
-    ;
-
-precedencePlusExpression
-    :
-    precedenceStarExpression (precedencePlusOperator^ precedenceStarExpression)*
-    ;
-
-
-precedenceAmpersandOperator
-    :
-    AMPERSAND
-    ;
-
-precedenceAmpersandExpression
-    :
-    precedencePlusExpression (precedenceAmpersandOperator^ precedencePlusExpression)*
-    ;
-
-
-precedenceBitwiseOrOperator
-    :
-    BITWISEOR
-    ;
-
-precedenceBitwiseOrExpression
-    :
-    precedenceAmpersandExpression (precedenceBitwiseOrOperator^ precedenceAmpersandExpression)*
-    ;
-
-
-// Equal operators supporting NOT prefix
-precedenceEqualNegatableOperator
-    :
-    KW_LIKE | KW_RLIKE | KW_REGEXP
-    ;
-
-precedenceEqualOperator
-    :
-    precedenceEqualNegatableOperator | EQUAL | EQUAL_NS | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
-    ;
-
-subQueryExpression 
-    : 
-    LPAREN! selectStatement[true] RPAREN!     
-    ;
-
-precedenceEqualExpression
-    :
-    (LPAREN precedenceBitwiseOrExpression COMMA) => precedenceEqualExpressionMutiple
-    |
-    precedenceEqualExpressionSingle
-    ;
-
-precedenceEqualExpressionSingle
-    :
-    (left=precedenceBitwiseOrExpression -> $left)
-    (
-       (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression)
-       -> ^(KW_NOT ^(precedenceEqualNegatableOperator $precedenceEqualExpressionSingle $notExpr))
-    | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
-       -> ^(precedenceEqualOperator $precedenceEqualExpressionSingle $equalExpr)
-    | (KW_NOT KW_IN LPAREN KW_SELECT)=>  (KW_NOT KW_IN subQueryExpression) 
-       -> ^(KW_NOT ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle))
-    | (KW_NOT KW_IN expressions)
-       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions))
-    | (KW_IN LPAREN KW_SELECT)=>  (KW_IN subQueryExpression) 
-       -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_IN) subQueryExpression $precedenceEqualExpressionSingle)
-    | (KW_IN expressions)
-       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionSingle expressions)
-    | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
-       -> ^(TOK_FUNCTION Identifier["between"] KW_TRUE $left $min $max)
-    | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
-       -> ^(TOK_FUNCTION Identifier["between"] KW_FALSE $left $min $max)
-    )*
-    | (KW_EXISTS LPAREN KW_SELECT)=> (KW_EXISTS subQueryExpression) -> ^(TOK_SUBQUERY_EXPR ^(TOK_SUBQUERY_OP KW_EXISTS) subQueryExpression)
-    ;
-
-expressions
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> expression+
-    ;
-
-//we transform the (col0, col1) in ((v00,v01),(v10,v11)) into struct(col0, col1) in (struct(v00,v01),struct(v10,v11))
-precedenceEqualExpressionMutiple
-    :
-    (LPAREN precedenceBitwiseOrExpression (COMMA precedenceBitwiseOrExpression)+ RPAREN -> ^(TOK_FUNCTION Identifier["struct"] precedenceBitwiseOrExpression+))
-    ( (KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
-       -> ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)
-    | (KW_NOT KW_IN LPAREN expressionsToStruct (COMMA expressionsToStruct)+ RPAREN)
-       -> ^(KW_NOT ^(TOK_FUNCTION KW_IN $precedenceEqualExpressionMutiple expressionsToStruct+)))
-    ;
-
-expressionsToStruct
-    :
-    LPAREN expression (COMMA expression)* RPAREN -> ^(TOK_FUNCTION Identifier["struct"] expression+)
-    ;
-
-precedenceNotOperator
-    :
-    KW_NOT
-    ;
-
-precedenceNotExpression
-    :
-    (precedenceNotOperator^)* precedenceEqualExpression
-    ;
-
-
-precedenceAndOperator
-    :
-    KW_AND
-    ;
-
-precedenceAndExpression
-    :
-    precedenceNotExpression (precedenceAndOperator^ precedenceNotExpression)*
-    ;
-
-
-precedenceOrOperator
-    :
-    KW_OR
-    ;
-
-precedenceOrExpression
-    :
-    precedenceAndExpression (precedenceOrOperator^ precedenceAndExpression)*
-    ;
-
-
-booleanValue
-    :
-    KW_TRUE^ | KW_FALSE^
-    ;
-
-booleanValueTok
-   :
-   KW_TRUE -> TOK_TRUE
-   | KW_FALSE -> TOK_FALSE
-   ;
-
-tableOrPartition
-   :
-   tableName partitionSpec? -> ^(TOK_TAB tableName partitionSpec?)
-   ;
-
-partitionSpec
-    :
-    KW_PARTITION
-     LPAREN partitionVal (COMMA  partitionVal )* RPAREN -> ^(TOK_PARTSPEC partitionVal +)
-    ;
-
-partitionVal
-    :
-    identifier (EQUAL constant)? -> ^(TOK_PARTVAL identifier constant?)
-    ;
-
-dropPartitionSpec
-    :
-    KW_PARTITION
-     LPAREN dropPartitionVal (COMMA  dropPartitionVal )* RPAREN -> ^(TOK_PARTSPEC dropPartitionVal +)
-    ;
-
-dropPartitionVal
-    :
-    identifier dropPartitionOperator constant -> ^(TOK_PARTVAL identifier dropPartitionOperator constant)
-    ;
-
-dropPartitionOperator
-    :
-    EQUAL | NOTEQUAL | LESSTHANOREQUALTO | LESSTHAN | GREATERTHANOREQUALTO | GREATERTHAN
-    ;
-
-sysFuncNames
-    :
-      KW_AND
-    | KW_OR
-    | KW_NOT
-    | KW_LIKE
-    | KW_IF
-    | KW_CASE
-    | KW_WHEN
-    | KW_TINYINT
-    | KW_SMALLINT
-    | KW_INT
-    | KW_BIGINT
-    | KW_FLOAT
-    | KW_DOUBLE
-    | KW_BOOLEAN
-    | KW_STRING
-    | KW_BINARY
-    | KW_ARRAY
-    | KW_MAP
-    | KW_STRUCT
-    | KW_UNIONTYPE
-    | EQUAL
-    | EQUAL_NS
-    | NOTEQUAL
-    | LESSTHANOREQUALTO
-    | LESSTHAN
-    | GREATERTHANOREQUALTO
-    | GREATERTHAN
-    | DIVIDE
-    | PLUS
-    | MINUS
-    | STAR
-    | MOD
-    | DIV
-    | AMPERSAND
-    | TILDE
-    | BITWISEOR
-    | BITWISEXOR
-    | KW_RLIKE
-    | KW_REGEXP
-    | KW_IN
-    | KW_BETWEEN
-    ;
-
-descFuncNames
-    :
-      (sysFuncNames) => sysFuncNames
-    | StringLiteral
-    | functionIdentifier
-    ;
-
-identifier
-    :
-    Identifier
-    | nonReserved -> Identifier[$nonReserved.text]
-    // If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false, 
-    // the sql11keywords in existing q tests will NOT be added back.
-    | {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text]
-    ;
-
-functionIdentifier
-@init { gParent.pushMsg("function identifier", state); }
-@after { gParent.popMsg(state); }
-    : db=identifier DOT fn=identifier
-    -> Identifier[$db.text + "." + $fn.text]
-    |
-    identifier
-    ;
-
-principalIdentifier
-@init { gParent.pushMsg("identifier for principal spec", state); }
-@after { gParent.popMsg(state); }
-    : identifier
-    | QuotedIdentifier
-    ;
-
-//The new version of nonReserved + sql11ReservedKeywordsUsedAsIdentifier = old version of nonReserved
-//Non reserved keywords are basically the keywords that can be used as identifiers.
-//All the KW_* are automatically not only keywords, but also reserved keywords.
-//That means, they can NOT be used as identifiers.
-//If you would like to use them as identifiers, put them in the nonReserved list below.
-//If you are not sure, please refer to the SQL2011 column in
-//http://www.postgresql.org/docs/9.5/static/sql-keywords-appendix.html
-nonReserved
-    :
-    KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS
-    | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS
-    | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY
-    | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY 
-    | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_ELEM_TYPE 
-    | KW_ENABLE | KW_ESCAPED | KW_EXCLUSIVE | KW_EXPLAIN | KW_EXPORT | KW_FIELDS | KW_FILE | KW_FILEFORMAT
-    | KW_FIRST | KW_FORMAT | KW_FORMATTED | KW_FUNCTIONS | KW_HOLD_DDLTIME | KW_HOUR | KW_IDXPROPERTIES | KW_IGNORE
-    | KW_INDEX | KW_INDEXES | KW_INPATH | KW_INPUTDRIVER | KW_INPUTFORMAT | KW_ITEMS | KW_JAR
-    | KW_KEYS | KW_KEY_TYPE | KW_LIMIT | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG
-    | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE
-    | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY
-    | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
-    | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE
-    | KW_ROLE | KW_ROLES | KW_SCHEMA | KW_SCHEMAS | KW_SECOND | KW_SEMI | KW_SERDE | KW_SERDEPROPERTIES | KW_SERVER | KW_SETS | KW_SHARED
-    | KW_SHOW | KW_SHOW_DATABASE | KW_SKEWED | KW_SORT | KW_SORTED | KW_SSL | KW_STATISTICS | KW_STORED
-    | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED
-    | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET
-    | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR
-    | KW_WORK
-    | KW_TRANSACTION
-    | KW_WRITE
-    | KW_ISOLATION
-    | KW_LEVEL
-    | KW_SNAPSHOT
-    | KW_AUTOCOMMIT
-    | KW_ANTI
-;
-
-//The following SQL2011 reserved keywords are used as cast function name only, but not as identifiers.
-sql11ReservedKeywordsUsedAsCastFunctionName
-    :
-    KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP
-    ;
-
-//The following SQL2011 reserved keywords are used as identifiers in many q tests, they may be added back due to backward compatibility.
-//We are planning to remove the following whole list after several releases.
-//Thus, please do not change the following list unless you know what to do.
-sql11ReservedKeywordsUsedAsIdentifier
-    :
-    KW_ALL | KW_ALTER | KW_ARRAY | KW_AS | KW_AUTHORIZATION | KW_BETWEEN | KW_BIGINT | KW_BINARY | KW_BOOLEAN 
-    | KW_BOTH | KW_BY | KW_CREATE | KW_CUBE | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_CURSOR | KW_DATE | KW_DECIMAL | KW_DELETE | KW_DESCRIBE 
-    | KW_DOUBLE | KW_DROP | KW_EXISTS | KW_EXTERNAL | KW_FALSE | KW_FETCH | KW_FLOAT | KW_FOR | KW_FULL | KW_GRANT 
-    | KW_GROUP | KW_GROUPING | KW_IMPORT | KW_IN | KW_INNER | KW_INSERT | KW_INT | KW_INTERSECT | KW_INTO | KW_IS | KW_LATERAL 
-    | KW_LEFT | KW_LIKE | KW_LOCAL | KW_NONE | KW_NULL | KW_OF | KW_ORDER | KW_OUT | KW_OUTER | KW_PARTITION 
-    | KW_PERCENT | KW_PROCEDURE | KW_RANGE | KW_READS | KW_REVOKE | KW_RIGHT 
-    | KW_ROLLUP | KW_ROW | KW_ROWS | KW_SET | KW_SMALLINT | KW_TABLE | KW_TIMESTAMP | KW_TO | KW_TRIGGER | KW_TRUE 
-    | KW_TRUNCATE | KW_UNION | KW_UPDATE | KW_USER | KW_USING | KW_VALUES | KW_WITH 
-//The following two keywords come from MySQL. Although they are not keywords in SQL2011, they are reserved keywords in MySQL.    
-    | KW_REGEXP | KW_RLIKE
-    ;

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

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


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