You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2014/02/23 07:42:08 UTC

[1/5] TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)

Repository: incubator-tajo
Updated Branches:
  refs/heads/branch-0.8.0 5fb959b7c -> 7f5d5eca5


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
new file mode 100644
index 0000000..1228324
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveQLAnalyzer.java
@@ -0,0 +1,1568 @@
+/**
+ * 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.tajo.engine.parser;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.TerminalNodeImpl;
+import org.apache.commons.lang.math.NumberUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.algebra.*;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.engine.parser.HiveQLParser.TableAllColumnsContext;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class HiveQLAnalyzer extends HiveQLParserBaseVisitor<Expr> {
+  private static final Log LOG = LogFactory.getLog(HiveQLAnalyzer.class.getName());
+  private HiveQLParser parser;
+
+  public Expr parse(String sql) {
+    HiveQLLexer lexer = new HiveQLLexer(new ANTLRNoCaseStringStream(sql));
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    parser = new HiveQLParser(tokens);
+    parser.setBuildParseTree(true);
+
+    HiveQLParser.StatementContext context;
+    try {
+      context = parser.statement();
+    } catch (SQLParseError e) {
+      throw new SQLSyntaxError(e);
+    }
+
+    return visit(context);
+  }
+
+  @Override
+  public Expr visitStatement(HiveQLParser.StatementContext ctx) {
+    return visitExecStatement(ctx.execStatement());
+  }
+
+  @Override
+  public Expr visitQueryStatement(HiveQLParser.QueryStatementContext ctx) {
+    Expr current = null;
+
+    if (ctx.body != null) {
+      current = visitBody(ctx.body(0));
+    }
+
+    if (ctx.regular_body() != null) {
+      current = visitRegular_body(ctx.regular_body());
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitBody(HiveQLParser.BodyContext ctx) {
+
+    Expr current = null;
+    Insert insert = null;
+
+    Projection select = null;
+
+    if (ctx.insertClause() != null) {
+      insert = visitInsertClause(ctx.insertClause());
+    }
+
+    if (ctx.selectClause() != null) {
+      select = (Projection) visitSelectClause(ctx.selectClause());
+      if (ctx.selectClause().KW_DISTINCT() != null) {
+        select.setDistinct();
+      }
+
+    }
+
+    for (int i = 0; i < ctx.getParent().getChildCount(); i++) {
+      if (ctx.getParent().getChild(i) instanceof HiveQLParser.FromClauseContext) {
+        HiveQLParser.FromClauseContext fromClauseContext = (HiveQLParser.FromClauseContext) ctx.getParent().getChild(i);
+        Expr from = visitFromClause(fromClauseContext);
+        current = from;
+      }
+    }
+
+    if (ctx.whereClause() != null) {
+      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+      where.setChild(current);
+      current = where;
+    }
+
+    if (ctx.groupByClause() != null) {
+      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+      aggregation.setChild(current);
+      current = aggregation;
+
+      if (ctx.havingClause() != null) {
+        Expr havingCondition = visitHavingClause(ctx.havingClause());
+        Having having = new Having(havingCondition);
+        having.setChild(current);
+        current = having;
+      }
+    }
+
+    if (ctx.orderByClause() != null) {
+      Sort sort = visitOrderByClause(ctx.orderByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.clusterByClause() != null) {
+      visitClusterByClause(ctx.clusterByClause());
+    }
+
+    if (ctx.distributeByClause() != null) {
+      visitDistributeByClause(ctx.distributeByClause());
+    }
+
+    if (ctx.sortByClause() != null) {
+      Sort sort = visitSortByClause(ctx.sortByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.window_clause() != null) {
+      Expr window = visitWindow_clause(ctx.window_clause());
+    }
+
+    if (ctx.limitClause() != null) {
+      Limit limit = visitLimitClause(ctx.limitClause());
+      limit.setChild(current);
+      current = limit;
+    }
+
+    Projection projection = new Projection();
+    projection.setNamedExprs(select.getNamedExprs());
+
+    if (current != null)
+      projection.setChild(current);
+
+    if (select.isDistinct())
+      projection.setDistinct();
+
+
+    if (insert != null) {
+      insert.setSubQuery(projection);
+      current = insert;
+    } else {
+      current = projection;
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitRegular_body(HiveQLParser.Regular_bodyContext ctx) {
+    Expr current = null;
+    Insert insert = null;
+
+    if (ctx.selectStatement() != null) {
+      current = visitSelectStatement(ctx.selectStatement());
+    } else {
+      Projection select = null;
+
+      if (ctx.insertClause() != null) {
+        insert = visitInsertClause(ctx.insertClause());
+      }
+
+      if (ctx.selectClause() != null) {
+        select = (Projection) visitSelectClause(ctx.selectClause());
+        if (ctx.selectClause().KW_DISTINCT() != null) {
+          select.setDistinct();
+        }
+
+      }
+
+      if (ctx.fromClause() != null) {
+        Expr from = visitFromClause(ctx.fromClause());
+        current = from;
+      }
+
+      if (ctx.whereClause() != null) {
+        Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+        where.setChild(current);
+        current = where;
+      }
+
+      if (ctx.groupByClause() != null) {
+        Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+        aggregation.setChild(current);
+        current = aggregation;
+
+        if (ctx.havingClause() != null) {
+          Expr havingCondition = visitHavingClause(ctx.havingClause());
+          Having having = new Having(havingCondition);
+          having.setChild(current);
+          current = having;
+        }
+      }
+
+      if (ctx.orderByClause() != null) {
+        Sort sort = visitOrderByClause(ctx.orderByClause());
+        sort.setChild(current);
+        current = sort;
+      }
+
+      if (ctx.clusterByClause() != null) {
+        visitClusterByClause(ctx.clusterByClause());
+      }
+
+      if (ctx.distributeByClause() != null) {
+        visitDistributeByClause(ctx.distributeByClause());
+      }
+
+      if (ctx.sortByClause() != null) {
+        Sort sort = visitSortByClause(ctx.sortByClause());
+        sort.setChild(current);
+        current = sort;
+      }
+
+      if (ctx.window_clause() != null) {
+        Expr window = visitWindow_clause(ctx.window_clause());
+      }
+
+      if (ctx.limitClause() != null) {
+        Limit limit = visitLimitClause(ctx.limitClause());
+        limit.setChild(current);
+        current = limit;
+      }
+
+      Projection projection = new Projection();
+      projection.setNamedExprs(select.getNamedExprs());
+
+      if (current != null)
+        projection.setChild(current);
+
+      if (select.isDistinct())
+        projection.setDistinct();
+
+      if (insert != null) {
+        insert.setSubQuery(projection);
+        current = insert;
+      } else {
+        current = projection;
+      }
+
+
+    }
+    return current;
+  }
+
+  /**
+   * This method implemented for parsing union all clause.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitQueryStatementExpression(HiveQLParser.QueryStatementExpressionContext ctx) {
+    Expr left = null, right = null, current = null;
+    if (ctx.queryStatement() != null) {
+      if (ctx.queryStatement().size() == 1)
+        return visitQueryStatement(ctx.queryStatement(0));
+
+      for (int i = 0; i < ctx.queryStatement().size(); i++) {
+        if (i == 0)
+          current = visitQueryStatement(ctx.queryStatement(i));
+        else
+          left = current;
+
+        if (i > 0) {
+          right = visitQueryStatement(ctx.queryStatement(i));
+          current = new SetOperation(OpType.Union, left, right, false);
+        }
+      }
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitSelectStatement(HiveQLParser.SelectStatementContext ctx) {
+    Expr current = null;
+
+    Projection select = (Projection) visitSelectClause(ctx.selectClause());
+
+    if (ctx.selectClause().KW_DISTINCT() != null) {
+      select.setDistinct();
+    }
+
+    Expr from = visitFromClause(ctx.fromClause());
+    current = from;
+
+    if (ctx.whereClause() != null) {
+      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
+      where.setChild(current);
+      current = where;
+    }
+
+    if (ctx.groupByClause() != null) {
+      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
+      aggregation.setChild(current);
+      current = aggregation;
+
+      if (ctx.havingClause() != null) {
+        Expr havingCondition = visitHavingClause(ctx.havingClause());
+        Having having = new Having(havingCondition);
+        having.setChild(current);
+        current = having;
+      }
+    }
+
+    if (ctx.orderByClause() != null) {
+      Sort sort = visitOrderByClause(ctx.orderByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.clusterByClause() != null) {
+      visitClusterByClause(ctx.clusterByClause());
+    }
+
+    if (ctx.distributeByClause() != null) {
+      visitDistributeByClause(ctx.distributeByClause());
+    }
+
+    if (ctx.sortByClause() != null) {
+      Sort sort = visitSortByClause(ctx.sortByClause());
+      sort.setChild(current);
+      current = sort;
+    }
+
+    if (ctx.window_clause() != null) {
+      Expr window = visitWindow_clause(ctx.window_clause());
+    }
+
+    if (ctx.limitClause() != null) {
+      Limit limit = visitLimitClause(ctx.limitClause());
+      limit.setChild(current);
+      current = limit;
+    }
+
+    Projection projection = new Projection();
+    projection.setNamedExprs(select.getNamedExprs());
+
+    if (current != null)
+      projection.setChild(current);
+
+    if (select.isDistinct())
+      projection.setDistinct();
+
+    current = projection;
+
+    return current;
+  }
+
+  @Override
+  public Expr visitFromClause(HiveQLParser.FromClauseContext ctx) {
+    return visitJoinSource(ctx.joinSource());
+  }
+
+  @Override
+  public Expr visitJoinSource(HiveQLParser.JoinSourceContext ctx) {
+    Expr[] relations = null;
+    RelationList relationList = null;
+
+    if (ctx.fromSource() != null) {
+      int fromCount = ctx.fromSource().size();
+      int uniqueJoinCount = ctx.uniqueJoinSource().size();
+
+      relations = new Expr[1];
+
+      Join current = null, parent = null;
+      JoinType type = null;
+      Expr left = null, right = null, condition = null;
+
+
+      if (fromCount == 1) {
+        relations[0] = visitFromSource(ctx.fromSource(0));
+      } else {
+        left = visitFromSource((HiveQLParser.FromSourceContext) ctx.getChild(0));
+
+        for (int i = 1; i < ctx.getChildCount(); i++) {
+          type = null;
+          right = null;
+          condition = null;
+
+          if (ctx.getChild(i) instanceof HiveQLParser.JoinTokenContext) {
+            type = getJoinType((HiveQLParser.JoinTokenContext) ctx.getChild(i));
+            if (i > 1)
+              left = parent;
+
+            if (i + 1 < ctx.getChildCount() && ctx.getChild(i + 1) instanceof HiveQLParser.FromSourceContext) {
+              right = visitFromSource((HiveQLParser.FromSourceContext) ctx.getChild(i + 1));
+            }
+
+            if (i + 3 < ctx.getChildCount() && ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+              condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+            }
+
+            if (type != null) {
+              current = new Join(type);
+              current.setLeft(left);
+              current.setRight(right);
+
+              if (condition != null)
+                current.setQual(condition);
+
+              parent = current;
+            }
+          }
+
+        }
+        relations[0] = current;
+      }
+
+      //TODO: implement unique join.
+      relationList = new RelationList(relations);
+    }
+
+    return relationList;
+  }
+
+  public JoinType getJoinType(HiveQLParser.JoinTokenContext context) {
+    JoinType type = JoinType.INNER;
+
+    if (context.KW_INNER() != null) {
+      type = JoinType.INNER;
+    }
+
+    if (context.KW_LEFT() != null && context.KW_OUTER() != null) {
+      type = JoinType.LEFT_OUTER;
+    }
+
+    if (context.KW_RIGHT() != null && context.KW_OUTER() != null) {
+      type = JoinType.RIGHT_OUTER;
+    }
+
+    if (context.KW_CROSS() != null) {
+      type = JoinType.CROSS;
+    }
+
+    if (context.KW_FULL() != null) {
+      type = JoinType.FULL_OUTER;
+    }
+
+    if (context.KW_SEMI() != null) {
+      type = null;
+    }
+    return type;
+  }
+
+  @Override
+  public Expr visitFromSource(HiveQLParser.FromSourceContext ctx) {
+    Expr current = null;
+
+    if (ctx.Identifier() != null && ctx.LPAREN() != null) {
+      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
+    }
+
+    if (ctx.tableSource() != null) {
+      current = visitTableSource(ctx.tableSource());
+    }
+
+    if (ctx.subQuerySource() != null) {
+      current = visitSubQuerySource(ctx.subQuerySource());
+
+      String tableAlias = "";
+      for (int i = 0; i < ctx.subQuerySource().getChildCount(); i++) {
+        if (ctx.subQuerySource().getChild(i) instanceof HiveQLParser.IdentifierContext) {
+          tableAlias = (ctx.subQuerySource().getChild(i)).getText();
+        }
+      }
+
+      TablePrimarySubQuery subQuery = new TablePrimarySubQuery(tableAlias, current);
+      current = subQuery;
+    }
+    // TODO: implement lateralView
+
+    return current;
+  }
+
+  @Override
+  public Expr visitSubQuerySource(HiveQLParser.SubQuerySourceContext ctx) {
+    Expr current = visitQueryStatementExpression(ctx.queryStatementExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitTableSource(HiveQLParser.TableSourceContext ctx) {
+    String tableName = "", alias = "";
+
+    if (ctx.tableName() != null)
+      tableName = ctx.tableName().getText();
+
+    if (ctx.alias != null) {
+      alias = ctx.alias.getText();
+      for (String token : HiveQLParser.tokenNames) {
+        if (token.replaceAll("'", "").equalsIgnoreCase(alias))
+          alias = "";
+      }
+    }
+
+    Relation relation = new Relation(tableName);
+    if (!alias.equals(""))
+      relation.setAlias(alias);
+
+    return relation;
+  }
+
+  @Override
+  public Expr visitSelectList(HiveQLParser.SelectListContext ctx) {
+    Expr current = null;
+    Projection projection = new Projection();
+    NamedExpr[] targets = new NamedExpr[ctx.selectItem().size()];
+    for (int i = 0; i < targets.length; i++) {
+      targets[i] = visitSelectItem(ctx.selectItem(i));
+    }
+
+    projection.setNamedExprs(targets);
+    current = projection;
+    return current;
+  }
+
+  @Override
+  public NamedExpr visitSelectItem(HiveQLParser.SelectItemContext ctx) {
+    NamedExpr target = null;
+
+    if (ctx.selectExpression() != null) {
+      target = new NamedExpr(visitSelectExpression(ctx.selectExpression()));
+    } else if (ctx.window_specification() != null) {
+      // TODO: if there is a window specification clause, we should handle it properly.
+    }
+
+    if (ctx.identifier().size() > 0 && target != null) {
+      target.setAlias(ctx.identifier(0).getText());
+    }
+    return target;
+  }
+
+  @Override
+  public Expr visitSelectExpression(HiveQLParser.SelectExpressionContext ctx) {
+    Expr current = null;
+
+    if (ctx.tableAllColumns() != null) {
+      current = visitTableAllColumns(ctx.tableAllColumns());
+    } else {
+      if (ctx.expression() != null) {
+        current = visitExpression(ctx.expression());
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitTableAllColumns(TableAllColumnsContext ctx) {
+    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
+    if (ctx.tableName() != null) {
+      target.setQualifier(ctx.tableName().getText());
+    }
+
+    return target;
+  }
+
+  @Override
+  public Expr visitExpression(HiveQLParser.ExpressionContext ctx) {
+    Expr current = visitPrecedenceOrExpression(ctx.precedenceOrExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceOrExpression(HiveQLParser.PrecedenceOrExpressionContext ctx) {
+    Expr current = null, left = null, right = null;
+
+    for (int i = 0; i < ctx.precedenceAndExpression().size(); i++) {
+      if (i == 0) {
+        left = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
+        current = left;
+      } else {
+        left = current;
+        right = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
+        current = new BinaryOperator(OpType.Or, left, right);
+      }
+
+    }
+    return current;
+  }
+
+  /**
+   * This method parse AND expressions at WHERE clause.
+   * And this convert 'x BETWEEN y AND z' expression into 'x >= y AND x <= z' expression
+   * because Tajo doesn't provide 'BETWEEN' expression.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitPrecedenceAndExpression(HiveQLParser.PrecedenceAndExpressionContext ctx) {
+    Expr current = null, left = null, right = null;
+
+    for (int i = 0; i < ctx.precedenceNotExpression().size(); i++) {
+      Expr min = null, max = null;
+
+      if (ctx.precedenceNotExpression(i).precedenceEqualExpression() != null) {
+        HiveQLParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceNotExpression(i)
+            .precedenceEqualExpression();
+        if (expressionContext.KW_BETWEEN() != null) {
+
+          if (expressionContext.min != null) {
+            min = visitPrecedenceBitwiseOrExpression(expressionContext.min);
+          }
+
+          if (expressionContext.max != null) {
+            max = visitPrecedenceBitwiseOrExpression(expressionContext.max);
+          }
+        }
+      }
+
+      if (min != null && max != null) {
+        left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+        if (left != null) {
+          if (i == 0) {
+            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
+            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
+            current = new BinaryOperator(OpType.And, minOperator, maxOperator);
+          } else {
+            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
+            current = new BinaryOperator(OpType.And, current, minOperator);
+
+            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
+            current = new BinaryOperator(OpType.And, current, maxOperator);
+          }
+        }
+      } else {
+        if (i == 0) {
+          left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+          current = left;
+        } else {
+          left = current;
+          right = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
+          current = new BinaryOperator(OpType.And, left, right);
+        }
+      }
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceNotExpression(HiveQLParser.PrecedenceNotExpressionContext ctx) {
+    HiveQLParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceEqualExpression();
+    Expr current = visitPrecedenceEqualExpression(expressionContext);
+    return current;
+  }
+
+  /**
+   * This method parse operators for equals expressions as follows:
+   * =, <>, !=, >=, >, <=, <, IN, NOT IN, LIKE, REGEXP, RLIKE
+   * <p/>
+   * In this case, this make RuntimeException>
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitPrecedenceEqualExpression(HiveQLParser.PrecedenceEqualExpressionContext ctx) {
+    Expr current = null, left = null, right = null, min = null, max = null;
+    OpType type = null;
+    boolean isNot = false, isIn = false;
+    for (int i = 0; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof HiveQLParser.PrecedenceBitwiseOrExpressionContext) {
+        if (i == 0) {
+          left = visitPrecedenceBitwiseOrExpression((HiveQLParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
+        } else {
+          right = visitPrecedenceBitwiseOrExpression((HiveQLParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
+        }
+      } else if (ctx.getChild(i) instanceof HiveQLParser.ExpressionsContext) {
+        right = visitExpressions((HiveQLParser.ExpressionsContext) ctx.getChild(i));
+      } else if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        int symbolType = ((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType();
+        switch (symbolType) {
+          case HiveQLLexer.KW_NOT:
+            isNot = true;
+            break;
+          case HiveQLLexer.KW_IN:
+            isIn = true;
+            break;
+          default:
+            break;
+        }
+      } else if (ctx.getChild(i) instanceof HiveQLParser.PrecedenceEqualOperatorContext
+          || ctx.getChild(i) instanceof HiveQLParser.PrecedenceEqualNegatableOperatorContext) {
+        String keyword = ctx.getChild(i).getText().toUpperCase();
+
+        if (keyword.equals(">")) {
+          type = OpType.GreaterThan;
+        } else if (keyword.equals("<=>")) {
+          throw new RuntimeException("Unexpected operator : <=>");
+        } else if (keyword.equals("=")) {
+          type = OpType.Equals;
+        } else if (keyword.equals("<=")) {
+          type = OpType.LessThanOrEquals;
+        } else if (keyword.equals("<")) {
+          type = OpType.LessThan;
+        } else if (keyword.equals(">=")) {
+          type = OpType.GreaterThanOrEquals;
+        } else if (keyword.equals("<>")) {
+          type = OpType.NotEquals;
+        } else if (keyword.equals("!=")) {
+          type = OpType.NotEquals;
+        } else if (keyword.equals("REGEXP")) {
+          type = OpType.Regexp;
+        } else if (keyword.equals("RLIKE")) {
+          type = OpType.Regexp;
+        } else if (keyword.equals("LIKE")) {
+          type = OpType.LikePredicate;
+        }
+      }
+    }
+
+    if (type != null && right != null) {
+      if (type.equals(OpType.LikePredicate)) {
+        PatternMatchPredicate like = new PatternMatchPredicate(OpType.LikePredicate,
+            isNot, left, right);
+        current = like;
+      } else if (type.equals(OpType.Regexp)) {
+        PatternMatchPredicate regex = new PatternMatchPredicate(OpType.Regexp, isNot, left, right);
+        current = regex;
+      } else {
+        BinaryOperator binaryOperator = new BinaryOperator(type, left, right);
+        current = binaryOperator;
+      }
+    } else if (isIn) {
+      InPredicate inPredicate = new InPredicate(left, right, isNot);
+      current = inPredicate;
+    } else {
+      current = left;
+    }
+
+    return current;
+  }
+
+  @Override
+  public ValueListExpr visitExpressions(HiveQLParser.ExpressionsContext ctx) {
+    int size = ctx.expression().size();
+    Expr[] exprs = new Expr[size];
+    for (int i = 0; i < size; i++) {
+      exprs[i] = visitExpression(ctx.expression(i));
+    }
+    return new ValueListExpr(exprs);
+  }
+
+  @Override
+  public Expr visitPrecedenceBitwiseOrExpression(HiveQLParser.PrecedenceBitwiseOrExpressionContext ctx) {
+    int expressionCount = ctx.precedenceAmpersandExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceBitwiseOrOperator(operatorIndex) != null) {
+        type = getPrecedenceBitwiseOrOperator(ctx.precedenceBitwiseOrOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
+        if (ctx.precedenceAmpersandExpression(i + 1) != null)
+          right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i + 1));
+      } else {
+        parentType = getPrecedenceBitwiseOrOperator((ctx.precedenceBitwiseOrOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 2));
+        parentRight = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedenceBitwiseOrOperator(HiveQLParser.PrecedenceBitwiseOrOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceAmpersandExpression(HiveQLParser.PrecedenceAmpersandExpressionContext ctx) {
+    int expressionCount = ctx.precedencePlusExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceAmpersandOperator(operatorIndex) != null) {
+        type = getPrecedenceAmpersandOperator(ctx.precedenceAmpersandOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
+        if (ctx.precedencePlusExpression(i + 1) != null)
+          right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i + 1));
+      } else {
+        parentType = getPrecedenceAmpersandOperator((ctx.precedenceAmpersandOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 2));
+        parentRight = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedenceAmpersandOperator(HiveQLParser.PrecedenceAmpersandOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedencePlusExpression(HiveQLParser.PrecedencePlusExpressionContext ctx) {
+    int expressionCount = ctx.precedenceStarExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedencePlusOperator(operatorIndex) != null) {
+        type = getPrecedencePlusOperator(ctx.precedencePlusOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
+        if (ctx.precedenceStarExpression(i + 1) != null)
+          right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i + 1));
+      } else {
+        parentType = getPrecedencePlusOperator((ctx.precedencePlusOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 2));
+        parentRight = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+    return current;
+  }
+
+  public OpType getPrecedencePlusOperator(HiveQLParser.PrecedencePlusOperatorContext ctx) {
+    OpType type = null;
+
+    if (ctx.MINUS() != null) {
+      type = OpType.Minus;
+    } else if (ctx.PLUS() != null) {
+      type = OpType.Plus;
+    }
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceStarExpression(HiveQLParser.PrecedenceStarExpressionContext ctx) {
+    int expressionCount = ctx.precedenceBitwiseXorExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceStarOperator(operatorIndex) != null) {
+        type = getPrecedenceStarOperator(ctx.precedenceStarOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
+        if (ctx.precedenceBitwiseXorExpression(i + 1) != null)
+          right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i + 1));
+      } else {
+        parentType = getPrecedenceStarOperator((ctx.precedenceStarOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 2));
+        parentRight = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+
+    return current;
+  }
+
+  public OpType getPrecedenceStarOperator(HiveQLParser.PrecedenceStarOperatorContext ctx) {
+    OpType type = null;
+
+    if (ctx.DIV() != null) {
+      type = OpType.Divide;
+    } else if (ctx.DIVIDE() != null) {
+      type = OpType.Divide;
+    } else if (ctx.MOD() != null) {
+      type = OpType.Modular;
+    } else if (ctx.STAR() != null) {
+      type = OpType.Multiply;
+    }
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceBitwiseXorExpression(HiveQLParser.PrecedenceBitwiseXorExpressionContext ctx) {
+    int expressionCount = ctx.precedenceUnarySuffixExpression().size();
+
+    Expr current = null, left = null, right = null, parentLeft, parentRight;
+    OpType type = null, parentType = null;
+
+    for (int i = 0; i < expressionCount; i += 2) {
+      int operatorIndex = (i == 0) ? 0 : i - 1;
+
+      if (ctx.precedenceBitwiseXorOperator(operatorIndex) != null) {
+        type = getPrecedenceBitwiseXorOperator(ctx.precedenceBitwiseXorOperator(operatorIndex));
+      }
+
+      if (i == 0) {
+        left = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
+        if (ctx.precedenceUnarySuffixExpression(i + 1) != null)
+          right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i + 1));
+      } else {
+        parentType = getPrecedenceBitwiseXorOperator((ctx.precedenceBitwiseXorOperator(operatorIndex - 1)));
+        parentLeft = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 2));
+        parentRight = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 1));
+        left = new BinaryOperator(parentType, parentLeft, parentRight);
+        right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
+      }
+
+      if (right != null) {
+        current = new BinaryOperator(type, left, right);
+      } else {
+        current = left;
+      }
+    }
+
+    return current;
+  }
+
+  public OpType getPrecedenceBitwiseXorOperator(HiveQLParser.PrecedenceBitwiseXorOperatorContext ctx) {
+    OpType type = null;
+    // TODO: It needs to consider how to support.
+
+    return type;
+  }
+
+  @Override
+  public Expr visitPrecedenceUnarySuffixExpression(HiveQLParser.PrecedenceUnarySuffixExpressionContext ctx) {
+    Expr current = visitPrecedenceUnaryPrefixExpression(ctx.precedenceUnaryPrefixExpression());
+
+    if (ctx.nullCondition() != null) {
+      boolean isNot = ctx.nullCondition().KW_NOT() == null ? false : true;
+      IsNullPredicate isNullPredicate = new IsNullPredicate(isNot, (ColumnReferenceExpr) current);
+      current = isNullPredicate;
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitPrecedenceUnaryPrefixExpression(HiveQLParser.PrecedenceUnaryPrefixExpressionContext ctx) {
+    Expr current = visitPrecedenceFieldExpression(ctx.precedenceFieldExpression());
+    return current;
+  }
+
+  @Override
+  public Expr visitNullCondition(HiveQLParser.NullConditionContext ctx) {
+    return new NullLiteral();
+  }
+
+  @Override
+  public Expr visitPrecedenceFieldExpression(HiveQLParser.PrecedenceFieldExpressionContext ctx) {
+    Expr current = visitAtomExpression(ctx.atomExpression());
+
+    if (ctx.DOT().size() > 0) {
+      ColumnReferenceExpr column = new ColumnReferenceExpr(ctx.identifier(0).getText());
+      ColumnReferenceExpr table = (ColumnReferenceExpr) current;
+      column.setQualifier(table.getName());
+      current = column;
+    }
+    return current;
+  }
+
+  @Override
+  public Expr visitAtomExpression(HiveQLParser.AtomExpressionContext ctx) {
+    Expr current = null;
+
+    if (ctx.KW_NULL() != null) {
+      current = new NullLiteral();
+    }
+    if (ctx.constant() != null) {
+      current = visitConstant(ctx.constant());
+    }
+    if (ctx.function() != null) {
+      current = visitFunction(ctx.function());
+    }
+    if (ctx.castExpression() != null) {
+      current = visitCastExpression(ctx.castExpression());
+    }
+    if (ctx.caseExpression() != null) {
+      current = visitCaseExpression(ctx.caseExpression());
+    }
+    if (ctx.whenExpression() != null) {
+      current = visitWhenExpression(ctx.whenExpression());
+    }
+    if (ctx.tableOrColumn() != null) {
+      current = visitTableOrColumn(ctx.tableOrColumn());
+    } else {
+      if (ctx.LPAREN() != null && ctx.RPAREN() != null) {
+        current = visitExpression(ctx.expression());
+      }
+    }
+
+    return current;
+  }
+
+  @Override
+  public Expr visitTableOrColumn(HiveQLParser.TableOrColumnContext ctx) {
+    ColumnReferenceExpr columnReferenceExpr = new ColumnReferenceExpr(ctx.identifier().getText());
+    return columnReferenceExpr;
+  }
+
+  @Override
+  public Expr visitIdentifier(HiveQLParser.IdentifierContext ctx) {
+    Expr current = null;
+
+    if (ctx.nonReserved() != null) {
+      current = new LiteralValue(ctx.nonReserved().getText(), LiteralValue.LiteralType.String);
+    } else {
+      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
+    }
+
+    return current;
+  }
+
+  @Override
+  public LiteralValue visitConstant(HiveQLParser.ConstantContext ctx) {
+    LiteralValue literalValue = null;
+
+    if (ctx.StringLiteral() != null) {
+      String value = ctx.StringLiteral().getText();
+      String strValue = "";
+      if ((value.startsWith("'") && value.endsWith("'")) || value.startsWith("\"") && value.endsWith("\"")) {
+        strValue = value.substring(1, value.length() - 1);
+      } else {
+        strValue = value;
+      }
+
+      literalValue = new LiteralValue(strValue, LiteralValue.LiteralType.String);
+    } else if (ctx.TinyintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.TinyintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.BigintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.BigintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Large_Integer);
+    } else if (ctx.DecimalLiteral() != null) {
+      literalValue = new LiteralValue(ctx.DecimalLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.Number() != null) {
+      try {
+        float floatValue = NumberUtils.createFloat(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Float);
+      } catch (NumberFormatException nf) {
+      }
+
+      // TODO: double type
+
+      try {
+        BigInteger bigIntegerVallue = NumberUtils.createBigInteger(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(),
+            LiteralValue.LiteralType.Unsigned_Large_Integer);
+      } catch (NumberFormatException nf) {
+      }
+
+      try {
+        int intValue = NumberUtils.createInteger(ctx.getText());
+        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Integer);
+      } catch (NumberFormatException nf) {
+      }
+
+    } else if (ctx.SmallintLiteral() != null) {
+      literalValue = new LiteralValue(ctx.SmallintLiteral().getSymbol().getText(),
+          LiteralValue.LiteralType.Unsigned_Integer);
+    } else if (ctx.booleanValue() != null) {
+      // TODO: boolean type
+    }
+
+    return literalValue;
+  }
+
+  @Override
+  public Expr visitFunction(HiveQLParser.FunctionContext ctx) {
+    Expr current = null;
+    String signature = ctx.functionName().getText();
+
+    boolean isDistinct = false;
+    if (ctx.getChild(2) != null) {
+      if (ctx.getChild(2) instanceof TerminalNodeImpl && ctx.getChild(2).getText().equalsIgnoreCase("DISTINCT")) {
+        isDistinct = true;
+      }
+    }
+
+    if (signature.equalsIgnoreCase("MIN")
+        || signature.equalsIgnoreCase("MAX")
+        || signature.equalsIgnoreCase("SUM")
+        || signature.equalsIgnoreCase("AVG")
+        || signature.equalsIgnoreCase("COUNT")
+        ) {
+      if (ctx.selectExpression().size() > 1) {
+        throw new RuntimeException("Exactly expected one argument.");
+      }
+
+      if (ctx.selectExpression().size() == 0) {
+        CountRowsFunctionExpr countRowsFunctionExpr = new CountRowsFunctionExpr();
+        current = countRowsFunctionExpr;
+      } else {
+        GeneralSetFunctionExpr setFunctionExpr = new GeneralSetFunctionExpr(signature, isDistinct,
+            visitSelectExpression(ctx.selectExpression(0)));
+        current = setFunctionExpr;
+      }
+    } else {
+      FunctionExpr functionExpr = new FunctionExpr(signature);
+      Expr[] params = new Expr[ctx.selectExpression().size()];
+      for (int i = 0; i < ctx.selectExpression().size(); i++) {
+        params[i] = visitSelectExpression(ctx.selectExpression(i));
+      }
+      functionExpr.setParams(params);
+      current = functionExpr;
+    }
+
+
+    return current;
+  }
+
+  /**
+   * This method parse CAST expression.
+   * This returns only expression field without casting type
+   * because Tajo doesn't provide CAST expression.
+   *
+   * @param ctx
+   * @return
+   */
+  @Override
+  public Expr visitCastExpression(HiveQLParser.CastExpressionContext ctx) {
+    return visitExpression(ctx.expression());
+  }
+
+  @Override
+  public Expr visitCaseExpression(HiveQLParser.CaseExpressionContext ctx) {
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+    Expr condition = null, result = null;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_WHEN) {
+          condition = null;
+          result = null;
+
+          if (ctx.getChild(i + 1) instanceof HiveQLParser.ExpressionContext) {
+            condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          }
+
+          if (ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+            result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+          }
+
+          if (condition != null && result != null) {
+            caseWhen.addWhen(condition, result);
+          }
+        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_ELSE) {
+          result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          caseWhen.setElseResult(result);
+        }
+      }
+    }
+
+    return caseWhen;
+  }
+
+  @Override
+  public Expr visitWhenExpression(HiveQLParser.WhenExpressionContext ctx) {
+    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
+    Expr condition = null, result = null;
+    for (int i = 1; i < ctx.getChildCount(); i++) {
+      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
+        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_WHEN) {
+          condition = null;
+          result = null;
+
+          if (ctx.getChild(i + 1) instanceof HiveQLParser.ExpressionContext) {
+            condition = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          }
+
+          if (ctx.getChild(i + 3) instanceof HiveQLParser.ExpressionContext) {
+            result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 3));
+          }
+
+          if (condition != null && result != null) {
+            caseWhen.addWhen(condition, result);
+          }
+        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveQLLexer.KW_ELSE) {
+          result = visitExpression((HiveQLParser.ExpressionContext) ctx.getChild(i + 1));
+          caseWhen.setElseResult(result);
+        }
+      }
+    }
+
+    return caseWhen;
+  }
+
+  @Override
+  public Aggregation visitGroupByClause(HiveQLParser.GroupByClauseContext ctx) {
+    Aggregation clause = new Aggregation();
+    if (ctx.groupByExpression().size() > 0) {
+      List<Expr> columns = new ArrayList<Expr>();
+      List<Expr> functions = new ArrayList<Expr>();
+
+      for (int i = 0; i < ctx.groupByExpression().size(); i++) {
+        Expr expr = visitGroupByExpression(ctx.groupByExpression(i));
+
+        if (expr instanceof ColumnReferenceExpr) {
+          columns.add(expr);
+        } else if (expr instanceof FunctionExpr) {
+          functions.add(expr);
+        } else {
+          //TODO: find another case.
+        }
+      }
+
+      Aggregation.GroupElement[] groups = null;
+
+      if (columns.size() > 0) {
+        groups = new Aggregation.GroupElement[1 + functions.size()];
+      } else {
+        groups = new Aggregation.GroupElement[functions.size()];
+      }
+
+      int index = 0;
+      if (columns.size() > 0) {
+        index = 0;
+        ColumnReferenceExpr[] columnReferenceExprs = new ColumnReferenceExpr[columns.size()];
+        for (int i = 0; i < columns.size(); i++) {
+          ColumnReferenceExpr expr = (ColumnReferenceExpr) columns.get(i);
+          columnReferenceExprs[i] = expr;
+        }
+        groups[index] = new Aggregation.GroupElement(Aggregation.GroupType.OrdinaryGroup, columnReferenceExprs);
+      }
+
+      if (functions.size() > 0) {
+        if (columns.size() == 0) {
+          index = 0;
+        } else {
+          index = 1;
+        }
+
+        for (int i = 0; i < functions.size(); i++) {
+          FunctionExpr function = (FunctionExpr) functions.get(i);
+
+          Expr[] params = function.getParams();
+          ColumnReferenceExpr[] column = new ColumnReferenceExpr[params.length];
+          for (int j = 0; j < column.length; j++)
+            column[j] = (ColumnReferenceExpr) params[j];
+
+          if (function.getSignature().equalsIgnoreCase("ROLLUP"))
+            groups[i + index] = new Aggregation.GroupElement(Aggregation.GroupType.Rollup, column);
+          else if (function.getSignature().equalsIgnoreCase("CUBE"))
+            groups[i + index] = new Aggregation.GroupElement(Aggregation.GroupType.Cube, column);
+          else
+            throw new RuntimeException("Unexpected aggregation function.");
+        }
+      }
+
+      clause.setGroups(groups);
+    }
+
+    //TODO: grouping set expression
+    return clause;
+  }
+
+  @Override
+  public Sort visitOrderByClause(HiveQLParser.OrderByClauseContext ctx) {
+    Sort clause = null;
+    Sort.SortSpec[] specs = null;
+
+    if (ctx.columnRefOrder().size() > 0) {
+      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
+      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
+        ColumnReferenceExpr column = (ColumnReferenceExpr) visitExpression(ctx.columnRefOrder().get(i).expression());
+        specs[i] = new Sort.SortSpec(column);
+        if (ctx.columnRefOrder(i).KW_DESC() != null) {
+          specs[i].setDescending();
+        }
+      }
+      clause = new Sort(specs);
+    }
+    return clause;
+
+  }
+
+  @Override
+  public Expr visitHavingClause(HiveQLParser.HavingClauseContext ctx) {
+    return visitHavingCondition(ctx.havingCondition());
+  }
+
+  @Override
+  public Expr visitClusterByClause(HiveQLParser.ClusterByClauseContext ctx) {
+    // TODO: It needs to consider how to support.
+    return null;
+  }
+
+  @Override
+  public Expr visitDistributeByClause(HiveQLParser.DistributeByClauseContext ctx) {
+    // TODO: It needs to consider how to support.
+
+    return null;
+  }
+
+  @Override
+  public Sort visitSortByClause(HiveQLParser.SortByClauseContext ctx) {
+    Sort clause = null;
+    Sort.SortSpec[] specs = null;
+
+    if (ctx.columnRefOrder().size() > 0) {
+      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
+      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
+        ColumnReferenceExpr column = (ColumnReferenceExpr) visitColumnRefOrder(ctx.columnRefOrder(i));
+        specs[i] = new Sort.SortSpec(column);
+
+        if (ctx.columnRefOrder(i).KW_DESC() != null) {
+          specs[i].setDescending();
+        }
+      }
+      clause = new Sort(specs);
+    }
+
+    return clause;
+  }
+
+  @Override
+  public Limit visitLimitClause(HiveQLParser.LimitClauseContext ctx) {
+    LiteralValue expr = new LiteralValue(ctx.Number().getText(), LiteralValue.LiteralType.Unsigned_Integer);
+    Limit limit = new Limit(expr);
+    return limit;
+  }
+
+  @Override
+  public Expr visitWindow_clause(HiveQLParser.Window_clauseContext ctx) {
+    // TODO: It needs to consider how to support.
+    return null;
+  }
+
+  @Override
+  public Insert visitInsertClause(HiveQLParser.InsertClauseContext ctx) {
+    Insert insert = new Insert();
+    if (ctx.KW_OVERWRITE() != null)
+      insert.setOverwrite();
+
+    if (ctx.tableOrPartition() != null) {
+      HiveQLParser.TableOrPartitionContext partitionContext = ctx.tableOrPartition();
+      if (partitionContext.tableName() != null) {
+        insert.setTableName(ctx.tableOrPartition().tableName().getText());
+      }
+    }
+
+    if (ctx.destination() != null) {
+      HiveQLParser.DestinationContext destination = ctx.destination();
+      if (destination.KW_DIRECTORY() != null) {
+        String location = destination.StringLiteral().getText();
+        location = location.replaceAll("\\'", "");
+        insert.setLocation(location);
+      } else if (destination.KW_TABLE() != null) {
+        if (destination.tableOrPartition() != null) {
+          HiveQLParser.TableOrPartitionContext partitionContext = destination.tableOrPartition();
+          if (partitionContext.tableName() != null) {
+            insert.setTableName(partitionContext.tableName().getText());
+          }
+        }
+
+        if (destination.tableFileFormat() != null) {
+          if (destination.tableFileFormat().KW_RCFILE() != null) {
+            insert.setStorageType("rcfile");
+          } else if (destination.tableFileFormat().KW_TEXTFILE() != null) {
+            insert.setStorageType("csv");
+          }
+
+        }
+      }
+    }
+
+    return insert;
+  }
+
+  @Override
+  public Expr visitCreateTableStatement(HiveQLParser.CreateTableStatementContext ctx) {
+    CreateTable createTable = null;
+    Map<String, String> params = new HashMap<String, String>();
+
+    if (ctx.name != null) {
+      createTable = new CreateTable(ctx.name.getText());
+      if (ctx.KW_EXTERNAL() != null) {
+        createTable.setExternal();
+      }
+
+      if (ctx.tableFileFormat() != null) {
+        if (ctx.tableFileFormat().KW_RCFILE() != null) {
+          createTable.setStorageType("rcfile");
+        } else if (ctx.tableFileFormat().KW_TEXTFILE() != null) {
+          createTable.setStorageType("csv");
+        }
+      }
+
+      if (ctx.tableRowFormat() != null) {
+        if (ctx.tableRowFormat().rowFormatDelimited() != null) {
+          String delimiter = ctx.tableRowFormat().rowFormatDelimited().tableRowFormatFieldIdentifier().getChild(3)
+              .getText().replaceAll("'", "");
+          params.put("csvfile.delimiter", SQLAnalyzer.escapeDelimiter(delimiter));
+        }
+      }
+
+      if (ctx.tableLocation() != null) {
+        String location = ctx.tableLocation().StringLiteral().getText();
+        location = location.replaceAll("'", "");
+        createTable.setLocation(location);
+
+      }
+
+      if (ctx.columnNameTypeList() != null) {
+        List<HiveQLParser.ColumnNameTypeContext> list = ctx.columnNameTypeList().columnNameType();
+
+        CreateTable.ColumnDefinition[] columns = new CreateTable.ColumnDefinition[list.size()];
+
+        for (int i = 0; i < list.size(); i++) {
+          HiveQLParser.ColumnNameTypeContext eachColumn = list.get(i);
+          String type = null;
+          if (eachColumn.colType().type() != null) {
+            if (eachColumn.colType().type().primitiveType() != null) {
+              HiveQLParser.PrimitiveTypeContext primitiveType = eachColumn.colType().type().primitiveType();
+
+              if (primitiveType.KW_STRING() != null) {
+                type = TajoDataTypes.Type.TEXT.name();
+              } else if (primitiveType.KW_TINYINT() != null) {
+                type = TajoDataTypes.Type.INT1.name();
+              } else if (primitiveType.KW_SMALLINT() != null) {
+                type = TajoDataTypes.Type.INT2.name();
+              } else if (primitiveType.KW_INT() != null) {
+                type = TajoDataTypes.Type.INT4.name();
+              } else if (primitiveType.KW_BIGINT() != null) {
+                type = TajoDataTypes.Type.INT8.name();
+              } else if (primitiveType.KW_FLOAT() != null) {
+                type = TajoDataTypes.Type.FLOAT4.name();
+              } else if (primitiveType.KW_DOUBLE() != null) {
+                type = TajoDataTypes.Type.FLOAT8.name();
+              } else if (primitiveType.KW_DECIMAL() != null) {
+                type = TajoDataTypes.Type.DECIMAL.name();
+              } else if (primitiveType.KW_BOOLEAN() != null) {
+                type = TajoDataTypes.Type.BOOLEAN.name();
+              } else if (primitiveType.KW_DATE() != null) {
+                type = TajoDataTypes.Type.DATE.name();
+              } else if (primitiveType.KW_DATETIME() != null) {
+                //TODO
+              } else if (primitiveType.KW_TIMESTAMP() != null) {
+                type = TajoDataTypes.Type.TIMESTAMP.name();
+              }
+
+              columns[i] = new CreateTable.ColumnDefinition(eachColumn.colName.Identifier().getText(), type);
+            }
+          }
+        }
+        if (columns != null) {
+          createTable.setTableElements(columns);
+        }
+
+        if (!params.isEmpty()) {
+          createTable.setParams(params);
+        }
+      }
+    }
+
+    return createTable;
+  }
+
+  @Override
+  public Expr visitDropTableStatement(HiveQLParser.DropTableStatementContext ctx) {
+    DropTable dropTable = new DropTable(ctx.tableName().getText(), false);
+    return dropTable;
+  }
+
+  /**
+   * 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 characteres. 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 funtion
+   * 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 ANTLRInputStream {
+
+    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);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 325f7db..d7ded93 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -37,7 +37,7 @@ import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.exception.IllegalQueryStatusException;
 import org.apache.tajo.engine.exception.VerifyException;
-import org.apache.tajo.engine.parser.HiveConverter;
+import org.apache.tajo.engine.parser.HiveQLAnalyzer;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.*;
 import org.apache.tajo.engine.planner.logical.*;
@@ -64,7 +64,7 @@ public class GlobalEngine extends AbstractService {
   private final AbstractStorageManager sm;
 
   private SQLAnalyzer analyzer;
-  private HiveConverter converter;
+  private HiveQLAnalyzer converter;
   private CatalogService catalog;
   private PreLogicalPlanVerifier preVerifier;
   private LogicalPlanner planner;
@@ -82,7 +82,7 @@ public class GlobalEngine extends AbstractService {
   public void start() {
     try  {
       analyzer = new SQLAnalyzer();
-      converter = new HiveConverter();
+      converter = new HiveQLAnalyzer();
       preVerifier = new PreLogicalPlanVerifier(context.getCatalog());
       planner = new LogicalPlanner(context.getCatalog());
       optimizer = new LogicalOptimizer(context.getConf());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 2c3ddfe..67feaf0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -34,7 +34,7 @@ import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.parser.HiveConverter;
+import org.apache.tajo.engine.parser.HiveQLAnalyzer;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.*;
 import org.apache.tajo.engine.planner.global.MasterPlan;
@@ -297,8 +297,8 @@ public class QueryMasterTask extends CompositeService {
     LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
     Expr expr;
     if (queryContext.isHiveQueryMode()) {
-      HiveConverter hiveConverter = new HiveConverter();
-      expr = hiveConverter.parse(sql);
+      HiveQLAnalyzer HiveQLAnalyzer = new HiveQLAnalyzer();
+      expr = HiveQLAnalyzer.parse(sql);
     } else {
       SQLAnalyzer analyzer = new SQLAnalyzer();
       expr = analyzer.parse(sql);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
deleted file mode 100644
index e445c92..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveConverter.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.parser;
-
-import org.antlr.v4.runtime.ANTLRInputStream;
-import org.antlr.v4.runtime.CommonTokenStream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.algebra.Expr;
-import org.apache.tajo.engine.parser.SQLParser.SqlContext;
-import org.apache.tajo.util.FileUtil;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestHiveConverter {
-  private static final Log LOG = LogFactory.getLog(TestHiveConverter.class.getName());
-
-  public static Expr parseQuery(String sql) {
-    ANTLRInputStream input = new ANTLRInputStream(sql);
-    SQLLexer lexer = new SQLLexer(input);
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
-    SQLParser parser = new SQLParser(tokens);
-    parser.setBuildParseTree(true);
-    SQLAnalyzer visitor = new SQLAnalyzer();
-    SqlContext context = parser.sql();
-    return visitor.visitSql(context);
-  }
-
-  public static Expr parseHiveQL(String sql) {
-    HiveConverter converter = new HiveConverter();
-    return converter.parse(sql);
-  }
-
-  public static String getMethodName(int depth) {
-    final StackTraceElement[] ste = Thread.currentThread().getStackTrace();
-    return ste[depth].getMethodName();
-  }
-
-  public static void compareJsonResult(Expr expr, Expr hiveExpr) throws IOException {
-    if (expr != null && hiveExpr != null) {
-      if (!expr.toJson().equals(hiveExpr.toJson())) {
-        LOG.info("### Tajo Parse Result ### \n" + expr.toJson());
-        LOG.info("### Hive Parse Result ### \n" + hiveExpr.toJson());
-        throw new IOException(getMethodName(3));
-      }
-    } else {
-      LOG.info("### Tajo Parse Result ### \n" + expr.toJson());
-      LOG.info("### Hive Parse Result ### \n" + hiveExpr.toJson());
-      throw new IOException(getMethodName(3));
-    }
-  }
-
-  @Test
-  public void testSelect1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_1.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect3() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_3.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect4() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_4.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect5() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_5.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect7() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_7.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect8() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_8.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect9() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_9.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_9.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect10() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_10.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_10.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  //@Test
-  public void testSelect11() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_11.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_11.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect12() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_12.hiveql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect13() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_13.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_13.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testSelect14() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_14.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testAsterisk1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_1.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testAsterisk2() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_2.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testAsterisk3() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_3.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testAsterisk4() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_4.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testGroupby1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/groupby_1.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin2() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_2.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin5() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_5.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin6() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_6.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin7() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_7.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  //@Test
-  public void testJoin9() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_9.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    assertEquals(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin12() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_12.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    assertEquals(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin13() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_13.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin14() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_14.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testJoin15() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_15.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_15.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testUnion1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/union_1.hiveql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testInsert1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_into_select_1.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-
-  }
-
-  @Test
-  public void testInsert2() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_2.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_2.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testCreate1() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_1.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_1.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testCreate2() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_2.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_2.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testCreate11() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_11.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_11.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testCreate12() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_12.sql"));
-    Expr expr = parseQuery(sql);
-    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_12.hiveql"));
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-
-  @Test
-  public void testDrop() throws IOException {
-    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/drop_table.sql"));
-    Expr expr = parseQuery(sql);
-    Expr hiveExpr = parseHiveQL(sql);
-    compareJsonResult(expr, hiveExpr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
new file mode 100644
index 0000000..dcdd862
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestHiveQLAnalyzer.java
@@ -0,0 +1,355 @@
+/**
+ * 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.tajo.engine.parser;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.engine.parser.SQLParser.SqlContext;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveQLAnalyzer {
+  private static final Log LOG = LogFactory.getLog(TestHiveQLAnalyzer.class.getName());
+
+  public static Expr parseQuery(String sql) {
+    ANTLRInputStream input = new ANTLRInputStream(sql);
+    SQLLexer lexer = new SQLLexer(input);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+    SQLParser parser = new SQLParser(tokens);
+    parser.setBuildParseTree(true);
+    SQLAnalyzer visitor = new SQLAnalyzer();
+    SqlContext context = parser.sql();
+    return visitor.visitSql(context);
+  }
+
+  public static Expr parseHiveQL(String sql) {
+    HiveQLAnalyzer converter = new HiveQLAnalyzer();
+    return converter.parse(sql);
+  }
+
+  public static String getMethodName(int depth) {
+    final StackTraceElement[] ste = Thread.currentThread().getStackTrace();
+    return ste[depth].getMethodName();
+  }
+
+  public static void compareJsonResult(Expr expr, Expr hiveExpr) throws IOException {
+    if (expr != null && hiveExpr != null) {
+      if (!expr.toJson().equals(hiveExpr.toJson())) {
+        LOG.info("### Tajo Parse Result ### \n" + expr.toJson());
+        LOG.info("### Hive Parse Result ### \n" + hiveExpr.toJson());
+        throw new IOException(getMethodName(3));
+      }
+    } else {
+      LOG.info("### Tajo Parse Result ### \n" + expr.toJson());
+      LOG.info("### Hive Parse Result ### \n" + hiveExpr.toJson());
+      throw new IOException(getMethodName(3));
+    }
+  }
+
+  @Test
+  public void testSelect1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_1.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_3.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_4.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect5() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_5.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect7() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_7.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect8() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_8.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect9() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_9.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_9.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect10() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_10.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_10.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  //@Test
+  public void testSelect11() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_11.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_11.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect12() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_12.hiveql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect13() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_13.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_13.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testSelect14() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/select_14.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_1.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_2.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk3() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_3.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testAsterisk4() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/asterisk_4.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testGroupby1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/groupby_1.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_2.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin5() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_5.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin6() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_6.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin7() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_7.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  //@Test
+  public void testJoin9() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_9.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    assertEquals(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin12() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_12.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    assertEquals(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin13() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_13.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin14() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_14.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testJoin15() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_15.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/join_15.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testUnion1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/union_1.hiveql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testInsert1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_into_select_1.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+
+  }
+
+  @Test
+  public void testInsert2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_2.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/insert_overwrite_into_select_2.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testCreate1() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_1.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_1.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testCreate2() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_2.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_2.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testCreate11() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_11.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_11.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testCreate12() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_12.sql"));
+    Expr expr = parseQuery(sql);
+    sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/create_table_12.hiveql"));
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+
+  @Test
+  public void testDrop() throws IOException {
+    String sql = FileUtil.readTextFile(new File("src/test/resources/queries/default/drop_table.sql"));
+    Expr expr = parseQuery(sql);
+    Expr hiveExpr = parseHiveQL(sql);
+    compareJsonResult(expr, hiveExpr);
+  }
+}


[4/5] git commit: TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)

Posted by hy...@apache.org.
TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)


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

Branch: refs/heads/branch-0.8.0
Commit: e8da943afa1a631e8521593be1bc7a2c50bb2c63
Parents: 5fb959b
Author: blrunner <jh...@gruter.com>
Authored: Fri Feb 21 18:41:33 2014 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Fri Feb 21 18:41:33 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 .../org/apache/tajo/engine/parser/HiveLexer.g4  |  392 ---
 .../org/apache/tajo/engine/parser/HiveParser.g4 | 2512 ------------------
 .../apache/tajo/engine/parser/HiveQLLexer.g4    |  392 +++
 .../apache/tajo/engine/parser/HiveQLParser.g4   | 2067 ++++++++++++++
 .../tajo/engine/parser/HiveConverter.java       | 1568 -----------
 .../tajo/engine/parser/HiveQLAnalyzer.java      | 1568 +++++++++++
 .../org/apache/tajo/master/GlobalEngine.java    |    6 +-
 .../master/querymaster/QueryMasterTask.java     |    6 +-
 .../tajo/engine/parser/TestHiveConverter.java   |  355 ---
 .../tajo/engine/parser/TestHiveQLAnalyzer.java  |  355 +++
 11 files changed, 4390 insertions(+), 4833 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index aca69b0..b5a2efd 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -263,6 +263,8 @@ Release 0.8.0 - unreleased
 
   BUG FIXES
 
+    TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)
+
     TAJO-575: Worker's env.jsp has wrong URL which go to worker's index.jsp. (hyoungjunkim via jaehwa)
 
     TAJO-609: PlannerUtil::getRelationLineage ignores PartitionedTableScanNode.

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveLexer.g4
deleted file mode 100644
index 6595655..0000000
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveLexer.g4
+++ /dev/null
@@ -1,392 +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 HiveLexer;
-
-
-// Keywords
-
-KW_TRUE : 'TRUE';
-KW_FALSE : 'FALSE';
-KW_ALL : 'ALL';
-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_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_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_IGNORE: 'IGNORE';
-KW_PROTECTION: 'PROTECTION';
-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_DECIMAL: 'DECIMAL';
-KW_STRING: 'STRING';
-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_SEQUENCEFILE: 'SEQUENCEFILE';
-KW_TEXTFILE: 'TEXTFILE';
-KW_RCFILE: 'RCFILE';
-KW_ORCFILE: 'ORC';
-KW_INPUTFORMAT: 'INPUTFORMAT';
-KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
-KW_INPUTDRIVER: 'INPUTDRIVER';
-KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
-KW_OFFLINE: 'OFFLINE';
-KW_ENABLE: 'ENABLE';
-KW_DISABLE: 'DISABLE';
-KW_READONLY: 'READONLY';
-KW_NO_DROP: 'NO_DROP';
-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_EXPLAIN: 'EXPLAIN';
-KW_EXTENDED: 'EXTENDED';
-KW_FORMATTED: 'FORMATTED';
-KW_PRETTY: 'PRETTY';
-KW_DEPENDENCY: 'DEPENDENCY';
-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_CASE: 'CASE';
-KW_WHEN: 'WHEN';
-KW_THEN: 'THEN';
-KW_ELSE: 'ELSE';
-KW_END: 'END';
-KW_MAPJOIN: 'MAPJOIN';
-KW_STREAMTABLE: 'STREAMTABLE';
-KW_HOLD_DDLTIME: 'HOLD_DDLTIME';
-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_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_INNER: 'INNER';
-
-// 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)?
-    ;
-    
-Identifier
-    :
-    (Letter | Digit) (Letter | Digit | '_')*
-    | '`' RegexComponent+ '`'
-    ;
-
-CharSetName
-    :
-    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
-    ;
-
-WS  :
-(' '|'\r'|'\t'|'\n') -> skip
-    ;
-
-COMMENT
-  : '--' (~('\n'|'\r'))*  -> skip
-  ;
-
-
-

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveParser.g4
deleted file mode 100644
index d3829a4..0000000
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveParser.g4
+++ /dev/null
@@ -1,2512 +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 HiveParser;
-
-options
-{
-tokenVocab=HiveLexer;
-language=Java;
-}
-
-tokens {
-TOK_INSERT;
-TOK_QUERY;
-TOK_SELECT;
-TOK_SELECTDI;
-TOK_SELEXPR;
-TOK_FROM;
-TOK_TAB;
-TOK_PARTSPEC;
-TOK_PARTVAL;
-TOK_DIR;
-TOK_LOCAL_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_UNION;
-TOK_JOIN;
-TOK_LEFTOUTERJOIN;
-TOK_RIGHTOUTERJOIN;
-TOK_FULLOUTERJOIN;
-TOK_UNIQUEJOIN;
-TOK_CROSSJOIN;
-TOK_LOAD;
-TOK_EXPORT;
-TOK_IMPORT;
-TOK_NULL;
-TOK_ISNULL;
-TOK_ISNOTNULL;
-TOK_TINYINT;
-TOK_SMALLINT;
-TOK_INT;
-TOK_BIGINT;
-TOK_BOOLEAN;
-TOK_FLOAT;
-TOK_DOUBLE;
-TOK_DATE;
-TOK_DATETIME;
-TOK_TIMESTAMP;
-TOK_STRING;
-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_DROPTABLE_PROPERTIES;
-TOK_LIKETABLE;
-TOK_DESCTABLE;
-TOK_DESCFUNCTION;
-TOK_ALTERTABLE_PARTITION;
-TOK_ALTERTABLE_RENAME;
-TOK_ALTERTABLE_ADDCOLS;
-TOK_ALTERTABLE_RENAMECOL;
-TOK_ALTERTABLE_RENAMEPART;
-TOK_ALTERTABLE_REPLACECOLS;
-TOK_ALTERTABLE_ADDPARTS;
-TOK_ALTERTABLE_DROPPARTS;
-TOK_ALTERTABLE_ALTERPARTS;
-TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE;
-TOK_ALTERTABLE_TOUCH;
-TOK_ALTERTABLE_ARCHIVE;
-TOK_ALTERTABLE_UNARCHIVE;
-TOK_ALTERTABLE_SERDEPROPERTIES;
-TOK_ALTERTABLE_SERIALIZER;
-TOK_TABLE_PARTITION;
-TOK_ALTERTABLE_FILEFORMAT;
-TOK_ALTERTABLE_LOCATION;
-TOK_ALTERTABLE_PROPERTIES;
-TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
-TOK_ALTERINDEX_REBUILD;
-TOK_ALTERINDEX_PROPERTIES;
-TOK_MSCK;
-TOK_SHOWDATABASES;
-TOK_SHOWTABLES;
-TOK_SHOWCOLUMNS;
-TOK_SHOWFUNCTIONS;
-TOK_SHOWPARTITIONS;
-TOK_SHOW_CREATETABLE;
-TOK_SHOW_TABLESTATUS;
-TOK_SHOW_TBLPROPERTIES;
-TOK_SHOWLOCKS;
-TOK_LOCKTABLE;
-TOK_UNLOCKTABLE;
-TOK_SWITCHDATABASE;
-TOK_DROPDATABASE;
-TOK_DROPTABLE;
-TOK_DATABASECOMMENT;
-TOK_TABCOLLIST;
-TOK_TABCOL;
-TOK_TABLECOMMENT;
-TOK_TABLEPARTCOLS;
-TOK_TABLEBUCKETS;
-TOK_TABLEROWFORMAT;
-TOK_TABLEROWFORMATFIELD;
-TOK_TABLEROWFORMATCOLLITEMS;
-TOK_TABLEROWFORMATMAPKEYS;
-TOK_TABLEROWFORMATLINES;
-TOK_TBLORCFILE;
-TOK_TBLSEQUENCEFILE;
-TOK_TBLTEXTFILE;
-TOK_TBLRCFILE;
-TOK_TABLEFILEFORMAT;
-TOK_FILEFORMAT_GENERIC;
-TOK_OFFLINE;
-TOK_ENABLE;
-TOK_DISABLE;
-TOK_READONLY;
-TOK_NO_DROP;
-TOK_STORAGEHANDLER;
-TOK_ALTERTABLE_CLUSTER_SORT;
-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_CREATEVIEW;
-TOK_DROPVIEW;
-TOK_ALTERVIEW_AS;
-TOK_ALTERVIEW_PROPERTIES;
-TOK_DROPVIEW_PROPERTIES;
-TOK_ALTERVIEW_ADDPARTS;
-TOK_ALTERVIEW_DROPPARTS;
-TOK_ALTERVIEW_RENAME;
-TOK_VIEWPARTCOLS;
-TOK_EXPLAIN;
-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_HOLD_DDLTIME;
-TOK_HINTARGLIST;
-TOK_USERSCRIPTCOLNAMES;
-TOK_USERSCRIPTCOLSCHEMA;
-TOK_RECORDREADER;
-TOK_RECORDWRITER;
-TOK_LEFTSEMIJOIN;
-TOK_LATERAL_VIEW;
-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_GRANT_WITH_OPTION;
-TOK_PRIV_ALL;
-TOK_PRIV_ALTER_METADATA;
-TOK_PRIV_ALTER_DATA;
-TOK_PRIV_DROP;
-TOK_PRIV_INDEX;
-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_SHOWINDEXES;
-TOK_INDEXCOMMENT;
-TOK_DESCDATABASE;
-TOK_DATABASEPROPERTIES;
-TOK_DATABASELOCATION;
-TOK_DBPROPLIST;
-TOK_ALTERDATABASE_PROPERTIES;
-TOK_ALTERTABLE_ALTERPARTS_MERGEFILES;
-TOK_TABNAME;
-TOK_TABSRC;
-TOK_RESTRICT;
-TOK_CASCADE;
-TOK_TABLESKEWED;
-TOK_TABCOLVALUE;
-TOK_TABCOLVALUE_PAIR;
-TOK_TABCOLVALUES;
-TOK_ALTERTABLE_SKEWED;
-TOK_ALTERTBLPART_SKEWED_LOCATION;
-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_IGNOREPROTECTION;
-}
-
-
-
-
-// Package headers
-@header {
-
-import java.util.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-}
-
-
-@members {
-  private static final Log LOG = LogFactory.getLog(HiveParser.class.getName());
-
-  Stack msgs = new Stack<String>();
-
-  private static HashMap<String, String> xlateMap;
-  static {
-    xlateMap = new HashMap<String, String>();
-
-    // Keywords
-    xlateMap.put("KW_TRUE", "TRUE");
-    xlateMap.put("KW_FALSE", "FALSE");
-    xlateMap.put("KW_ALL", "ALL");
-    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_REANME", "REANME");
-    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\$");
-
-    // 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;
-  }
-
-}
-
-@rulecatch {
-catch (RecognitionException e) {
- reportError(e);
-  throw e;
-}
-}
-
-//----------------------- Rules for parsing selectClause -----------------------------
-// select a,b,c ...
-selectClause
-@init { msgs.push("select clause"); }
-@after { msgs.pop(); }
-    :
-    KW_SELECT hintClause? (((KW_ALL | dist=KW_DISTINCT)? selectList)
-                          | (transform=KW_TRANSFORM selectTrfmClause))
-    |
-    trfmClause
-    ;
-
-selectList
-@init { msgs.push("select list"); }
-@after { msgs.pop(); }
-    :
-    selectItem ( COMMA  selectItem )* 
-    ;
-
-selectTrfmClause
-@init { msgs.push("transform clause"); }
-@after { msgs.pop(); }
-    :
-    LPAREN selectExpressionList RPAREN
-    inSerde=rowFormat inRec=recordWriter
-    KW_USING StringLiteral
-    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
-    outSerde=rowFormat outRec=recordReader
-    ;
-
-hintClause
-@init { msgs.push("hint clause"); }
-@after { msgs.pop(); }
-    :
-    DIVIDE STAR PLUS hintList STAR DIVIDE 
-    ;
-
-hintList
-@init { msgs.push("hint list"); }
-@after { msgs.pop(); }
-    :
-    hintItem (COMMA hintItem)* 
-    ;
-
-hintItem
-@init { msgs.push("hint item"); }
-@after { msgs.pop(); }
-    :
-    hintName (LPAREN hintArgs RPAREN)? 
-    ;
-
-hintName
-@init { msgs.push("hint name"); }
-@after { msgs.pop(); }
-    :
-    KW_MAPJOIN 
-    | KW_STREAMTABLE 
-    | KW_HOLD_DDLTIME 
-    ;
-
-hintArgs
-@init { msgs.push("hint arguments"); }
-@after { msgs.pop(); }
-    :
-    hintArgName (COMMA hintArgName)* 
-    ;
-
-hintArgName
-@init { msgs.push("hint argument name"); }
-@after { msgs.pop(); }
-    :
-    identifier
-    ;
-
-selectItem
-@init { msgs.push("selection target"); }
-@after { msgs.pop(); }
-    :
-    ( selectExpression (KW_OVER ws=window_specification )?
-      ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?
-    ) 
-    ;
-
-trfmClause
-@init { msgs.push("transform clause"); }
-@after { msgs.pop(); }
-    :
-    (   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
-    ;
-
-selectExpression
-@init { msgs.push("select expression"); }
-@after { msgs.pop(); }
-    :
-    expression | tableAllColumns
-    ;
-
-selectExpressionList
-@init { msgs.push("select expression list"); }
-@after { msgs.pop(); }
-    :
-    selectExpression (COMMA selectExpression)* 
-    ;
-
-
-//---------------------- Rules for windowing clauses -------------------------------
-window_clause 
-@init { msgs.push("window_clause"); }
-@after { msgs.pop(); } 
-:
-  KW_WINDOW window_defn (COMMA window_defn)* 
-;  
-
-window_defn 
-@init { msgs.push("window_defn"); }
-@after { msgs.pop(); } 
-:
-  Identifier KW_AS window_specification 
-;  
-
-window_specification 
-@init { msgs.push("window_specification"); }
-@after { msgs.pop(); } 
-:
-  (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) 
-;
-
-window_frame :
- window_range_expression |
- window_value_expression
-;
-
-window_range_expression 
-@init { msgs.push("window_range_expression"); }
-@after { msgs.pop(); } 
-:
- KW_ROWS sb=window_frame_start_boundary 
- KW_ROWS KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
-;
-
-window_value_expression 
-@init { msgs.push("window_value_expression"); }
-@after { msgs.pop(); } 
-:
- KW_RANGE sb=window_frame_start_boundary 
- KW_RANGE KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
-;
-
-window_frame_start_boundary 
-@init { msgs.push("windowframestartboundary"); }
-@after { msgs.pop(); } 
-:
-  KW_UNBOUNDED KW_PRECEDING  
-  KW_CURRENT KW_ROW  
-  Number KW_PRECEDING 
-;
-
-window_frame_boundary 
-@init { msgs.push("windowframeboundary"); }
-@after { msgs.pop(); } 
-:
-  KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING)  
-  KW_CURRENT KW_ROW  
-  Number (d=KW_PRECEDING | d=KW_FOLLOWING ) 
-;   
-
-
-tableAllColumns
-    : STAR
-    | tableName DOT STAR
-    ;
-
-// (table|column)
-tableOrColumn
-@init { msgs.push("table or column identifier"); }
-@after { msgs.pop(); }
-    :
-    identifier 
-    ;
-
-expressionList
-@init { msgs.push("expression list"); }
-@after { msgs.pop(); }
-    :
-    expression (COMMA expression)* 
-    ;
-
-aliasList
-@init { msgs.push("alias list"); }
-@after { msgs.pop(); }
-    :
-    identifier (COMMA identifier)* 
-    ;
-
-
-//----------------------- Rules for parsing fromClause ------------------------------
-// from [col1, col2, col3] table1, [col4, col5] table2
-fromClause
-@init { msgs.push("from clause"); }
-@after { msgs.pop(); }
-    :
-    KW_FROM joinSource 
-    ;
-
-joinSource
-@init { msgs.push("join source"); }
-@after { msgs.pop(); }
-    : fromSource ( joinToken fromSource (KW_ON expression)? 
-    )*
-    | uniqueJoinToken uniqueJoinSource (COMMA uniqueJoinSource)+
-    ;
-
-uniqueJoinSource
-@init { msgs.push("join source"); }
-@after { msgs.pop(); }
-    : KW_PRESERVE? fromSource uniqueJoinExpr
-    ;
-
-uniqueJoinExpr
-@init { msgs.push("unique join expression list"); }
-@after { msgs.pop(); }
-    : LPAREN e1+=expression (COMMA e1+=expression)* RPAREN
-    ;
-
-uniqueJoinToken
-@init { msgs.push("unique join"); }
-@after { msgs.pop(); }
-    : KW_UNIQUEJOIN 
-;
-
-joinToken
-@init { msgs.push("join type specifier"); }
-@after { msgs.pop(); }
-    :
-      KW_JOIN                    
-    | KW_INNER  KW_JOIN            
-    | KW_CROSS KW_JOIN            
-    | KW_LEFT  KW_OUTER KW_JOIN   
-    | KW_RIGHT KW_OUTER KW_JOIN  
-    | KW_FULL  KW_OUTER KW_JOIN  
-    | KW_LEFT  KW_SEMI  KW_JOIN  
-    ;
-
-lateralView
-@init {msgs.push("lateral view"); }
-@after {msgs.pop(); }
-	:
-	KW_LATERAL KW_VIEW function tableAlias KW_AS identifier (COMMA identifier)* 
-	;
-
-tableAlias
-@init {msgs.push("table alias"); }
-@after {msgs.pop(); }
-    :
-    identifier 
-    ;
-
-fromSource
-@init { msgs.push("from source"); }
-@after { msgs.pop(); }
-    :
-    ((Identifier LPAREN) | tableSource | subQuerySource) (lateralView)*
-    ;
-
-tableBucketSample
-@init { msgs.push("table bucket sample specification"); }
-@after { msgs.pop(); }
-    :
-    KW_TABLESAMPLE LPAREN KW_BUCKET (numerator=Number) KW_OUT KW_OF (denominator=Number) (KW_ON expr+=expression (COMMA expr+=expression)*)? RPAREN 
-    ;
-
-splitSample
-@init { msgs.push("table split sample specification"); }
-@after { msgs.pop(); }
-    :
-    KW_TABLESAMPLE LPAREN  (numerator=Number) (percent=KW_PERCENT|KW_ROWS) RPAREN
-    |
-    KW_TABLESAMPLE LPAREN  (numerator=ByteLengthLiteral) RPAREN
-    ;
-
-tableSample
-@init { msgs.push("table sample specification"); }
-@after { msgs.pop(); }
-    :
-    tableBucketSample |
-    splitSample
-    ;
-
-tableSource
-@init { msgs.push("table source"); }
-@after { msgs.pop(); }
-    : tabname=tableName (ts=tableSample)? (alias=identifier)?
-    ;
-
-tableName
-@init { msgs.push("table name"); }
-@after { msgs.pop(); }
-    :
-    db=identifier DOT tab=identifier
-    |
-    tab=identifier
-    ;
-
-viewName
-@init { msgs.push("view name"); }
-@after { msgs.pop(); }
-    :
-    (db=identifier DOT)? view=identifier
-    ;
-
-subQuerySource
-@init { msgs.push("subquery source"); }
-@after { msgs.pop(); }
-    :
-    LPAREN queryStatementExpression RPAREN identifier 
-    ;
-
-//---------------------- Rules for parsing PTF clauses -----------------------------
-partitioningSpec
-@init { msgs.push("partitioningSpec clause"); }
-@after { msgs.pop(); } 
-   :
-   partitionByClause orderByClause? 
-   orderByClause 
-   distributeByClause sortByClause? 
-   sortByClause 
-   clusterByClause 
-   ;
-
-partitionTableFunctionSource
-@init { msgs.push("partitionTableFunctionSource clause"); }
-@after { msgs.pop(); } 
-   :
-   subQuerySource |
-   tableSource |
-   partitionedTableFunction
-   ;
-
-partitionedTableFunction
-@init { msgs.push("ptf clause"); }
-@after { msgs.pop(); } 
-   :
-   name=Identifier
-   LPAREN KW_ON ptfsrc=partitionTableFunctionSource partitioningSpec?
-     ((Identifier LPAREN expression RPAREN ) )? 
-   RPAREN alias=Identifier? 
-   ;
-
-//----------------------- Rules for parsing whereClause -----------------------------
-// where a=b and ...
-whereClause
-@init { msgs.push("where clause"); }
-@after { msgs.pop(); }
-    :
-    KW_WHERE searchCondition 
-    ;
-
-searchCondition
-@init { msgs.push("search condition"); }
-@after { msgs.pop(); }
-    :
-    expression
-    ;
-
-//-----------------------------------------------------------------------------------
-
-
-// group by a,b
-groupByClause
-@init { msgs.push("group by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_GROUP KW_BY
-    groupByExpression
-    ( COMMA groupByExpression )*
-    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
-    (sets=KW_GROUPING KW_SETS 
-    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
-    ;
-
-groupingSetExpression
-@init {msgs.push("grouping set expression"); }
-@after {msgs.pop(); }
-   :
-   groupByExpression
-   |
-   LPAREN 
-   groupByExpression (COMMA groupByExpression)*
-   RPAREN
-   |
-   LPAREN
-   RPAREN
-   ;
-
-
-groupByExpression
-@init { msgs.push("group by expression"); }
-@after { msgs.pop(); }
-    :
-    expression
-    ;
-
-havingClause
-@init { msgs.push("having clause"); }
-@after { msgs.pop(); }
-    :
-    KW_HAVING havingCondition 
-    ;
-
-havingCondition
-@init { msgs.push("having condition"); }
-@after { msgs.pop(); }
-    :
-    expression
-    ;
-
-// order by a,b
-orderByClause
-@init { msgs.push("order by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_ORDER KW_BY
-    LPAREN columnRefOrder
-    ( COMMA columnRefOrder)* RPAREN 
-    |
-    KW_ORDER KW_BY
-    columnRefOrder
-    ( COMMA columnRefOrder)* 
-    ;
-
-clusterByClause
-@init { msgs.push("cluster by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_CLUSTER KW_BY
-    LPAREN expression (COMMA expression)* RPAREN
-    |
-    KW_CLUSTER KW_BY
-    expression
-    ((COMMA))*
-    ;
-
-partitionByClause
-@init  { msgs.push("partition by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_PARTITION KW_BY
-    LPAREN expression (COMMA expression)* RPAREN
-    |
-    KW_PARTITION KW_BY
-    expression ((COMMA))*
-    ;
-
-distributeByClause
-@init { msgs.push("distribute by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_DISTRIBUTE KW_BY
-    LPAREN expression (COMMA expression)* RPAREN
-    |
-    KW_DISTRIBUTE KW_BY
-    expression ((COMMA))*
-    ;
-
-sortByClause
-@init { msgs.push("sort by clause"); }
-@after { msgs.pop(); }
-    :
-    KW_SORT KW_BY
-    LPAREN columnRefOrder
-    ( COMMA columnRefOrder)* RPAREN 
-    |
-    KW_SORT KW_BY
-    columnRefOrder
-    ( (COMMA))*
-    ;
-
-// fun(par1, par2, par3)
-function
-@init { msgs.push("function specification"); }
-@after { msgs.pop(); }
-    :
-    functionName
-    LPAREN
-      (
-        (star=STAR)
-        | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)?
-      )
-    RPAREN 
-    ;
-
-functionName
-@init { msgs.push("function name"); }
-@after { msgs.pop(); }
-    : // Keyword IF is also a function name
-    KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE | identifier
-    ;
-
-castExpression
-@init { msgs.push("cast expression"); }
-@after { msgs.pop(); }
-    :
-    KW_CAST
-    LPAREN
-          expression
-          KW_AS
-          primitiveType
-    RPAREN 
-    ;
-
-caseExpression
-@init { msgs.push("case expression"); }
-@after { msgs.pop(); }
-    :
-    KW_CASE expression
-    (KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END 
-    ;
-
-whenExpression
-@init { msgs.push("case expression"); }
-@after { msgs.pop(); }
-    :
-    KW_CASE
-     ( KW_WHEN expression KW_THEN expression)+
-    (KW_ELSE expression)?
-    KW_END 
-    ;
-
-constant
-@init { msgs.push("constant"); }
-@after { msgs.pop(); }
-    :
-    Number
-    | StringLiteral
-    | stringLiteralSequence
-    | BigintLiteral
-    | SmallintLiteral
-    | TinyintLiteral
-    | DecimalLiteral
-    | charSetStringLiteral
-    | booleanValue
-    ;
-
-stringLiteralSequence
-    :
-    StringLiteral StringLiteral+ 
-    ;
-
-charSetStringLiteral
-@init { msgs.push("character string literal"); }
-@after { msgs.pop(); }
-    :
-    csName=CharSetName csLiteral=CharSetLiteral 
-    ;
-
-expression
-@init { msgs.push("expression specification"); }
-@after { msgs.pop(); }
-    :
-    precedenceOrExpression
-    ;
-
-atomExpression
-    :
-    KW_NULL 
-    | constant
-    | function
-    | castExpression
-    | caseExpression
-    | whenExpression
-    | tableOrColumn
-    | LPAREN expression RPAREN
-    ;
-
-
-precedenceFieldExpression
-    :
-    atomExpression ((LSQUARE expression RSQUARE) | (DOT identifier))*
-    ;
-
-precedenceUnaryOperator
-    :
-    PLUS | MINUS | TILDE
-    ;
-
-nullCondition
-    :
-    KW_NULL     
-    | KW_NOT KW_NULL     
-    ;
-
-precedenceUnaryPrefixExpression
-    :
-    (precedenceUnaryOperator)* precedenceFieldExpression
-    ;
-
-precedenceUnarySuffixExpression
-    : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)?
-    ;
-
-
-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
-    ;
-
-precedenceEqualExpression
-    :
-    (left=precedenceBitwiseOrExpression     
-    )
-    (
-       (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression) 
-    | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
-    | (KW_NOT KW_IN expressions) 
-    | (KW_IN expressions) 
-    | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) 
-    | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
-    )*
-    ;
-
-expressions
-    :
-    LPAREN expression (COMMA expression)* RPAREN 
-    ;
-
-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
-    ;
-
-tableOrPartition
-   :
-   tableName partitionSpec? 
-   ;
-
-partitionSpec
-    :
-    KW_PARTITION
-     LPAREN partitionVal (COMMA  partitionVal )* RPAREN 
-    ;
-
-partitionVal
-    :
-    identifier (EQUAL constant)? 
-    ;
-
-dropPartitionSpec
-    :
-    KW_PARTITION
-     LPAREN dropPartitionVal (COMMA  dropPartitionVal )* RPAREN 
-    ;
-
-dropPartitionVal
-    :
-    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
-    | StringLiteral
-    | identifier
-    ;
-
-identifier
-    :
-    Identifier
-    | nonReserved 
-    ;
-    
-nonReserved
-    :
-    KW_TRUE | KW_FALSE | KW_LIKE | KW_EXISTS | KW_ASC | KW_DESC | KW_ORDER | KW_GROUP | KW_BY | KW_AS | KW_INSERT | KW_OVERWRITE | KW_OUTER | KW_LEFT | KW_RIGHT | KW_FULL | KW_PARTITION | KW_PARTITIONS | KW_TABLE | KW_TABLES | KW_COLUMNS | KW_INDEX | KW_INDEXES | KW_REBUILD | KW_FUNCTIONS | KW_SHOW | KW_MSCK | KW_REPAIR | KW_DIRECTORY | KW_LOCAL | KW_USING | KW_CLUSTER | KW_DISTRIBUTE | KW_SORT | KW_UNION | KW_LOAD | KW_EXPORT | KW_IMPORT | KW_DATA | KW_INPATH | KW_IS | KW_NULL | KW_CREATE | KW_EXTERNAL | KW_ALTER | KW_CHANGE | KW_FIRST | KW_AFTER | KW_DESCRIBE | KW_DROP | KW_RENAME | KW_IGNORE | KW_PROTECTION | KW_TO | KW_COMMENT | KW_BOOLEAN | KW_TINYINT | KW_SMALLINT | KW_INT | KW_BIGINT | KW_FLOAT | KW_DOUBLE | KW_DATE | KW_DATETIME | KW_TIMESTAMP | KW_DECIMAL | KW_STRING | KW_ARRAY | KW_STRUCT | KW_UNIONTYPE | KW_PARTITIONED | KW_CLUSTERED | KW_SORTED | KW_INTO | KW_BUCKETS | KW_ROW | KW_ROWS | KW_FORMAT | KW_DELIMITED | KW_FIELDS | KW_TERMINATED | KW_ESCAPED | KW_COLLECTION | 
 KW_ITEMS | KW_KEYS | KW_KEY_TYPE | KW_LINES | KW_STORED | KW_FILEFORMAT | KW_SEQUENCEFILE | KW_TEXTFILE | KW_RCFILE | KW_ORCFILE | KW_INPUTFORMAT | KW_OUTPUTFORMAT | KW_INPUTDRIVER | KW_OUTPUTDRIVER | KW_OFFLINE | KW_ENABLE | KW_DISABLE | KW_READONLY | KW_NO_DROP | KW_LOCATION | KW_BUCKET | KW_OUT | KW_OF | KW_PERCENT | KW_ADD | KW_REPLACE | KW_RLIKE | KW_REGEXP | KW_TEMPORARY | KW_EXPLAIN | KW_FORMATTED | KW_PRETTY | KW_DEPENDENCY | KW_SERDE | KW_WITH | KW_DEFERRED | KW_SERDEPROPERTIES | KW_DBPROPERTIES | KW_LIMIT | KW_SET | KW_UNSET | KW_TBLPROPERTIES | KW_IDXPROPERTIES | KW_VALUE_TYPE | KW_ELEM_TYPE | KW_MAPJOIN | KW_STREAMTABLE | KW_HOLD_DDLTIME | KW_CLUSTERSTATUS | KW_UTC | KW_UTCTIMESTAMP | KW_LONG | KW_DELETE | KW_PLUS | KW_MINUS | KW_FETCH | KW_INTERSECT | KW_VIEW | KW_IN | KW_DATABASES | KW_MATERIALIZED | KW_SCHEMA | KW_SCHEMAS | KW_GRANT | KW_REVOKE | KW_SSL | KW_UNDO | KW_LOCK | KW_LOCKS | KW_UNLOCK | KW_SHARED | KW_EXCLUSIVE | KW_PROCEDURE | KW_UNSIGNED | KW_WHILE | KW_R
 EAD | KW_READS | KW_PURGE | KW_RANGE | KW_ANALYZE | KW_BEFORE | KW_BETWEEN | KW_BOTH | KW_BINARY | KW_CONTINUE | KW_CURSOR | KW_TRIGGER | KW_RECORDREADER | KW_RECORDWRITER | KW_SEMI | KW_LATERAL | KW_TOUCH | KW_ARCHIVE | KW_UNARCHIVE | KW_COMPUTE | KW_STATISTICS | KW_USE | KW_OPTION | KW_CONCATENATE | KW_SHOW_DATABASE | KW_UPDATE | KW_RESTRICT | KW_CASCADE | KW_SKEWED | KW_ROLLUP | KW_CUBE | KW_DIRECTORIES | KW_FOR | KW_GROUPING | KW_SETS | KW_TRUNCATE | KW_NOSCAN | KW_USER | KW_ROLE | KW_INNER
-    ;
-
-//-----------------------------------------------------------------------------------
-
-// starting rule
-statement
-	: explainStatement EOF
-	| execStatement EOF
-	;
-
-explainStatement
-@init { msgs.push("explain statement"); }
-@after { msgs.pop(); }
-	: KW_EXPLAIN (explainOptions=KW_EXTENDED|explainOptions=KW_FORMATTED|explainOptions=KW_DEPENDENCY)? execStatement
-	;
-
-execStatement
-@init { msgs.push("statement"); }
-@after { msgs.pop(); }
-    : queryStatementExpression
-    | loadStatement
-    | exportStatement
-    | importStatement
-    | ddlStatement
-    ;
-
-loadStatement
-@init { msgs.push("load statement"); }
-@after { msgs.pop(); }
-    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition)
-    ;
-
-exportStatement
-@init { msgs.push("export statement"); }
-@after { msgs.pop(); }
-    : KW_EXPORT KW_TABLE (tab=tableOrPartition) KW_TO (path=StringLiteral)
-    ;
-
-importStatement
-@init { msgs.push("import statement"); }
-@after { msgs.pop(); }
-	: KW_IMPORT ((ext=KW_EXTERNAL)? KW_TABLE (tab=tableOrPartition))? KW_FROM (path=StringLiteral) tableLocation?
-    ;
-
-ddlStatement
-@init { msgs.push("ddl statement"); }
-@after { msgs.pop(); }
-    : createDatabaseStatement
-    | switchDatabaseStatement
-    | dropDatabaseStatement
-    | createTableStatement
-    | dropTableStatement
-    | truncateTableStatement
-    | alterStatement
-    | descStatement
-    | showStatement
-    | metastoreCheck
-    | createViewStatement
-    | dropViewStatement
-    | createFunctionStatement
-    | createIndexStatement
-    | dropIndexStatement
-    | dropFunctionStatement
-    | analyzeStatement
-    | lockStatement
-    | unlockStatement
-    | createRoleStatement
-    | dropRoleStatement
-    | grantPrivileges
-    | revokePrivileges
-    | showGrants
-    | showRoleGrants
-    | grantRole
-    | revokeRole
-    ;
-
-ifExists
-@init { msgs.push("if exists clause"); }
-@after { msgs.pop(); }
-    : KW_IF KW_EXISTS
-    ;
-
-restrictOrCascade
-@init { msgs.push("restrict or cascade clause"); }
-@after { msgs.pop(); }
-    : KW_RESTRICT
-    | KW_CASCADE
-    ;
-
-ifNotExists
-@init { msgs.push("if not exists clause"); }
-@after { msgs.pop(); }
-    : KW_IF KW_NOT KW_EXISTS
-    ;
-
-storedAsDirs
-@init { msgs.push("stored as directories"); }
-@after { msgs.pop(); }
-    : KW_STORED KW_AS KW_DIRECTORIES
-    ;
-
-orReplace
-@init { msgs.push("or replace clause"); }
-@after { msgs.pop(); }
-    : KW_OR KW_REPLACE
-    ;
-
-ignoreProtection
-@init { msgs.push("ignore protection clause"); }
-@after { msgs.pop(); }
-        : KW_IGNORE KW_PROTECTION
-        ;
-
-createDatabaseStatement
-@init { msgs.push("create database statement"); }
-@after { msgs.pop(); }
-    : KW_CREATE (KW_DATABASE|KW_SCHEMA)
-        ifNotExists?
-        name=identifier
-        databaseComment?
-        dbLocation?
-        (KW_WITH KW_DBPROPERTIES dbprops=dbProperties)?
-    ;
-
-dbLocation
-@init { msgs.push("database location specification"); }
-@after { msgs.pop(); }
-    :
-      KW_LOCATION locn=StringLiteral 
-    ;
-
-dbProperties
-@init { msgs.push("dbproperties"); }
-@after { msgs.pop(); }
-    :
-      LPAREN dbPropertiesList RPAREN 
-    ;
-
-dbPropertiesList
-@init { msgs.push("database properties list"); }
-@after { msgs.pop(); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* 
-    ;
-
-
-switchDatabaseStatement
-@init { msgs.push("switch database statement"); }
-@after { msgs.pop(); }
-    : KW_USE identifier
-    ;
-
-dropDatabaseStatement
-@init { msgs.push("drop database statement"); }
-@after { msgs.pop(); }
-    : KW_DROP (KW_DATABASE|KW_SCHEMA) ifExists? identifier restrictOrCascade?
-    ;
-
-databaseComment
-@init { msgs.push("database's comment"); }
-@after { msgs.pop(); }
-    : KW_COMMENT comment=StringLiteral
-    ;
-
-createTableStatement
-@init { msgs.push("create table statement"); }
-@after { msgs.pop(); }
-    : KW_CREATE (ext=KW_EXTERNAL)? KW_TABLE ifNotExists? name=tableName
-      (  like=KW_LIKE likeName=tableName
-         tableLocation?
-         tablePropertiesPrefixed?
-       | (LPAREN columnNameTypeList RPAREN)?
-         tableComment?
-         tablePartition?
-         tableBuckets?
-         tableSkewed?
-         tableRowFormat?
-         tableFileFormat?
-         tableLocation?
-         tablePropertiesPrefixed?
-         (KW_AS selectStatement)?
-      )
-    ;
-
-truncateTableStatement
-@init { msgs.push("truncate table statement"); }
-@after { msgs.pop(); }
-    : KW_TRUNCATE KW_TABLE tablePartitionPrefix 
-;
-
-createIndexStatement
-@init { msgs.push("create index statement");}
-@after {msgs.pop();}
-    : 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?
-    ;
-
-indexComment
-@init { msgs.push("comment on an index");}
-@after {msgs.pop();}
-        :
-                KW_COMMENT comment=StringLiteral  
-        ;
-
-autoRebuild
-@init { msgs.push("auto rebuild index");}
-@after {msgs.pop();}
-    : KW_WITH KW_DEFERRED KW_REBUILD
-    ;
-
-indexTblName
-@init { msgs.push("index table name");}
-@after {msgs.pop();}
-    : KW_IN KW_TABLE indexTbl=tableName
-    ;
-
-indexPropertiesPrefixed
-@init { msgs.push("table properties with prefix"); }
-@after { msgs.pop(); }
-    :
-        KW_IDXPROPERTIES indexProperties
-    ;
-
-indexProperties
-@init { msgs.push("index properties"); }
-@after { msgs.pop(); }
-    :
-      LPAREN indexPropertiesList RPAREN 
-    ;
-
-indexPropertiesList
-@init { msgs.push("index properties list"); }
-@after { msgs.pop(); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* 
-    ;
-
-dropIndexStatement
-@init { msgs.push("drop index statement");}
-@after {msgs.pop();}
-    : KW_DROP KW_INDEX ifExists? indexName=identifier KW_ON tab=tableName
-    ;
-
-dropTableStatement
-@init { msgs.push("drop statement"); }
-@after { msgs.pop(); }
-    : KW_DROP KW_TABLE ifExists? tableName 
-    ;
-
-alterStatement
-@init { msgs.push("alter statement"); }
-@after { msgs.pop(); }
-    : 
-    KW_ALTER
-        (
-            KW_TABLE alterTableStatementSuffix
-        |
-            KW_VIEW alterViewStatementSuffix
-        |
-            KW_INDEX alterIndexStatementSuffix
-        |
-            KW_DATABASE alterDatabaseStatementSuffix
-        )
-    ;
-
-alterTableStatementSuffix
-@init { msgs.push("alter table statement"); }
-@after { msgs.pop(); }
-    : alterStatementSuffixRename
-    | alterStatementSuffixAddCol
-    | alterStatementSuffixRenameCol
-    | alterStatementSuffixDropPartitions
-    | alterStatementSuffixAddPartitions
-    | alterStatementSuffixTouch
-    | alterStatementSuffixArchive
-    | alterStatementSuffixUnArchive
-    | alterStatementSuffixProperties
-    | alterTblPartitionStatement
-    | alterStatementSuffixSkewedby
-    ;
-
-alterViewStatementSuffix
-@init { msgs.push("alter view statement"); }
-@after { msgs.pop(); }
-    : alterViewSuffixProperties
-    | alterStatementSuffixRename
-    | alterStatementSuffixAddPartitions
-    | alterStatementSuffixDropPartitions
-    | name=tableName KW_AS selectStatement
-    ;
-
-alterIndexStatementSuffix
-@init { msgs.push("alter index statement"); }
-@after { msgs.pop(); }
-    : indexName=identifier
-      (KW_ON tableNameId=identifier)
-      partitionSpec?
-    (
-      KW_REBUILD
-    |
-      KW_SET KW_IDXPROPERTIES
-      indexProperties
-    )
-    ;
-
-alterDatabaseStatementSuffix
-@init { msgs.push("alter database statement"); }
-@after { msgs.pop(); }
-    : alterDatabaseSuffixProperties
-    ;
-
-alterDatabaseSuffixProperties
-@init { msgs.push("alter database properties statement"); }
-@after { msgs.pop(); }
-    : name=identifier KW_SET KW_DBPROPERTIES dbProperties
-    ;
-
-alterStatementSuffixRename
-@init { msgs.push("rename statement"); }
-@after { msgs.pop(); }
-    : oldName=identifier KW_RENAME KW_TO newName=identifier
-    ;
-
-alterStatementSuffixAddCol
-@init { msgs.push("add column statement"); }
-@after { msgs.pop(); }
-    : identifier (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
-    ;
-
-alterStatementSuffixRenameCol
-@init { msgs.push("rename column name"); }
-@after { msgs.pop(); }
-    : identifier KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
-    ;
-
-alterStatementChangeColPosition
-    : first=KW_FIRST|KW_AFTER afterCol=identifier
-    ;
-
-alterStatementSuffixAddPartitions
-@init { msgs.push("add partition statement"); }
-@after { msgs.pop(); }
-    : identifier KW_ADD ifNotExists? partitionSpec partitionLocation? (partitionSpec partitionLocation?)*
-    ;
-
-alterStatementSuffixTouch
-@init { msgs.push("touch statement"); }
-@after { msgs.pop(); }
-    : identifier KW_TOUCH (partitionSpec)*
-    ;
-
-alterStatementSuffixArchive
-@init { msgs.push("archive statement"); }
-@after { msgs.pop(); }
-    : identifier KW_ARCHIVE (partitionSpec)*
-    ;
-
-alterStatementSuffixUnArchive
-@init { msgs.push("unarchive statement"); }
-@after { msgs.pop(); }
-    : identifier KW_UNARCHIVE (partitionSpec)*
-    ;
-
-partitionLocation
-@init { msgs.push("partition location"); }
-@after { msgs.pop(); }
-    :
-      KW_LOCATION locn=StringLiteral 
-    ;
-
-alterStatementSuffixDropPartitions
-@init { msgs.push("drop partition statement"); }
-@after { msgs.pop(); }
-    : identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
-    ;
-
-alterStatementSuffixProperties
-@init { msgs.push("alter properties statement"); }
-@after { msgs.pop(); }
-    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
-    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    ;
-
-alterViewSuffixProperties
-@init { msgs.push("alter view properties statement"); }
-@after { msgs.pop(); }
-    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
-    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
-    ;
-
-alterStatementSuffixSerdeProperties
-@init { msgs.push("alter serdes statement"); }
-@after { msgs.pop(); }
-    : KW_SET KW_SERDE serdeName=StringLiteral (KW_WITH KW_SERDEPROPERTIES tableProperties)?
-    | KW_SET KW_SERDEPROPERTIES tableProperties
-    ;
-
-tablePartitionPrefix
-@init {msgs.push("table partition prefix");}
-@after {msgs.pop();}
-  :name=identifier partitionSpec?
-  ;
-
-alterTblPartitionStatement
-@init {msgs.push("alter table partition statement");}
-@after {msgs.pop();}
-  : tablePartitionPrefix alterTblPartitionStatementSuffix
-  |Identifier KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
-  ;
-
-alterTblPartitionStatementSuffix
-@init {msgs.push("alter table partition statement suffix");}
-@after {msgs.pop();}
-  : alterStatementSuffixFileFormat
-  | alterStatementSuffixLocation
-  | alterStatementSuffixProtectMode
-  | alterStatementSuffixMergeFiles
-  | alterStatementSuffixSerdeProperties
-  | alterStatementSuffixRenamePart
-  | alterStatementSuffixBucketNum
-  | alterTblPartitionStatementSuffixSkewedLocation
-  | alterStatementSuffixClusterbySortby
-  ;
-
-alterStatementSuffixFileFormat
-@init {msgs.push("alter fileformat statement"); }
-@after {msgs.pop();}
-	: KW_SET KW_FILEFORMAT fileFormat
-	;
-
-alterStatementSuffixClusterbySortby
-@init {msgs.push("alter partition cluster by sort by statement");}
-@after {msgs.pop();}
-  : KW_NOT KW_CLUSTERED 
-  | KW_NOT KW_SORTED 
-  | tableBuckets 
-  ;
-
-alterTblPartitionStatementSuffixSkewedLocation
-@init {msgs.push("alter partition skewed location");}
-@after {msgs.pop();}
-  : KW_SET KW_SKEWED KW_LOCATION skewedLocations
-  ;
-  
-skewedLocations
-@init { msgs.push("skewed locations"); }
-@after { msgs.pop(); }
-    :
-      LPAREN skewedLocationsList RPAREN 
-    ;
-
-skewedLocationsList
-@init { msgs.push("skewed locations list"); }
-@after { msgs.pop(); }
-    :
-      skewedLocationMap (COMMA skewedLocationMap)* 
-    ;
-
-skewedLocationMap
-@init { msgs.push("specifying skewed location map"); }
-@after { msgs.pop(); }
-    :
-      key=skewedValueLocationElement EQUAL value=StringLiteral 
-    ;
-
-alterStatementSuffixLocation
-@init {msgs.push("alter location");}
-@after {msgs.pop();}
-  : KW_SET KW_LOCATION newLoc=StringLiteral
-  ;
-
-	
-alterStatementSuffixSkewedby
-@init {msgs.push("alter skewed by statement");}
-@after{msgs.pop();}
-	:name=identifier tableSkewed
-	|
-	name=identifier KW_NOT KW_SKEWED
-	|
-	name=identifier KW_NOT storedAsDirs
-	;
-
-alterStatementSuffixProtectMode
-@init { msgs.push("alter partition protect mode statement"); }
-@after { msgs.pop(); }
-    : alterProtectMode
-    ;
-
-alterStatementSuffixRenamePart
-@init { msgs.push("alter table rename partition statement"); }
-@after { msgs.pop(); }
-    : KW_RENAME KW_TO partitionSpec
-    ;
-
-alterStatementSuffixMergeFiles
-@init { msgs.push(""); }
-@after { msgs.pop(); }
-    : KW_CONCATENATE
-    ;
-
-alterProtectMode
-@init { msgs.push("protect mode specification enable"); }
-@after { msgs.pop(); }
-    : KW_ENABLE alterProtectModeMode  
-    | KW_DISABLE alterProtectModeMode  
-    ;
-
-alterProtectModeMode
-@init { msgs.push("protect mode specification enable"); }
-@after { msgs.pop(); }
-    : KW_OFFLINE  
-    | KW_NO_DROP KW_CASCADE? 
-    | KW_READONLY  
-    ;
-
-alterStatementSuffixBucketNum
-@init { msgs.push(""); }
-@after { msgs.pop(); }
-    : KW_INTO num=Number KW_BUCKETS
-    ;
-
-fileFormat
-@init { msgs.push("file format specification"); }
-@after { msgs.pop(); }
-    : KW_SEQUENCEFILE  
-    | KW_TEXTFILE  
-    | KW_RCFILE  
-    | KW_ORCFILE 
-    | KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
-    | genericSpec=identifier 
-    ;
-
-tabTypeExpr
-@init { msgs.push("specifying table types"); }
-@after { msgs.pop(); }
-
-   : 
-   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))*
-   ;
-
-descTabTypeExpr
-@init { msgs.push("specifying describe table types"); }
-@after { msgs.pop(); }
-
-   : 
-   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))* identifier?
-   ;
-
-partTypeExpr
-@init { msgs.push("specifying table partitions"); }
-@after { msgs.pop(); }
-    :  tabTypeExpr partitionSpec? 
-    ;
-
-descPartTypeExpr
-@init { msgs.push("specifying describe table partitions"); }
-@after { msgs.pop(); }
-    :  descTabTypeExpr partitionSpec? 
-    ;
-
-descStatement
-@init { msgs.push("describe statement"); }
-@after { msgs.pop(); }
-    : (KW_DESCRIBE|KW_DESC) (descOptions=KW_FORMATTED|descOptions=KW_EXTENDED|descOptions=KW_PRETTY)? (parttype=descPartTypeExpr) 
-    | (KW_DESCRIBE|KW_DESC) KW_FUNCTION KW_EXTENDED? (name=descFuncNames) 
-    | (KW_DESCRIBE|KW_DESC) KW_DATABASE KW_EXTENDED? (dbName=identifier) 
-    ;
-
-analyzeStatement
-@init { msgs.push("analyze statement"); }
-@after { msgs.pop(); }
-    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) | (KW_FOR KW_COLUMNS statsColumnName=columnNameList))? 
-    ;
-
-showStatement
-@init { msgs.push("show statement"); }
-@after { msgs.pop(); }
-    : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? 
-    | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  
-    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tabname=tableName ((KW_FROM|KW_IN) db_name=identifier)? 
-    | KW_SHOW KW_FUNCTIONS showStmtIdentifier?  
-    | KW_SHOW KW_PARTITIONS identifier partitionSpec? 
-    | KW_SHOW KW_CREATE KW_TABLE tabName=tableName 
-    | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
-    | KW_SHOW KW_TBLPROPERTIES tblName=identifier (LPAREN prptyName=StringLiteral RPAREN)? 
-    | KW_SHOW KW_LOCKS (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? 
-    | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
-    ;
-
-lockStatement
-@init { msgs.push("lock statement"); }
-@after { msgs.pop(); }
-    : KW_LOCK KW_TABLE tableName partitionSpec? lockMode 
-    ;
-
-lockMode
-@init { msgs.push("lock mode"); }
-@after { msgs.pop(); }
-    : KW_SHARED | KW_EXCLUSIVE
-    ;
-
-unlockStatement
-@init { msgs.push("unlock statement"); }
-@after { msgs.pop(); }
-    : KW_UNLOCK KW_TABLE tableName partitionSpec?  
-    ;
-
-createRoleStatement
-@init { msgs.push("create role"); }
-@after { msgs.pop(); }
-    : KW_CREATE KW_ROLE roleName=identifier
-    ;
-
-dropRoleStatement
-@init {msgs.push("drop role");}
-@after {msgs.pop();}
-    : KW_DROP KW_ROLE roleName=identifier
-    ;
-
-grantPrivileges
-@init {msgs.push("grant privileges");}
-@after {msgs.pop();}
-    : KW_GRANT privList=privilegeList
-      privilegeObject?
-      KW_TO principalSpecification
-      (KW_WITH withOption)?
-    ;
-
-revokePrivileges
-@init {msgs.push("revoke privileges");}
-@afer {msgs.pop();}
-    : KW_REVOKE privilegeList privilegeObject? KW_FROM principalSpecification
-    ;
-
-grantRole
-@init {msgs.push("grant role");}
-@after {msgs.pop();}
-    : KW_GRANT KW_ROLE identifier (COMMA identifier)* KW_TO principalSpecification
-    ;
-
-revokeRole
-@init {msgs.push("revoke role");}
-@after {msgs.pop();}
-    : KW_REVOKE KW_ROLE identifier (COMMA identifier)* KW_FROM principalSpecification
-    ;
-
-showRoleGrants
-@init {msgs.push("show role grants");}
-@after {msgs.pop();}
-    : KW_SHOW KW_ROLE KW_GRANT principalName
-    ;
-
-showGrants
-@init {msgs.push("show grants");}
-@after {msgs.pop();}
-    : KW_SHOW KW_GRANT principalName privilegeIncludeColObject?
-    ;
-
-privilegeIncludeColObject
-@init {msgs.push("privilege object including columns");}
-@after {msgs.pop();}
-    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier (LPAREN cols=columnNameList RPAREN)? partitionSpec?
-    ;
-
-privilegeObject
-@init {msgs.push("privilege subject");}
-@after {msgs.pop();}
-    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier partitionSpec?
-    ;
-
-privilegeList
-@init {msgs.push("grant privilege list");}
-@after {msgs.pop();}
-    : privlegeDef (COMMA privlegeDef)*
-    ;
-
-privlegeDef
-@init {msgs.push("grant privilege");}
-@after {msgs.pop();}
-    : privilegeType (LPAREN cols=columnNameList RPAREN)?
-    ;
-
-privilegeType
-@init {msgs.push("privilege type");}
-@after {msgs.pop();}
-    : KW_ALL 
-    | KW_ALTER 
-    | KW_UPDATE 
-    | KW_CREATE 
-    | KW_DROP 
-    | KW_INDEX 
-    | KW_LOCK 
-    | KW_SELECT 
-    | KW_SHOW_DATABASE 
-    ;
-
-principalSpecification
-@init { msgs.push("user/group/role name list"); }
-@after { msgs.pop(); }
-    : principalName (COMMA principalName)* 
-    ;
-
-principalName
-@init {msgs.push("user|group|role name");}
-@after {msgs.pop();}
-    : KW_USER identifier 
-    | KW_GROUP identifier 
-    | KW_ROLE identifier 
-    ;
-
-withOption
-@init {msgs.push("grant with option");}
-@after {msgs.pop();}
-    : KW_GRANT KW_OPTION
-    ;
-
-metastoreCheck
-@init { msgs.push("metastore check statement"); }
-@after { msgs.pop(); }
-    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE table=identifier partitionSpec? (COMMA partitionSpec)*)?
-    ;
-
-createFunctionStatement
-@init { msgs.push("create function statement"); }
-@after { msgs.pop(); }
-    : KW_CREATE KW_TEMPORARY KW_FUNCTION identifier KW_AS StringLiteral
-    ;
-
-dropFunctionStatement
-@init { msgs.push("drop temporary function statement"); }
-@after { msgs.pop(); }
-    : KW_DROP KW_TEMPORARY KW_FUNCTION ifExists? identifier
-    ;
-
-createViewStatement
-@init {
-    msgs.push("create view statement");
-}
-@after { msgs.pop(); }
-    : KW_CREATE (orReplace)? KW_VIEW (ifNotExists)? name=tableName
-        (LPAREN columnNameCommentList RPAREN)? tableComment? viewPartition?
-        tablePropertiesPrefixed?
-        KW_AS
-        selectStatement
-    ;
-
-viewPartition
-@init { msgs.push("view partition specification"); }
-@after { msgs.pop(); }
-    : KW_PARTITIONED KW_ON LPAREN columnNameList RPAREN
-    ;
-
-dropViewStatement
-@init { msgs.push("drop view statement"); }
-@after { msgs.pop(); }
-    : KW_DROP KW_VIEW ifExists? viewName 
-    ;
-
-showStmtIdentifier
-@init { msgs.push("identifier for show statement"); }
-@after { msgs.pop(); }
-    : identifier
-    | StringLiteral
-    ;
-
-tableComment
-@init { msgs.push("table's comment"); }
-@after { msgs.pop(); }
-    :
-      KW_COMMENT comment=StringLiteral  
-    ;
-
-tablePartition
-@init { msgs.push("table partition specification"); }
-@after { msgs.pop(); }
-    : KW_PARTITIONED KW_BY LPAREN columnNameTypeList RPAREN
-    ;
-
-tableBuckets
-@init { msgs.push("table buckets specification"); }
-@after { msgs.pop(); }
-    :
-      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS
-    ;
-
-tableSkewed
-@init { msgs.push("table skewed specification"); }
-@after { msgs.pop(); }
-    :
-     KW_SKEWED KW_BY LPAREN skewedCols=columnNameList RPAREN KW_ON LPAREN (skewedValues=skewedValueElement) RPAREN (storedAsDirs)?
-    ;
-
-rowFormat
-@init { msgs.push("serde specification"); }
-@after { msgs.pop(); }
-    : rowFormatSerde 
-    | rowFormatDelimited 
-    ;
-
-recordReader
-@init { msgs.push("record reader specification"); }
-@after { msgs.pop(); }
-    : KW_RECORDREADER StringLiteral 
-    ;
-
-recordWriter
-@init { msgs.push("record writer specification"); }
-@after { msgs.pop(); }
-    : KW_RECORDWRITER StringLiteral 
-    ;
-
-rowFormatSerde
-@init { msgs.push("serde format specification"); }
-@after { msgs.pop(); }
-    : KW_ROW KW_FORMAT KW_SERDE name=StringLiteral (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
-    ;
-
-rowFormatDelimited
-@init { msgs.push("serde properties specification"); }
-@after { msgs.pop(); }
-    :
-      KW_ROW KW_FORMAT KW_DELIMITED tableRowFormatFieldIdentifier? tableRowFormatCollItemsIdentifier? tableRowFormatMapKeysIdentifier? tableRowFormatLinesIdentifier?
-    ;
-
-tableRowFormat
-@init { msgs.push("table row format specification"); }
-@after { msgs.pop(); }
-    :
-      rowFormatDelimited
-    | rowFormatSerde
-    ;
-
-tablePropertiesPrefixed
-@init { msgs.push("table properties with prefix"); }
-@after { msgs.pop(); }
-    :
-        KW_TBLPROPERTIES tableProperties
-    ;
-
-tableProperties
-@init { msgs.push("table properties"); }
-@after { msgs.pop(); }
-    :
-      LPAREN tablePropertiesList RPAREN 
-    ;
-
-tablePropertiesList
-@init { msgs.push("table properties list"); }
-@after { msgs.pop(); }
-    :
-      keyValueProperty (COMMA keyValueProperty)* 
-    |
-      keyProperty (COMMA keyProperty)* 
-    ;
-
-keyValueProperty
-@init { msgs.push("specifying key/value property"); }
-@after { msgs.pop(); }
-    :
-      key=StringLiteral EQUAL value=StringLiteral 
-    ;
-
-keyProperty
-@init { msgs.push("specifying key property"); }
-@after { msgs.pop(); }
-    :
-      key=StringLiteral 
-    ;
-
-tableRowFormatFieldIdentifier
-@init { msgs.push("table row format's field separator"); }
-@after { msgs.pop(); }
-    :
-      KW_FIELDS KW_TERMINATED KW_BY fldIdnt=StringLiteral (KW_ESCAPED KW_BY fldEscape=StringLiteral)?
-    ;
-
-tableRowFormatCollItemsIdentifier
-@init { msgs.push("table row format's column separator"); }
-@after { msgs.pop(); }
-    :
-      KW_COLLECTION KW_ITEMS KW_TERMINATED KW_BY collIdnt=StringLiteral
-    ;
-
-tableRowFormatMapKeysIdentifier
-@init { msgs.push("table row format's map key separator"); }
-@after { msgs.pop(); }
-    :
-      KW_MAP KW_KEYS KW_TERMINATED KW_BY mapKeysIdnt=StringLiteral
-    ;
-
-tableRowFormatLinesIdentifier
-@init { msgs.push("table row format's line separator"); }
-@after { msgs.pop(); }
-    :
-      KW_LINES KW_TERMINATED KW_BY linesIdnt=StringLiteral
-    ;
-
-tableFileFormat
-@init { msgs.push("table file format specification"); }
-@after { msgs.pop(); }
-    :
-      KW_STORED KW_AS KW_SEQUENCEFILE  
-      | KW_STORED KW_AS KW_TEXTFILE  
-      | KW_STORED KW_AS KW_RCFILE  
-      | KW_STORED KW_AS KW_ORCFILE 
-      | KW_STORED KW_AS KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
-      | KW_STORED KW_BY storageHandler=StringLiteral
-         (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
-      | KW_STORED KW_AS genericSpec=identifier
-    ;
-
-tableLocation
-@init { msgs.push("table location specification"); }
-@after { msgs.pop(); }
-    :
-      KW_LOCATION locn=StringLiteral 
-    ;
-
-columnNameTypeList
-@init { msgs.push("column name type list"); }
-@after { msgs.pop(); }
-    : columnNameType (COMMA columnNameType)* 
-    ;
-
-columnNameColonTypeList
-@init { msgs.push("column name type list"); }
-@after { msgs.pop(); }
-    : columnNameColonType (COMMA columnNameColonType)* 
-    ;
-
-columnNameList
-@init { msgs.push("column name list"); }
-@after { msgs.pop(); }
-    : columnName (COMMA columnName)* 
-    ;
-
-columnName
-@init { msgs.push("column name"); }
-@after { msgs.pop(); }
-    :
-      identifier
-    ;
-
-columnNameOrderList
-@init { msgs.push("column name order list"); }
-@after { msgs.pop(); }
-    : columnNameOrder (COMMA columnNameOrder)* 
-    ;
-
-skewedValueElement
-@init { msgs.push("skewed value element"); }
-@after { msgs.pop(); }
-    : 
-      skewedColumnValues
-     | skewedColumnValuePairList
-    ;
-
-skewedColumnValuePairList
-@init { msgs.push("column value pair list"); }
-@after { msgs.pop(); }
-    : skewedColumnValuePair (COMMA skewedColumnValuePair)* 
-    ;
-
-skewedColumnValuePair
-@init { msgs.push("column value pair"); }
-@after { msgs.pop(); }
-    : 
-      LPAREN colValues=skewedColumnValues RPAREN 
-    ;
-
-skewedColumnValues
-@init { msgs.push("column values"); }
-@after { msgs.pop(); }
-    : skewedColumnValue (COMMA skewedColumnValue)* 
-    ;
-
-skewedColumnValue
-@init { msgs.push("column value"); }
-@after { msgs.pop(); }
-    :
-      constant
-    ;
-
-skewedValueLocationElement
-@init { msgs.push("skewed value location element"); }
-@after { msgs.pop(); }
-    : 
-      skewedColumnValue
-     | skewedColumnValuePair
-    ;
-    
-columnNameOrder
-@init { msgs.push("column name order"); }
-@after { msgs.pop(); }
-    : identifier (asc=KW_ASC | desc=KW_DESC)?
-    ;
-
-columnNameCommentList
-@init { msgs.push("column name comment list"); }
-@after { msgs.pop(); }
-    : columnNameComment (COMMA columnNameComment)* 
-    ;
-
-columnNameComment
-@init { msgs.push("column name comment"); }
-@after { msgs.pop(); }
-    : colName=identifier (KW_COMMENT comment=StringLiteral)?
-    ;
-
-columnRefOrder
-@init { msgs.push("column order"); }
-@after { msgs.pop(); }
-    : expression (asc=KW_ASC | desc=KW_DESC)?
-    ;
-
-columnNameType
-@init { msgs.push("column specification"); }
-@after { msgs.pop(); }
-    : colName=identifier colType (KW_COMMENT comment=StringLiteral)?
-    ;
-
-columnNameColonType
-@init { msgs.push("column specification"); }
-@after { msgs.pop(); }
-    : colName=identifier COLON colType (KW_COMMENT comment=StringLiteral)?
-    ;
-
-colType
-@init { msgs.push("column type"); }
-@after { msgs.pop(); }
-    : type
-    ;
-
-colTypeList
-@init { msgs.push("column type list"); }
-@after { msgs.pop(); }
-    : colType (COMMA colType)* 
-    ;
-
-type
-    : primitiveType
-    | listType
-    | structType
-    | mapType
-    | unionType;
-
-primitiveType
-@init { msgs.push("primitive type specification"); }
-@after { msgs.pop(); }
-    : KW_TINYINT       
-    | KW_SMALLINT      
-    | KW_INT           
-    | KW_BIGINT        
-    | KW_BOOLEAN       
-    | KW_FLOAT         
-    | KW_DOUBLE        
-    | KW_DATE          
-    | KW_DATETIME      
-    | KW_TIMESTAMP     
-    | KW_STRING        
-    | KW_BINARY        
-    | KW_DECIMAL       
-    ;
-
-listType
-@init { msgs.push("list type"); }
-@after { msgs.pop(); }
-    : KW_ARRAY LESSTHAN type GREATERTHAN   
-    ;
-
-structType
-@init { msgs.push("struct type"); }
-@after { msgs.pop(); }
-    : KW_STRUCT LESSTHAN columnNameColonTypeList GREATERTHAN 
-    ;
-
-mapType
-@init { msgs.push("map type"); }
-@after { msgs.pop(); }
-    : KW_MAP LESSTHAN left=primitiveType COMMA right=type GREATERTHAN
-    ;
-
-unionType
-@init { msgs.push("uniontype type"); }
-@after { msgs.pop(); }
-    : KW_UNIONTYPE LESSTHAN colTypeList GREATERTHAN 
-    ;
-
-queryOperator
-@init { msgs.push("query operator"); }
-@after { msgs.pop(); }
-    : KW_UNION KW_ALL 
-    ;
-
-// select statement select ... from ... where ... group by ... order by ...
-queryStatementExpression
-    : 
-    queryStatement (queryOperator queryStatement)*
-    ;
-
-queryStatement
-    :
-    fromClause
-    ( b+=body )+ 
-    | regular_body
-    ;
-
-regular_body
-   :
-   insertClause
-   selectClause
-   fromClause
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? 
-   |
-   selectStatement
-   ;
-
-selectStatement
-   :
-   selectClause
-   fromClause
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? 
-   ;
-
-
-body
-   :
-   insertClause
-   selectClause
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? 
-   |
-   selectClause
-   whereClause?
-   groupByClause?
-   havingClause?
-   orderByClause?
-   clusterByClause?
-   distributeByClause?
-   sortByClause?
-   window_clause?
-   limitClause? 
-   ;
-
-insertClause
-@init { msgs.push("insert clause"); }
-@after { msgs.pop(); }
-   :
-     KW_INSERT KW_OVERWRITE destination ifNotExists? 
-   | KW_INSERT KW_INTO KW_TABLE tableOrPartition
-   ;
-
-destination
-@init { msgs.push("destination specification"); }
-@after { msgs.pop(); }
-   :
-     KW_LOCAL KW_DIRECTORY StringLiteral tableRowFormat? tableFileFormat? 
-   | KW_DIRECTORY StringLiteral 
-   | KW_TABLE tableOrPartition 
-   ;
-
-limitClause
-@init { msgs.push("limit clause"); }
-@after { msgs.pop(); }
-   :
-   KW_LIMIT num=Number 
-   ;
-
-    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
new file mode 100644
index 0000000..8eba9a4
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
@@ -0,0 +1,392 @@
+/**
+   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 HiveQLLexer;
+
+
+// Keywords
+
+KW_TRUE : 'TRUE';
+KW_FALSE : 'FALSE';
+KW_ALL : 'ALL';
+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_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_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_IGNORE: 'IGNORE';
+KW_PROTECTION: 'PROTECTION';
+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_DECIMAL: 'DECIMAL';
+KW_STRING: 'STRING';
+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_SEQUENCEFILE: 'SEQUENCEFILE';
+KW_TEXTFILE: 'TEXTFILE';
+KW_RCFILE: 'RCFILE';
+KW_ORCFILE: 'ORC';
+KW_INPUTFORMAT: 'INPUTFORMAT';
+KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
+KW_INPUTDRIVER: 'INPUTDRIVER';
+KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
+KW_OFFLINE: 'OFFLINE';
+KW_ENABLE: 'ENABLE';
+KW_DISABLE: 'DISABLE';
+KW_READONLY: 'READONLY';
+KW_NO_DROP: 'NO_DROP';
+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_EXPLAIN: 'EXPLAIN';
+KW_EXTENDED: 'EXTENDED';
+KW_FORMATTED: 'FORMATTED';
+KW_PRETTY: 'PRETTY';
+KW_DEPENDENCY: 'DEPENDENCY';
+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_CASE: 'CASE';
+KW_WHEN: 'WHEN';
+KW_THEN: 'THEN';
+KW_ELSE: 'ELSE';
+KW_END: 'END';
+KW_MAPJOIN: 'MAPJOIN';
+KW_STREAMTABLE: 'STREAMTABLE';
+KW_HOLD_DDLTIME: 'HOLD_DDLTIME';
+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_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_INNER: 'INNER';
+
+// 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)?
+    ;
+    
+Identifier
+    :
+    (Letter | Digit) (Letter | Digit | '_')*
+    | '`' RegexComponent+ '`'
+    ;
+
+CharSetName
+    :
+    '_' (Letter | Digit | '_' | '-' | '.' | ':' )+
+    ;
+
+WS  :
+(' '|'\r'|'\t'|'\n') -> skip
+    ;
+
+COMMENT
+  : '--' (~('\n'|'\r'))*  -> skip
+  ;
+
+
+


[2/5] TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
deleted file mode 100644
index a762720..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/HiveConverter.java
+++ /dev/null
@@ -1,1568 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.parser;
-
-import org.antlr.v4.runtime.ANTLRInputStream;
-import org.antlr.v4.runtime.CharStream;
-import org.antlr.v4.runtime.CommonTokenStream;
-import org.antlr.v4.runtime.tree.TerminalNodeImpl;
-import org.apache.commons.lang.math.NumberUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.algebra.*;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.engine.parser.HiveParser.TableAllColumnsContext;
-
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class HiveConverter extends HiveParserBaseVisitor<Expr> {
-  private static final Log LOG = LogFactory.getLog(HiveConverter.class.getName());
-  private HiveParser parser;
-
-  public Expr parse(String sql) {
-    HiveLexer lexer = new HiveLexer(new ANTLRNoCaseStringStream(sql));
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
-    parser = new HiveParser(tokens);
-    parser.setBuildParseTree(true);
-
-    HiveParser.StatementContext context;
-    try {
-      context = parser.statement();
-    } catch (SQLParseError e) {
-      throw new SQLSyntaxError(e);
-    }
-
-    return visit(context);
-  }
-
-  @Override
-  public Expr visitStatement(HiveParser.StatementContext ctx) {
-    return visitExecStatement(ctx.execStatement());
-  }
-
-  @Override
-  public Expr visitQueryStatement(HiveParser.QueryStatementContext ctx) {
-    Expr current = null;
-
-    if (ctx.body != null) {
-      current = visitBody(ctx.body(0));
-    }
-
-    if (ctx.regular_body() != null) {
-      current = visitRegular_body(ctx.regular_body());
-    }
-
-    return current;
-  }
-
-  @Override
-  public Expr visitBody(HiveParser.BodyContext ctx) {
-
-    Expr current = null;
-    Insert insert = null;
-
-    Projection select = null;
-
-    if (ctx.insertClause() != null) {
-      insert = visitInsertClause(ctx.insertClause());
-    }
-
-    if (ctx.selectClause() != null) {
-      select = (Projection) visitSelectClause(ctx.selectClause());
-      if (ctx.selectClause().KW_DISTINCT() != null) {
-        select.setDistinct();
-      }
-
-    }
-
-    for (int i = 0; i < ctx.getParent().getChildCount(); i++) {
-      if (ctx.getParent().getChild(i) instanceof HiveParser.FromClauseContext) {
-        HiveParser.FromClauseContext fromClauseContext = (HiveParser.FromClauseContext) ctx.getParent().getChild(i);
-        Expr from = visitFromClause(fromClauseContext);
-        current = from;
-      }
-    }
-
-    if (ctx.whereClause() != null) {
-      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
-      where.setChild(current);
-      current = where;
-    }
-
-    if (ctx.groupByClause() != null) {
-      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
-      aggregation.setChild(current);
-      current = aggregation;
-
-      if (ctx.havingClause() != null) {
-        Expr havingCondition = visitHavingClause(ctx.havingClause());
-        Having having = new Having(havingCondition);
-        having.setChild(current);
-        current = having;
-      }
-    }
-
-    if (ctx.orderByClause() != null) {
-      Sort sort = visitOrderByClause(ctx.orderByClause());
-      sort.setChild(current);
-      current = sort;
-    }
-
-    if (ctx.clusterByClause() != null) {
-      visitClusterByClause(ctx.clusterByClause());
-    }
-
-    if (ctx.distributeByClause() != null) {
-      visitDistributeByClause(ctx.distributeByClause());
-    }
-
-    if (ctx.sortByClause() != null) {
-      Sort sort = visitSortByClause(ctx.sortByClause());
-      sort.setChild(current);
-      current = sort;
-    }
-
-    if (ctx.window_clause() != null) {
-      Expr window = visitWindow_clause(ctx.window_clause());
-    }
-
-    if (ctx.limitClause() != null) {
-      Limit limit = visitLimitClause(ctx.limitClause());
-      limit.setChild(current);
-      current = limit;
-    }
-
-    Projection projection = new Projection();
-    projection.setNamedExprs(select.getNamedExprs());
-
-    if (current != null)
-      projection.setChild(current);
-
-    if (select.isDistinct())
-      projection.setDistinct();
-
-
-    if (insert != null) {
-      insert.setSubQuery(projection);
-      current = insert;
-    } else {
-      current = projection;
-    }
-
-    return current;
-  }
-
-  @Override
-  public Expr visitRegular_body(HiveParser.Regular_bodyContext ctx) {
-    Expr current = null;
-    Insert insert = null;
-
-    if (ctx.selectStatement() != null) {
-      current = visitSelectStatement(ctx.selectStatement());
-    } else {
-      Projection select = null;
-
-      if (ctx.insertClause() != null) {
-        insert = visitInsertClause(ctx.insertClause());
-      }
-
-      if (ctx.selectClause() != null) {
-        select = (Projection) visitSelectClause(ctx.selectClause());
-        if (ctx.selectClause().KW_DISTINCT() != null) {
-          select.setDistinct();
-        }
-
-      }
-
-      if (ctx.fromClause() != null) {
-        Expr from = visitFromClause(ctx.fromClause());
-        current = from;
-      }
-
-      if (ctx.whereClause() != null) {
-        Selection where = new Selection(visitWhereClause(ctx.whereClause()));
-        where.setChild(current);
-        current = where;
-      }
-
-      if (ctx.groupByClause() != null) {
-        Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
-        aggregation.setChild(current);
-        current = aggregation;
-
-        if (ctx.havingClause() != null) {
-          Expr havingCondition = visitHavingClause(ctx.havingClause());
-          Having having = new Having(havingCondition);
-          having.setChild(current);
-          current = having;
-        }
-      }
-
-      if (ctx.orderByClause() != null) {
-        Sort sort = visitOrderByClause(ctx.orderByClause());
-        sort.setChild(current);
-        current = sort;
-      }
-
-      if (ctx.clusterByClause() != null) {
-        visitClusterByClause(ctx.clusterByClause());
-      }
-
-      if (ctx.distributeByClause() != null) {
-        visitDistributeByClause(ctx.distributeByClause());
-      }
-
-      if (ctx.sortByClause() != null) {
-        Sort sort = visitSortByClause(ctx.sortByClause());
-        sort.setChild(current);
-        current = sort;
-      }
-
-      if (ctx.window_clause() != null) {
-        Expr window = visitWindow_clause(ctx.window_clause());
-      }
-
-      if (ctx.limitClause() != null) {
-        Limit limit = visitLimitClause(ctx.limitClause());
-        limit.setChild(current);
-        current = limit;
-      }
-
-      Projection projection = new Projection();
-      projection.setNamedExprs(select.getNamedExprs());
-
-      if (current != null)
-        projection.setChild(current);
-
-      if (select.isDistinct())
-        projection.setDistinct();
-
-      if (insert != null) {
-        insert.setSubQuery(projection);
-        current = insert;
-      } else {
-        current = projection;
-      }
-
-
-    }
-    return current;
-  }
-
-  /**
-   * This method implemented for parsing union all clause.
-   *
-   * @param ctx
-   * @return
-   */
-  @Override
-  public Expr visitQueryStatementExpression(HiveParser.QueryStatementExpressionContext ctx) {
-    Expr left = null, right = null, current = null;
-    if (ctx.queryStatement() != null) {
-      if (ctx.queryStatement().size() == 1)
-        return visitQueryStatement(ctx.queryStatement(0));
-
-      for (int i = 0; i < ctx.queryStatement().size(); i++) {
-        if (i == 0)
-          current = visitQueryStatement(ctx.queryStatement(i));
-        else
-          left = current;
-
-        if (i > 0) {
-          right = visitQueryStatement(ctx.queryStatement(i));
-          current = new SetOperation(OpType.Union, left, right, false);
-        }
-      }
-    }
-    return current;
-  }
-
-  @Override
-  public Expr visitSelectStatement(HiveParser.SelectStatementContext ctx) {
-    Expr current = null;
-
-    Projection select = (Projection) visitSelectClause(ctx.selectClause());
-
-    if (ctx.selectClause().KW_DISTINCT() != null) {
-      select.setDistinct();
-    }
-
-    Expr from = visitFromClause(ctx.fromClause());
-    current = from;
-
-    if (ctx.whereClause() != null) {
-      Selection where = new Selection(visitWhereClause(ctx.whereClause()));
-      where.setChild(current);
-      current = where;
-    }
-
-    if (ctx.groupByClause() != null) {
-      Aggregation aggregation = visitGroupByClause(ctx.groupByClause());
-      aggregation.setChild(current);
-      current = aggregation;
-
-      if (ctx.havingClause() != null) {
-        Expr havingCondition = visitHavingClause(ctx.havingClause());
-        Having having = new Having(havingCondition);
-        having.setChild(current);
-        current = having;
-      }
-    }
-
-    if (ctx.orderByClause() != null) {
-      Sort sort = visitOrderByClause(ctx.orderByClause());
-      sort.setChild(current);
-      current = sort;
-    }
-
-    if (ctx.clusterByClause() != null) {
-      visitClusterByClause(ctx.clusterByClause());
-    }
-
-    if (ctx.distributeByClause() != null) {
-      visitDistributeByClause(ctx.distributeByClause());
-    }
-
-    if (ctx.sortByClause() != null) {
-      Sort sort = visitSortByClause(ctx.sortByClause());
-      sort.setChild(current);
-      current = sort;
-    }
-
-    if (ctx.window_clause() != null) {
-      Expr window = visitWindow_clause(ctx.window_clause());
-    }
-
-    if (ctx.limitClause() != null) {
-      Limit limit = visitLimitClause(ctx.limitClause());
-      limit.setChild(current);
-      current = limit;
-    }
-
-    Projection projection = new Projection();
-    projection.setNamedExprs(select.getNamedExprs());
-
-    if (current != null)
-      projection.setChild(current);
-
-    if (select.isDistinct())
-      projection.setDistinct();
-
-    current = projection;
-
-    return current;
-  }
-
-  @Override
-  public Expr visitFromClause(HiveParser.FromClauseContext ctx) {
-    return visitJoinSource(ctx.joinSource());
-  }
-
-  @Override
-  public Expr visitJoinSource(HiveParser.JoinSourceContext ctx) {
-    Expr[] relations = null;
-    RelationList relationList = null;
-
-    if (ctx.fromSource() != null) {
-      int fromCount = ctx.fromSource().size();
-      int uniqueJoinCount = ctx.uniqueJoinSource().size();
-
-      relations = new Expr[1];
-
-      Join current = null, parent = null;
-      JoinType type = null;
-      Expr left = null, right = null, condition = null;
-
-
-      if (fromCount == 1) {
-        relations[0] = visitFromSource(ctx.fromSource(0));
-      } else {
-        left = visitFromSource((HiveParser.FromSourceContext) ctx.getChild(0));
-
-        for (int i = 1; i < ctx.getChildCount(); i++) {
-          type = null;
-          right = null;
-          condition = null;
-
-          if (ctx.getChild(i) instanceof HiveParser.JoinTokenContext) {
-            type = getJoinType((HiveParser.JoinTokenContext) ctx.getChild(i));
-            if (i > 1)
-              left = parent;
-
-            if (i + 1 < ctx.getChildCount() && ctx.getChild(i + 1) instanceof HiveParser.FromSourceContext) {
-              right = visitFromSource((HiveParser.FromSourceContext) ctx.getChild(i + 1));
-            }
-
-            if (i + 3 < ctx.getChildCount() && ctx.getChild(i + 3) instanceof HiveParser.ExpressionContext) {
-              condition = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 3));
-            }
-
-            if (type != null) {
-              current = new Join(type);
-              current.setLeft(left);
-              current.setRight(right);
-
-              if (condition != null)
-                current.setQual(condition);
-
-              parent = current;
-            }
-          }
-
-        }
-        relations[0] = current;
-      }
-
-      //TODO: implement unique join.
-      relationList = new RelationList(relations);
-    }
-
-    return relationList;
-  }
-
-  public JoinType getJoinType(HiveParser.JoinTokenContext context) {
-    JoinType type = JoinType.INNER;
-
-    if (context.KW_INNER() != null) {
-      type = JoinType.INNER;
-    }
-
-    if (context.KW_LEFT() != null && context.KW_OUTER() != null) {
-      type = JoinType.LEFT_OUTER;
-    }
-
-    if (context.KW_RIGHT() != null && context.KW_OUTER() != null) {
-      type = JoinType.RIGHT_OUTER;
-    }
-
-    if (context.KW_CROSS() != null) {
-      type = JoinType.CROSS;
-    }
-
-    if (context.KW_FULL() != null) {
-      type = JoinType.FULL_OUTER;
-    }
-
-    if (context.KW_SEMI() != null) {
-      type = null;
-    }
-    return type;
-  }
-
-  @Override
-  public Expr visitFromSource(HiveParser.FromSourceContext ctx) {
-    Expr current = null;
-
-    if (ctx.Identifier() != null && ctx.LPAREN() != null) {
-      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
-    }
-
-    if (ctx.tableSource() != null) {
-      current = visitTableSource(ctx.tableSource());
-    }
-
-    if (ctx.subQuerySource() != null) {
-      current = visitSubQuerySource(ctx.subQuerySource());
-
-      String tableAlias = "";
-      for (int i = 0; i < ctx.subQuerySource().getChildCount(); i++) {
-        if (ctx.subQuerySource().getChild(i) instanceof HiveParser.IdentifierContext) {
-          tableAlias = (ctx.subQuerySource().getChild(i)).getText();
-        }
-      }
-
-      TablePrimarySubQuery subQuery = new TablePrimarySubQuery(tableAlias, current);
-      current = subQuery;
-    }
-    // TODO: implement lateralView
-
-    return current;
-  }
-
-  @Override
-  public Expr visitSubQuerySource(HiveParser.SubQuerySourceContext ctx) {
-    Expr current = visitQueryStatementExpression(ctx.queryStatementExpression());
-    return current;
-  }
-
-  @Override
-  public Expr visitTableSource(HiveParser.TableSourceContext ctx) {
-    String tableName = "", alias = "";
-
-    if (ctx.tableName() != null)
-      tableName = ctx.tableName().getText();
-
-    if (ctx.alias != null) {
-      alias = ctx.alias.getText();
-      for (String token : HiveParser.tokenNames) {
-        if (token.replaceAll("'", "").equalsIgnoreCase(alias))
-          alias = "";
-      }
-    }
-
-    Relation relation = new Relation(tableName);
-    if (!alias.equals(""))
-      relation.setAlias(alias);
-
-    return relation;
-  }
-
-  @Override
-  public Expr visitSelectList(HiveParser.SelectListContext ctx) {
-    Expr current = null;
-    Projection projection = new Projection();
-    NamedExpr[] targets = new NamedExpr[ctx.selectItem().size()];
-    for (int i = 0; i < targets.length; i++) {
-      targets[i] = visitSelectItem(ctx.selectItem(i));
-    }
-
-    projection.setNamedExprs(targets);
-    current = projection;
-    return current;
-  }
-
-  @Override
-  public NamedExpr visitSelectItem(HiveParser.SelectItemContext ctx) {
-    NamedExpr target = null;
-
-    if (ctx.selectExpression() != null) {
-      target = new NamedExpr(visitSelectExpression(ctx.selectExpression()));
-    } else if (ctx.window_specification() != null) {
-      // TODO: if there is a window specification clause, we should handle it properly.
-    }
-
-    if (ctx.identifier().size() > 0 && target != null) {
-      target.setAlias(ctx.identifier(0).getText());
-    }
-    return target;
-  }
-
-  @Override
-  public Expr visitSelectExpression(HiveParser.SelectExpressionContext ctx) {
-    Expr current = null;
-
-    if (ctx.tableAllColumns() != null) {
-      current = visitTableAllColumns(ctx.tableAllColumns());
-    } else {
-      if (ctx.expression() != null) {
-        current = visitExpression(ctx.expression());
-      }
-    }
-
-    return current;
-  }
-
-  @Override
-  public Expr visitTableAllColumns(TableAllColumnsContext ctx) {
-    QualifiedAsteriskExpr target = new QualifiedAsteriskExpr();
-    if (ctx.tableName() != null) {
-      target.setQualifier(ctx.tableName().getText());
-    }
-
-    return target;
-  }
-
-  @Override
-  public Expr visitExpression(HiveParser.ExpressionContext ctx) {
-    Expr current = visitPrecedenceOrExpression(ctx.precedenceOrExpression());
-    return current;
-  }
-
-  @Override
-  public Expr visitPrecedenceOrExpression(HiveParser.PrecedenceOrExpressionContext ctx) {
-    Expr current = null, left = null, right = null;
-
-    for (int i = 0; i < ctx.precedenceAndExpression().size(); i++) {
-      if (i == 0) {
-        left = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
-        current = left;
-      } else {
-        left = current;
-        right = visitPrecedenceAndExpression(ctx.precedenceAndExpression(i));
-        current = new BinaryOperator(OpType.Or, left, right);
-      }
-
-    }
-    return current;
-  }
-
-  /**
-   * This method parse AND expressions at WHERE clause.
-   * And this convert 'x BETWEEN y AND z' expression into 'x >= y AND x <= z' expression
-   * because Tajo doesn't provide 'BETWEEN' expression.
-   *
-   * @param ctx
-   * @return
-   */
-  @Override
-  public Expr visitPrecedenceAndExpression(HiveParser.PrecedenceAndExpressionContext ctx) {
-    Expr current = null, left = null, right = null;
-
-    for (int i = 0; i < ctx.precedenceNotExpression().size(); i++) {
-      Expr min = null, max = null;
-
-      if (ctx.precedenceNotExpression(i).precedenceEqualExpression() != null) {
-        HiveParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceNotExpression(i)
-            .precedenceEqualExpression();
-        if (expressionContext.KW_BETWEEN() != null) {
-
-          if (expressionContext.min != null) {
-            min = visitPrecedenceBitwiseOrExpression(expressionContext.min);
-          }
-
-          if (expressionContext.max != null) {
-            max = visitPrecedenceBitwiseOrExpression(expressionContext.max);
-          }
-        }
-      }
-
-      if (min != null && max != null) {
-        left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
-        if (left != null) {
-          if (i == 0) {
-            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
-            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
-            current = new BinaryOperator(OpType.And, minOperator, maxOperator);
-          } else {
-            BinaryOperator minOperator = new BinaryOperator(OpType.GreaterThanOrEquals, left, min);
-            current = new BinaryOperator(OpType.And, current, minOperator);
-
-            BinaryOperator maxOperator = new BinaryOperator(OpType.LessThanOrEquals, left, max);
-            current = new BinaryOperator(OpType.And, current, maxOperator);
-          }
-        }
-      } else {
-        if (i == 0) {
-          left = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
-          current = left;
-        } else {
-          left = current;
-          right = visitPrecedenceNotExpression(ctx.precedenceNotExpression(i));
-          current = new BinaryOperator(OpType.And, left, right);
-        }
-      }
-    }
-    return current;
-  }
-
-  @Override
-  public Expr visitPrecedenceNotExpression(HiveParser.PrecedenceNotExpressionContext ctx) {
-    HiveParser.PrecedenceEqualExpressionContext expressionContext = ctx.precedenceEqualExpression();
-    Expr current = visitPrecedenceEqualExpression(expressionContext);
-    return current;
-  }
-
-  /**
-   * This method parse operators for equals expressions as follows:
-   * =, <>, !=, >=, >, <=, <, IN, NOT IN, LIKE, REGEXP, RLIKE
-   * <p/>
-   * In this case, this make RuntimeException>
-   *
-   * @param ctx
-   * @return
-   */
-  @Override
-  public Expr visitPrecedenceEqualExpression(HiveParser.PrecedenceEqualExpressionContext ctx) {
-    Expr current = null, left = null, right = null, min = null, max = null;
-    OpType type = null;
-    boolean isNot = false, isIn = false;
-    for (int i = 0; i < ctx.getChildCount(); i++) {
-      if (ctx.getChild(i) instanceof HiveParser.PrecedenceBitwiseOrExpressionContext) {
-        if (i == 0) {
-          left = visitPrecedenceBitwiseOrExpression((HiveParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
-        } else {
-          right = visitPrecedenceBitwiseOrExpression((HiveParser.PrecedenceBitwiseOrExpressionContext) ctx.getChild(i));
-        }
-      } else if (ctx.getChild(i) instanceof HiveParser.ExpressionsContext) {
-        right = visitExpressions((HiveParser.ExpressionsContext) ctx.getChild(i));
-      } else if (ctx.getChild(i) instanceof TerminalNodeImpl) {
-        int symbolType = ((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType();
-        switch (symbolType) {
-          case HiveLexer.KW_NOT:
-            isNot = true;
-            break;
-          case HiveLexer.KW_IN:
-            isIn = true;
-            break;
-          default:
-            break;
-        }
-      } else if (ctx.getChild(i) instanceof HiveParser.PrecedenceEqualOperatorContext
-          || ctx.getChild(i) instanceof HiveParser.PrecedenceEqualNegatableOperatorContext) {
-        String keyword = ctx.getChild(i).getText().toUpperCase();
-
-        if (keyword.equals(">")) {
-          type = OpType.GreaterThan;
-        } else if (keyword.equals("<=>")) {
-          throw new RuntimeException("Unexpected operator : <=>");
-        } else if (keyword.equals("=")) {
-          type = OpType.Equals;
-        } else if (keyword.equals("<=")) {
-          type = OpType.LessThanOrEquals;
-        } else if (keyword.equals("<")) {
-          type = OpType.LessThan;
-        } else if (keyword.equals(">=")) {
-          type = OpType.GreaterThanOrEquals;
-        } else if (keyword.equals("<>")) {
-          type = OpType.NotEquals;
-        } else if (keyword.equals("!=")) {
-          type = OpType.NotEquals;
-        } else if (keyword.equals("REGEXP")) {
-          type = OpType.Regexp;
-        } else if (keyword.equals("RLIKE")) {
-          type = OpType.Regexp;
-        } else if (keyword.equals("LIKE")) {
-          type = OpType.LikePredicate;
-        }
-      }
-    }
-
-    if (type != null && right != null) {
-      if (type.equals(OpType.LikePredicate)) {
-        PatternMatchPredicate like = new PatternMatchPredicate(OpType.LikePredicate,
-            isNot, left, right);
-        current = like;
-      } else if (type.equals(OpType.Regexp)) {
-        PatternMatchPredicate regex = new PatternMatchPredicate(OpType.Regexp, isNot, left, right);
-        current = regex;
-      } else {
-        BinaryOperator binaryOperator = new BinaryOperator(type, left, right);
-        current = binaryOperator;
-      }
-    } else if (isIn) {
-      InPredicate inPredicate = new InPredicate(left, right, isNot);
-      current = inPredicate;
-    } else {
-      current = left;
-    }
-
-    return current;
-  }
-
-  @Override
-  public ValueListExpr visitExpressions(HiveParser.ExpressionsContext ctx) {
-    int size = ctx.expression().size();
-    Expr[] exprs = new Expr[size];
-    for (int i = 0; i < size; i++) {
-      exprs[i] = visitExpression(ctx.expression(i));
-    }
-    return new ValueListExpr(exprs);
-  }
-
-  @Override
-  public Expr visitPrecedenceBitwiseOrExpression(HiveParser.PrecedenceBitwiseOrExpressionContext ctx) {
-    int expressionCount = ctx.precedenceAmpersandExpression().size();
-
-    Expr current = null, left = null, right = null, parentLeft, parentRight;
-    OpType type = null, parentType = null;
-
-    for (int i = 0; i < expressionCount; i += 2) {
-      int operatorIndex = (i == 0) ? 0 : i - 1;
-
-      if (ctx.precedenceBitwiseOrOperator(operatorIndex) != null) {
-        type = getPrecedenceBitwiseOrOperator(ctx.precedenceBitwiseOrOperator(operatorIndex));
-      }
-
-      if (i == 0) {
-        left = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
-        if (ctx.precedenceAmpersandExpression(i + 1) != null)
-          right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i + 1));
-      } else {
-        parentType = getPrecedenceBitwiseOrOperator((ctx.precedenceBitwiseOrOperator(operatorIndex - 1)));
-        parentLeft = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 2));
-        parentRight = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i - 1));
-        left = new BinaryOperator(parentType, parentLeft, parentRight);
-        right = visitPrecedenceAmpersandExpression(ctx.precedenceAmpersandExpression(i));
-      }
-
-      if (right != null) {
-        current = new BinaryOperator(type, left, right);
-      } else {
-        current = left;
-      }
-    }
-    return current;
-  }
-
-  public OpType getPrecedenceBitwiseOrOperator(HiveParser.PrecedenceBitwiseOrOperatorContext ctx) {
-    OpType type = null;
-    // TODO: It needs to consider how to support.
-    return type;
-  }
-
-  @Override
-  public Expr visitPrecedenceAmpersandExpression(HiveParser.PrecedenceAmpersandExpressionContext ctx) {
-    int expressionCount = ctx.precedencePlusExpression().size();
-
-    Expr current = null, left = null, right = null, parentLeft, parentRight;
-    OpType type = null, parentType = null;
-
-    for (int i = 0; i < expressionCount; i += 2) {
-      int operatorIndex = (i == 0) ? 0 : i - 1;
-
-      if (ctx.precedenceAmpersandOperator(operatorIndex) != null) {
-        type = getPrecedenceAmpersandOperator(ctx.precedenceAmpersandOperator(operatorIndex));
-      }
-
-      if (i == 0) {
-        left = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
-        if (ctx.precedencePlusExpression(i + 1) != null)
-          right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i + 1));
-      } else {
-        parentType = getPrecedenceAmpersandOperator((ctx.precedenceAmpersandOperator(operatorIndex - 1)));
-        parentLeft = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 2));
-        parentRight = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i - 1));
-        left = new BinaryOperator(parentType, parentLeft, parentRight);
-        right = visitPrecedencePlusExpression(ctx.precedencePlusExpression(i));
-      }
-
-      if (right != null) {
-        current = new BinaryOperator(type, left, right);
-      } else {
-        current = left;
-      }
-    }
-    return current;
-  }
-
-  public OpType getPrecedenceAmpersandOperator(HiveParser.PrecedenceAmpersandOperatorContext ctx) {
-    OpType type = null;
-    // TODO: It needs to consider how to support.
-    return type;
-  }
-
-  @Override
-  public Expr visitPrecedencePlusExpression(HiveParser.PrecedencePlusExpressionContext ctx) {
-    int expressionCount = ctx.precedenceStarExpression().size();
-
-    Expr current = null, left = null, right = null, parentLeft, parentRight;
-    OpType type = null, parentType = null;
-
-    for (int i = 0; i < expressionCount; i += 2) {
-      int operatorIndex = (i == 0) ? 0 : i - 1;
-
-      if (ctx.precedencePlusOperator(operatorIndex) != null) {
-        type = getPrecedencePlusOperator(ctx.precedencePlusOperator(operatorIndex));
-      }
-
-      if (i == 0) {
-        left = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
-        if (ctx.precedenceStarExpression(i + 1) != null)
-          right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i + 1));
-      } else {
-        parentType = getPrecedencePlusOperator((ctx.precedencePlusOperator(operatorIndex - 1)));
-        parentLeft = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 2));
-        parentRight = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i - 1));
-        left = new BinaryOperator(parentType, parentLeft, parentRight);
-        right = visitPrecedenceStarExpression(ctx.precedenceStarExpression(i));
-      }
-
-      if (right != null) {
-        current = new BinaryOperator(type, left, right);
-      } else {
-        current = left;
-      }
-    }
-    return current;
-  }
-
-  public OpType getPrecedencePlusOperator(HiveParser.PrecedencePlusOperatorContext ctx) {
-    OpType type = null;
-
-    if (ctx.MINUS() != null) {
-      type = OpType.Minus;
-    } else if (ctx.PLUS() != null) {
-      type = OpType.Plus;
-    }
-
-    return type;
-  }
-
-  @Override
-  public Expr visitPrecedenceStarExpression(HiveParser.PrecedenceStarExpressionContext ctx) {
-    int expressionCount = ctx.precedenceBitwiseXorExpression().size();
-
-    Expr current = null, left = null, right = null, parentLeft, parentRight;
-    OpType type = null, parentType = null;
-
-    for (int i = 0; i < expressionCount; i += 2) {
-      int operatorIndex = (i == 0) ? 0 : i - 1;
-
-      if (ctx.precedenceStarOperator(operatorIndex) != null) {
-        type = getPrecedenceStarOperator(ctx.precedenceStarOperator(operatorIndex));
-      }
-
-      if (i == 0) {
-        left = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
-        if (ctx.precedenceBitwiseXorExpression(i + 1) != null)
-          right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i + 1));
-      } else {
-        parentType = getPrecedenceStarOperator((ctx.precedenceStarOperator(operatorIndex - 1)));
-        parentLeft = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 2));
-        parentRight = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i - 1));
-        left = new BinaryOperator(parentType, parentLeft, parentRight);
-        right = visitPrecedenceBitwiseXorExpression(ctx.precedenceBitwiseXorExpression(i));
-      }
-
-      if (right != null) {
-        current = new BinaryOperator(type, left, right);
-      } else {
-        current = left;
-      }
-    }
-
-    return current;
-  }
-
-  public OpType getPrecedenceStarOperator(HiveParser.PrecedenceStarOperatorContext ctx) {
-    OpType type = null;
-
-    if (ctx.DIV() != null) {
-      type = OpType.Divide;
-    } else if (ctx.DIVIDE() != null) {
-      type = OpType.Divide;
-    } else if (ctx.MOD() != null) {
-      type = OpType.Modular;
-    } else if (ctx.STAR() != null) {
-      type = OpType.Multiply;
-    }
-
-    return type;
-  }
-
-  @Override
-  public Expr visitPrecedenceBitwiseXorExpression(HiveParser.PrecedenceBitwiseXorExpressionContext ctx) {
-    int expressionCount = ctx.precedenceUnarySuffixExpression().size();
-
-    Expr current = null, left = null, right = null, parentLeft, parentRight;
-    OpType type = null, parentType = null;
-
-    for (int i = 0; i < expressionCount; i += 2) {
-      int operatorIndex = (i == 0) ? 0 : i - 1;
-
-      if (ctx.precedenceBitwiseXorOperator(operatorIndex) != null) {
-        type = getPrecedenceBitwiseXorOperator(ctx.precedenceBitwiseXorOperator(operatorIndex));
-      }
-
-      if (i == 0) {
-        left = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
-        if (ctx.precedenceUnarySuffixExpression(i + 1) != null)
-          right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i + 1));
-      } else {
-        parentType = getPrecedenceBitwiseXorOperator((ctx.precedenceBitwiseXorOperator(operatorIndex - 1)));
-        parentLeft = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 2));
-        parentRight = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i - 1));
-        left = new BinaryOperator(parentType, parentLeft, parentRight);
-        right = visitPrecedenceUnarySuffixExpression(ctx.precedenceUnarySuffixExpression(i));
-      }
-
-      if (right != null) {
-        current = new BinaryOperator(type, left, right);
-      } else {
-        current = left;
-      }
-    }
-
-    return current;
-  }
-
-  public OpType getPrecedenceBitwiseXorOperator(HiveParser.PrecedenceBitwiseXorOperatorContext ctx) {
-    OpType type = null;
-    // TODO: It needs to consider how to support.
-
-    return type;
-  }
-
-  @Override
-  public Expr visitPrecedenceUnarySuffixExpression(HiveParser.PrecedenceUnarySuffixExpressionContext ctx) {
-    Expr current = visitPrecedenceUnaryPrefixExpression(ctx.precedenceUnaryPrefixExpression());
-
-    if (ctx.nullCondition() != null) {
-      boolean isNot = ctx.nullCondition().KW_NOT() == null ? false : true;
-      IsNullPredicate isNullPredicate = new IsNullPredicate(isNot, (ColumnReferenceExpr) current);
-      current = isNullPredicate;
-    }
-
-    return current;
-  }
-
-  @Override
-  public Expr visitPrecedenceUnaryPrefixExpression(HiveParser.PrecedenceUnaryPrefixExpressionContext ctx) {
-    Expr current = visitPrecedenceFieldExpression(ctx.precedenceFieldExpression());
-    return current;
-  }
-
-  @Override
-  public Expr visitNullCondition(HiveParser.NullConditionContext ctx) {
-    return new NullLiteral();
-  }
-
-  @Override
-  public Expr visitPrecedenceFieldExpression(HiveParser.PrecedenceFieldExpressionContext ctx) {
-    Expr current = visitAtomExpression(ctx.atomExpression());
-
-    if (ctx.DOT().size() > 0) {
-      ColumnReferenceExpr column = new ColumnReferenceExpr(ctx.identifier(0).getText());
-      ColumnReferenceExpr table = (ColumnReferenceExpr) current;
-      column.setQualifier(table.getName());
-      current = column;
-    }
-    return current;
-  }
-
-  @Override
-  public Expr visitAtomExpression(HiveParser.AtomExpressionContext ctx) {
-    Expr current = null;
-
-    if (ctx.KW_NULL() != null) {
-      current = new NullLiteral();
-    }
-    if (ctx.constant() != null) {
-      current = visitConstant(ctx.constant());
-    }
-    if (ctx.function() != null) {
-      current = visitFunction(ctx.function());
-    }
-    if (ctx.castExpression() != null) {
-      current = visitCastExpression(ctx.castExpression());
-    }
-    if (ctx.caseExpression() != null) {
-      current = visitCaseExpression(ctx.caseExpression());
-    }
-    if (ctx.whenExpression() != null) {
-      current = visitWhenExpression(ctx.whenExpression());
-    }
-    if (ctx.tableOrColumn() != null) {
-      current = visitTableOrColumn(ctx.tableOrColumn());
-    } else {
-      if (ctx.LPAREN() != null && ctx.RPAREN() != null) {
-        current = visitExpression(ctx.expression());
-      }
-    }
-
-    return current;
-  }
-
-  @Override
-  public Expr visitTableOrColumn(HiveParser.TableOrColumnContext ctx) {
-    ColumnReferenceExpr columnReferenceExpr = new ColumnReferenceExpr(ctx.identifier().getText());
-    return columnReferenceExpr;
-  }
-
-  @Override
-  public Expr visitIdentifier(HiveParser.IdentifierContext ctx) {
-    Expr current = null;
-
-    if (ctx.nonReserved() != null) {
-      current = new LiteralValue(ctx.nonReserved().getText(), LiteralValue.LiteralType.String);
-    } else {
-      current = new LiteralValue(ctx.Identifier().getText(), LiteralValue.LiteralType.String);
-    }
-
-    return current;
-  }
-
-  @Override
-  public LiteralValue visitConstant(HiveParser.ConstantContext ctx) {
-    LiteralValue literalValue = null;
-
-    if (ctx.StringLiteral() != null) {
-      String value = ctx.StringLiteral().getText();
-      String strValue = "";
-      if ((value.startsWith("'") && value.endsWith("'")) || value.startsWith("\"") && value.endsWith("\"")) {
-        strValue = value.substring(1, value.length() - 1);
-      } else {
-        strValue = value;
-      }
-
-      literalValue = new LiteralValue(strValue, LiteralValue.LiteralType.String);
-    } else if (ctx.TinyintLiteral() != null) {
-      literalValue = new LiteralValue(ctx.TinyintLiteral().getSymbol().getText(),
-          LiteralValue.LiteralType.Unsigned_Integer);
-    } else if (ctx.BigintLiteral() != null) {
-      literalValue = new LiteralValue(ctx.BigintLiteral().getSymbol().getText(),
-          LiteralValue.LiteralType.Unsigned_Large_Integer);
-    } else if (ctx.DecimalLiteral() != null) {
-      literalValue = new LiteralValue(ctx.DecimalLiteral().getSymbol().getText(),
-          LiteralValue.LiteralType.Unsigned_Integer);
-    } else if (ctx.Number() != null) {
-      try {
-        float floatValue = NumberUtils.createFloat(ctx.getText());
-        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Float);
-      } catch (NumberFormatException nf) {
-      }
-
-      // TODO: double type
-
-      try {
-        BigInteger bigIntegerVallue = NumberUtils.createBigInteger(ctx.getText());
-        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(),
-            LiteralValue.LiteralType.Unsigned_Large_Integer);
-      } catch (NumberFormatException nf) {
-      }
-
-      try {
-        int intValue = NumberUtils.createInteger(ctx.getText());
-        literalValue = new LiteralValue(ctx.Number().getSymbol().getText(), LiteralValue.LiteralType.Unsigned_Integer);
-      } catch (NumberFormatException nf) {
-      }
-
-    } else if (ctx.SmallintLiteral() != null) {
-      literalValue = new LiteralValue(ctx.SmallintLiteral().getSymbol().getText(),
-          LiteralValue.LiteralType.Unsigned_Integer);
-    } else if (ctx.booleanValue() != null) {
-      // TODO: boolean type
-    }
-
-    return literalValue;
-  }
-
-  @Override
-  public Expr visitFunction(HiveParser.FunctionContext ctx) {
-    Expr current = null;
-    String signature = ctx.functionName().getText();
-
-    boolean isDistinct = false;
-    if (ctx.getChild(2) != null) {
-      if (ctx.getChild(2) instanceof TerminalNodeImpl && ctx.getChild(2).getText().equalsIgnoreCase("DISTINCT")) {
-        isDistinct = true;
-      }
-    }
-
-    if (signature.equalsIgnoreCase("MIN")
-        || signature.equalsIgnoreCase("MAX")
-        || signature.equalsIgnoreCase("SUM")
-        || signature.equalsIgnoreCase("AVG")
-        || signature.equalsIgnoreCase("COUNT")
-        ) {
-      if (ctx.selectExpression().size() > 1) {
-        throw new RuntimeException("Exactly expected one argument.");
-      }
-
-      if (ctx.selectExpression().size() == 0) {
-        CountRowsFunctionExpr countRowsFunctionExpr = new CountRowsFunctionExpr();
-        current = countRowsFunctionExpr;
-      } else {
-        GeneralSetFunctionExpr setFunctionExpr = new GeneralSetFunctionExpr(signature, isDistinct,
-            visitSelectExpression(ctx.selectExpression(0)));
-        current = setFunctionExpr;
-      }
-    } else {
-      FunctionExpr functionExpr = new FunctionExpr(signature);
-      Expr[] params = new Expr[ctx.selectExpression().size()];
-      for (int i = 0; i < ctx.selectExpression().size(); i++) {
-        params[i] = visitSelectExpression(ctx.selectExpression(i));
-      }
-      functionExpr.setParams(params);
-      current = functionExpr;
-    }
-
-
-    return current;
-  }
-
-  /**
-   * This method parse CAST expression.
-   * This returns only expression field without casting type
-   * because Tajo doesn't provide CAST expression.
-   *
-   * @param ctx
-   * @return
-   */
-  @Override
-  public Expr visitCastExpression(HiveParser.CastExpressionContext ctx) {
-    return visitExpression(ctx.expression());
-  }
-
-  @Override
-  public Expr visitCaseExpression(HiveParser.CaseExpressionContext ctx) {
-    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
-    Expr condition = null, result = null;
-    for (int i = 1; i < ctx.getChildCount(); i++) {
-      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
-        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveLexer.KW_WHEN) {
-          condition = null;
-          result = null;
-
-          if (ctx.getChild(i + 1) instanceof HiveParser.ExpressionContext) {
-            condition = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 1));
-          }
-
-          if (ctx.getChild(i + 3) instanceof HiveParser.ExpressionContext) {
-            result = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 3));
-          }
-
-          if (condition != null && result != null) {
-            caseWhen.addWhen(condition, result);
-          }
-        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveLexer.KW_ELSE) {
-          result = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 1));
-          caseWhen.setElseResult(result);
-        }
-      }
-    }
-
-    return caseWhen;
-  }
-
-  @Override
-  public Expr visitWhenExpression(HiveParser.WhenExpressionContext ctx) {
-    CaseWhenPredicate caseWhen = new CaseWhenPredicate();
-    Expr condition = null, result = null;
-    for (int i = 1; i < ctx.getChildCount(); i++) {
-      if (ctx.getChild(i) instanceof TerminalNodeImpl) {
-        if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveLexer.KW_WHEN) {
-          condition = null;
-          result = null;
-
-          if (ctx.getChild(i + 1) instanceof HiveParser.ExpressionContext) {
-            condition = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 1));
-          }
-
-          if (ctx.getChild(i + 3) instanceof HiveParser.ExpressionContext) {
-            result = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 3));
-          }
-
-          if (condition != null && result != null) {
-            caseWhen.addWhen(condition, result);
-          }
-        } else if (((TerminalNodeImpl) ctx.getChild(i)).getSymbol().getType() == HiveLexer.KW_ELSE) {
-          result = visitExpression((HiveParser.ExpressionContext) ctx.getChild(i + 1));
-          caseWhen.setElseResult(result);
-        }
-      }
-    }
-
-    return caseWhen;
-  }
-
-  @Override
-  public Aggregation visitGroupByClause(HiveParser.GroupByClauseContext ctx) {
-    Aggregation clause = new Aggregation();
-    if (ctx.groupByExpression().size() > 0) {
-      List<Expr> columns = new ArrayList<Expr>();
-      List<Expr> functions = new ArrayList<Expr>();
-
-      for (int i = 0; i < ctx.groupByExpression().size(); i++) {
-        Expr expr = visitGroupByExpression(ctx.groupByExpression(i));
-
-        if (expr instanceof ColumnReferenceExpr) {
-          columns.add(expr);
-        } else if (expr instanceof FunctionExpr) {
-          functions.add(expr);
-        } else {
-          //TODO: find another case.
-        }
-      }
-
-      Aggregation.GroupElement[] groups = null;
-
-      if (columns.size() > 0) {
-        groups = new Aggregation.GroupElement[1 + functions.size()];
-      } else {
-        groups = new Aggregation.GroupElement[functions.size()];
-      }
-
-      int index = 0;
-      if (columns.size() > 0) {
-        index = 0;
-        ColumnReferenceExpr[] columnReferenceExprs = new ColumnReferenceExpr[columns.size()];
-        for (int i = 0; i < columns.size(); i++) {
-          ColumnReferenceExpr expr = (ColumnReferenceExpr) columns.get(i);
-          columnReferenceExprs[i] = expr;
-        }
-        groups[index] = new Aggregation.GroupElement(Aggregation.GroupType.OrdinaryGroup, columnReferenceExprs);
-      }
-
-      if (functions.size() > 0) {
-        if (columns.size() == 0) {
-          index = 0;
-        } else {
-          index = 1;
-        }
-
-        for (int i = 0; i < functions.size(); i++) {
-          FunctionExpr function = (FunctionExpr) functions.get(i);
-
-          Expr[] params = function.getParams();
-          ColumnReferenceExpr[] column = new ColumnReferenceExpr[params.length];
-          for (int j = 0; j < column.length; j++)
-            column[j] = (ColumnReferenceExpr) params[j];
-
-          if (function.getSignature().equalsIgnoreCase("ROLLUP"))
-            groups[i + index] = new Aggregation.GroupElement(Aggregation.GroupType.Rollup, column);
-          else if (function.getSignature().equalsIgnoreCase("CUBE"))
-            groups[i + index] = new Aggregation.GroupElement(Aggregation.GroupType.Cube, column);
-          else
-            throw new RuntimeException("Unexpected aggregation function.");
-        }
-      }
-
-      clause.setGroups(groups);
-    }
-
-    //TODO: grouping set expression
-    return clause;
-  }
-
-  @Override
-  public Sort visitOrderByClause(HiveParser.OrderByClauseContext ctx) {
-    Sort clause = null;
-    Sort.SortSpec[] specs = null;
-
-    if (ctx.columnRefOrder().size() > 0) {
-      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
-      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
-        ColumnReferenceExpr column = (ColumnReferenceExpr) visitExpression(ctx.columnRefOrder().get(i).expression());
-        specs[i] = new Sort.SortSpec(column);
-        if (ctx.columnRefOrder(i).KW_DESC() != null) {
-          specs[i].setDescending();
-        }
-      }
-      clause = new Sort(specs);
-    }
-    return clause;
-
-  }
-
-  @Override
-  public Expr visitHavingClause(HiveParser.HavingClauseContext ctx) {
-    return visitHavingCondition(ctx.havingCondition());
-  }
-
-  @Override
-  public Expr visitClusterByClause(HiveParser.ClusterByClauseContext ctx) {
-    // TODO: It needs to consider how to support.
-    return null;
-  }
-
-  @Override
-  public Expr visitDistributeByClause(HiveParser.DistributeByClauseContext ctx) {
-    // TODO: It needs to consider how to support.
-
-    return null;
-  }
-
-  @Override
-  public Sort visitSortByClause(HiveParser.SortByClauseContext ctx) {
-    Sort clause = null;
-    Sort.SortSpec[] specs = null;
-
-    if (ctx.columnRefOrder().size() > 0) {
-      specs = new Sort.SortSpec[ctx.columnRefOrder().size()];
-      for (int i = 0; i < ctx.columnRefOrder().size(); i++) {
-        ColumnReferenceExpr column = (ColumnReferenceExpr) visitColumnRefOrder(ctx.columnRefOrder(i));
-        specs[i] = new Sort.SortSpec(column);
-
-        if (ctx.columnRefOrder(i).KW_DESC() != null) {
-          specs[i].setDescending();
-        }
-      }
-      clause = new Sort(specs);
-    }
-
-    return clause;
-  }
-
-  @Override
-  public Limit visitLimitClause(HiveParser.LimitClauseContext ctx) {
-    LiteralValue expr = new LiteralValue(ctx.Number().getText(), LiteralValue.LiteralType.Unsigned_Integer);
-    Limit limit = new Limit(expr);
-    return limit;
-  }
-
-  @Override
-  public Expr visitWindow_clause(HiveParser.Window_clauseContext ctx) {
-    // TODO: It needs to consider how to support.
-    return null;
-  }
-
-  @Override
-  public Insert visitInsertClause(HiveParser.InsertClauseContext ctx) {
-    Insert insert = new Insert();
-    if (ctx.KW_OVERWRITE() != null)
-      insert.setOverwrite();
-
-    if (ctx.tableOrPartition() != null) {
-      HiveParser.TableOrPartitionContext partitionContext = ctx.tableOrPartition();
-      if (partitionContext.tableName() != null) {
-        insert.setTableName(ctx.tableOrPartition().tableName().getText());
-      }
-    }
-
-    if (ctx.destination() != null) {
-      HiveParser.DestinationContext destination = ctx.destination();
-      if (destination.KW_DIRECTORY() != null) {
-        String location = destination.StringLiteral().getText();
-        location = location.replaceAll("\\'", "");
-        insert.setLocation(location);
-      } else if (destination.KW_TABLE() != null) {
-        if (destination.tableOrPartition() != null) {
-          HiveParser.TableOrPartitionContext partitionContext = destination.tableOrPartition();
-          if (partitionContext.tableName() != null) {
-            insert.setTableName(partitionContext.tableName().getText());
-          }
-        }
-
-        if (destination.tableFileFormat() != null) {
-          if (destination.tableFileFormat().KW_RCFILE() != null) {
-            insert.setStorageType("rcfile");
-          } else if (destination.tableFileFormat().KW_TEXTFILE() != null) {
-            insert.setStorageType("csv");
-          }
-
-        }
-      }
-    }
-
-    return insert;
-  }
-
-  @Override
-  public Expr visitCreateTableStatement(HiveParser.CreateTableStatementContext ctx) {
-    CreateTable createTable = null;
-    Map<String, String> params = new HashMap<String, String>();
-
-    if (ctx.name != null) {
-      createTable = new CreateTable(ctx.name.getText());
-      if (ctx.KW_EXTERNAL() != null) {
-        createTable.setExternal();
-      }
-
-      if (ctx.tableFileFormat() != null) {
-        if (ctx.tableFileFormat().KW_RCFILE() != null) {
-          createTable.setStorageType("rcfile");
-        } else if (ctx.tableFileFormat().KW_TEXTFILE() != null) {
-          createTable.setStorageType("csv");
-        }
-      }
-
-      if (ctx.tableRowFormat() != null) {
-        if (ctx.tableRowFormat().rowFormatDelimited() != null) {
-          String delimiter = ctx.tableRowFormat().rowFormatDelimited().tableRowFormatFieldIdentifier().getChild(3)
-              .getText().replaceAll("'", "");
-          params.put("csvfile.delimiter", SQLAnalyzer.escapeDelimiter(delimiter));
-        }
-      }
-
-      if (ctx.tableLocation() != null) {
-        String location = ctx.tableLocation().StringLiteral().getText();
-        location = location.replaceAll("'", "");
-        createTable.setLocation(location);
-
-      }
-
-      if (ctx.columnNameTypeList() != null) {
-        List<HiveParser.ColumnNameTypeContext> list = ctx.columnNameTypeList().columnNameType();
-
-        CreateTable.ColumnDefinition[] columns = new CreateTable.ColumnDefinition[list.size()];
-
-        for (int i = 0; i < list.size(); i++) {
-          HiveParser.ColumnNameTypeContext eachColumn = list.get(i);
-          String type = null;
-          if (eachColumn.colType().type() != null) {
-            if (eachColumn.colType().type().primitiveType() != null) {
-              HiveParser.PrimitiveTypeContext primitiveType = eachColumn.colType().type().primitiveType();
-
-              if (primitiveType.KW_STRING() != null) {
-                type = TajoDataTypes.Type.TEXT.name();
-              } else if (primitiveType.KW_TINYINT() != null) {
-                type = TajoDataTypes.Type.INT1.name();
-              } else if (primitiveType.KW_SMALLINT() != null) {
-                type = TajoDataTypes.Type.INT2.name();
-              } else if (primitiveType.KW_INT() != null) {
-                type = TajoDataTypes.Type.INT4.name();
-              } else if (primitiveType.KW_BIGINT() != null) {
-                type = TajoDataTypes.Type.INT8.name();
-              } else if (primitiveType.KW_FLOAT() != null) {
-                type = TajoDataTypes.Type.FLOAT4.name();
-              } else if (primitiveType.KW_DOUBLE() != null) {
-                type = TajoDataTypes.Type.FLOAT8.name();
-              } else if (primitiveType.KW_DECIMAL() != null) {
-                type = TajoDataTypes.Type.DECIMAL.name();
-              } else if (primitiveType.KW_BOOLEAN() != null) {
-                type = TajoDataTypes.Type.BOOLEAN.name();
-              } else if (primitiveType.KW_DATE() != null) {
-                type = TajoDataTypes.Type.DATE.name();
-              } else if (primitiveType.KW_DATETIME() != null) {
-                //TODO
-              } else if (primitiveType.KW_TIMESTAMP() != null) {
-                type = TajoDataTypes.Type.TIMESTAMP.name();
-              }
-
-              columns[i] = new CreateTable.ColumnDefinition(eachColumn.colName.Identifier().getText(), type);
-            }
-          }
-        }
-        if (columns != null) {
-          createTable.setTableElements(columns);
-        }
-
-        if (!params.isEmpty()) {
-          createTable.setParams(params);
-        }
-      }
-    }
-
-    return createTable;
-  }
-
-  @Override
-  public Expr visitDropTableStatement(HiveParser.DropTableStatementContext ctx) {
-    DropTable dropTable = new DropTable(ctx.tableName().getText(), false);
-    return dropTable;
-  }
-
-  /**
-   * 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 characteres. 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 funtion
-   * 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 ANTLRInputStream {
-
-    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);
-    }
-  }
-}
\ No newline at end of file


[3/5] TAJO-590: Rename HiveConverter to HiveQLAnalyzer. (jaehwa)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/e8da943a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
new file mode 100644
index 0000000..6a85695
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLParser.g4
@@ -0,0 +1,2067 @@
+/**
+   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 HiveQLParser;
+
+options
+{
+tokenVocab=HiveQLLexer;
+language=Java;
+}
+
+
+// Package headers
+@header {
+import java.util.*;
+}
+
+
+@members {
+  Stack msgs = new Stack<String>();
+}
+
+@rulecatch {
+catch (RecognitionException e) {
+ reportError(e);
+  throw e;
+}
+}
+
+//----------------------- Rules for parsing selectClause -----------------------------
+// select a,b,c ...
+selectClause
+@init { msgs.push("select clause"); }
+@after { msgs.pop(); }
+    :
+    KW_SELECT hintClause? (((KW_ALL | dist=KW_DISTINCT)? selectList)
+                          | (transform=KW_TRANSFORM selectTrfmClause))
+    |
+    trfmClause
+    ;
+
+selectList
+@init { msgs.push("select list"); }
+@after { msgs.pop(); }
+    :
+    selectItem ( COMMA  selectItem )* 
+    ;
+
+selectTrfmClause
+@init { msgs.push("transform clause"); }
+@after { msgs.pop(); }
+    :
+    LPAREN selectExpressionList RPAREN
+    inSerde=rowFormat inRec=recordWriter
+    KW_USING StringLiteral
+    ( KW_AS ((LPAREN (aliasList | columnNameTypeList) RPAREN) | (aliasList | columnNameTypeList)))?
+    outSerde=rowFormat outRec=recordReader
+    ;
+
+hintClause
+@init { msgs.push("hint clause"); }
+@after { msgs.pop(); }
+    :
+    DIVIDE STAR PLUS hintList STAR DIVIDE 
+    ;
+
+hintList
+@init { msgs.push("hint list"); }
+@after { msgs.pop(); }
+    :
+    hintItem (COMMA hintItem)* 
+    ;
+
+hintItem
+@init { msgs.push("hint item"); }
+@after { msgs.pop(); }
+    :
+    hintName (LPAREN hintArgs RPAREN)? 
+    ;
+
+hintName
+@init { msgs.push("hint name"); }
+@after { msgs.pop(); }
+    :
+    KW_MAPJOIN 
+    | KW_STREAMTABLE 
+    | KW_HOLD_DDLTIME 
+    ;
+
+hintArgs
+@init { msgs.push("hint arguments"); }
+@after { msgs.pop(); }
+    :
+    hintArgName (COMMA hintArgName)* 
+    ;
+
+hintArgName
+@init { msgs.push("hint argument name"); }
+@after { msgs.pop(); }
+    :
+    identifier
+    ;
+
+selectItem
+@init { msgs.push("selection target"); }
+@after { msgs.pop(); }
+    :
+    ( selectExpression (KW_OVER ws=window_specification )?
+      ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))?
+    ) 
+    ;
+
+trfmClause
+@init { msgs.push("transform clause"); }
+@after { msgs.pop(); }
+    :
+    (   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
+    ;
+
+selectExpression
+@init { msgs.push("select expression"); }
+@after { msgs.pop(); }
+    :
+    expression | tableAllColumns
+    ;
+
+selectExpressionList
+@init { msgs.push("select expression list"); }
+@after { msgs.pop(); }
+    :
+    selectExpression (COMMA selectExpression)* 
+    ;
+
+
+//---------------------- Rules for windowing clauses -------------------------------
+window_clause 
+@init { msgs.push("window_clause"); }
+@after { msgs.pop(); } 
+:
+  KW_WINDOW window_defn (COMMA window_defn)* 
+;  
+
+window_defn 
+@init { msgs.push("window_defn"); }
+@after { msgs.pop(); } 
+:
+  Identifier KW_AS window_specification 
+;  
+
+window_specification 
+@init { msgs.push("window_specification"); }
+@after { msgs.pop(); } 
+:
+  (Identifier | ( LPAREN Identifier? partitioningSpec? window_frame? RPAREN)) 
+;
+
+window_frame :
+ window_range_expression |
+ window_value_expression
+;
+
+window_range_expression 
+@init { msgs.push("window_range_expression"); }
+@after { msgs.pop(); } 
+:
+ KW_ROWS sb=window_frame_start_boundary 
+ KW_ROWS KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
+;
+
+window_value_expression 
+@init { msgs.push("window_value_expression"); }
+@after { msgs.pop(); } 
+:
+ KW_RANGE sb=window_frame_start_boundary 
+ KW_RANGE KW_BETWEEN s=window_frame_boundary KW_AND end=window_frame_boundary 
+;
+
+window_frame_start_boundary 
+@init { msgs.push("windowframestartboundary"); }
+@after { msgs.pop(); } 
+:
+  KW_UNBOUNDED KW_PRECEDING  
+  KW_CURRENT KW_ROW  
+  Number KW_PRECEDING 
+;
+
+window_frame_boundary 
+@init { msgs.push("windowframeboundary"); }
+@after { msgs.pop(); } 
+:
+  KW_UNBOUNDED (r=KW_PRECEDING|r=KW_FOLLOWING)  
+  KW_CURRENT KW_ROW  
+  Number (d=KW_PRECEDING | d=KW_FOLLOWING ) 
+;   
+
+
+tableAllColumns
+    : STAR
+    | tableName DOT STAR
+    ;
+
+// (table|column)
+tableOrColumn
+@init { msgs.push("table or column identifier"); }
+@after { msgs.pop(); }
+    :
+    identifier 
+    ;
+
+expressionList
+@init { msgs.push("expression list"); }
+@after { msgs.pop(); }
+    :
+    expression (COMMA expression)* 
+    ;
+
+aliasList
+@init { msgs.push("alias list"); }
+@after { msgs.pop(); }
+    :
+    identifier (COMMA identifier)* 
+    ;
+
+
+//----------------------- Rules for parsing fromClause ------------------------------
+// from [col1, col2, col3] table1, [col4, col5] table2
+fromClause
+@init { msgs.push("from clause"); }
+@after { msgs.pop(); }
+    :
+    KW_FROM joinSource 
+    ;
+
+joinSource
+@init { msgs.push("join source"); }
+@after { msgs.pop(); }
+    : fromSource ( joinToken fromSource (KW_ON expression)? 
+    )*
+    | uniqueJoinToken uniqueJoinSource (COMMA uniqueJoinSource)+
+    ;
+
+uniqueJoinSource
+@init { msgs.push("join source"); }
+@after { msgs.pop(); }
+    : KW_PRESERVE? fromSource uniqueJoinExpr
+    ;
+
+uniqueJoinExpr
+@init { msgs.push("unique join expression list"); }
+@after { msgs.pop(); }
+    : LPAREN e1+=expression (COMMA e1+=expression)* RPAREN
+    ;
+
+uniqueJoinToken
+@init { msgs.push("unique join"); }
+@after { msgs.pop(); }
+    : KW_UNIQUEJOIN 
+;
+
+joinToken
+@init { msgs.push("join type specifier"); }
+@after { msgs.pop(); }
+    :
+      KW_JOIN                    
+    | KW_INNER  KW_JOIN            
+    | KW_CROSS KW_JOIN            
+    | KW_LEFT  KW_OUTER KW_JOIN   
+    | KW_RIGHT KW_OUTER KW_JOIN  
+    | KW_FULL  KW_OUTER KW_JOIN  
+    | KW_LEFT  KW_SEMI  KW_JOIN  
+    ;
+
+lateralView
+@init {msgs.push("lateral view"); }
+@after {msgs.pop(); }
+	:
+	KW_LATERAL KW_VIEW function tableAlias KW_AS identifier (COMMA identifier)* 
+	;
+
+tableAlias
+@init {msgs.push("table alias"); }
+@after {msgs.pop(); }
+    :
+    identifier 
+    ;
+
+fromSource
+@init { msgs.push("from source"); }
+@after { msgs.pop(); }
+    :
+    ((Identifier LPAREN) | tableSource | subQuerySource) (lateralView)*
+    ;
+
+tableBucketSample
+@init { msgs.push("table bucket sample specification"); }
+@after { msgs.pop(); }
+    :
+    KW_TABLESAMPLE LPAREN KW_BUCKET (numerator=Number) KW_OUT KW_OF (denominator=Number) (KW_ON expr+=expression (COMMA expr+=expression)*)? RPAREN 
+    ;
+
+splitSample
+@init { msgs.push("table split sample specification"); }
+@after { msgs.pop(); }
+    :
+    KW_TABLESAMPLE LPAREN  (numerator=Number) (percent=KW_PERCENT|KW_ROWS) RPAREN
+    |
+    KW_TABLESAMPLE LPAREN  (numerator=ByteLengthLiteral) RPAREN
+    ;
+
+tableSample
+@init { msgs.push("table sample specification"); }
+@after { msgs.pop(); }
+    :
+    tableBucketSample |
+    splitSample
+    ;
+
+tableSource
+@init { msgs.push("table source"); }
+@after { msgs.pop(); }
+    : tabname=tableName (ts=tableSample)? (alias=identifier)?
+    ;
+
+tableName
+@init { msgs.push("table name"); }
+@after { msgs.pop(); }
+    :
+    db=identifier DOT tab=identifier
+    |
+    tab=identifier
+    ;
+
+viewName
+@init { msgs.push("view name"); }
+@after { msgs.pop(); }
+    :
+    (db=identifier DOT)? view=identifier
+    ;
+
+subQuerySource
+@init { msgs.push("subquery source"); }
+@after { msgs.pop(); }
+    :
+    LPAREN queryStatementExpression RPAREN identifier 
+    ;
+
+//---------------------- Rules for parsing PTF clauses -----------------------------
+partitioningSpec
+@init { msgs.push("partitioningSpec clause"); }
+@after { msgs.pop(); } 
+   :
+   partitionByClause orderByClause? 
+   orderByClause 
+   distributeByClause sortByClause? 
+   sortByClause 
+   clusterByClause 
+   ;
+
+partitionTableFunctionSource
+@init { msgs.push("partitionTableFunctionSource clause"); }
+@after { msgs.pop(); } 
+   :
+   subQuerySource |
+   tableSource |
+   partitionedTableFunction
+   ;
+
+partitionedTableFunction
+@init { msgs.push("ptf clause"); }
+@after { msgs.pop(); } 
+   :
+   name=Identifier
+   LPAREN KW_ON ptfsrc=partitionTableFunctionSource partitioningSpec?
+     ((Identifier LPAREN expression RPAREN ) )? 
+   RPAREN alias=Identifier? 
+   ;
+
+//----------------------- Rules for parsing whereClause -----------------------------
+// where a=b and ...
+whereClause
+@init { msgs.push("where clause"); }
+@after { msgs.pop(); }
+    :
+    KW_WHERE searchCondition 
+    ;
+
+searchCondition
+@init { msgs.push("search condition"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+//-----------------------------------------------------------------------------------
+
+
+// group by a,b
+groupByClause
+@init { msgs.push("group by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_GROUP KW_BY
+    groupByExpression
+    ( COMMA groupByExpression )*
+    ((rollup=KW_WITH KW_ROLLUP) | (cube=KW_WITH KW_CUBE)) ?
+    (sets=KW_GROUPING KW_SETS 
+    LPAREN groupingSetExpression ( COMMA groupingSetExpression)*  RPAREN ) ?
+    ;
+
+groupingSetExpression
+@init {msgs.push("grouping set expression"); }
+@after {msgs.pop(); }
+   :
+   groupByExpression
+   |
+   LPAREN 
+   groupByExpression (COMMA groupByExpression)*
+   RPAREN
+   |
+   LPAREN
+   RPAREN
+   ;
+
+
+groupByExpression
+@init { msgs.push("group by expression"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+havingClause
+@init { msgs.push("having clause"); }
+@after { msgs.pop(); }
+    :
+    KW_HAVING havingCondition 
+    ;
+
+havingCondition
+@init { msgs.push("having condition"); }
+@after { msgs.pop(); }
+    :
+    expression
+    ;
+
+// order by a,b
+orderByClause
+@init { msgs.push("order by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_ORDER KW_BY
+    LPAREN columnRefOrder
+    ( COMMA columnRefOrder)* RPAREN 
+    |
+    KW_ORDER KW_BY
+    columnRefOrder
+    ( COMMA columnRefOrder)* 
+    ;
+
+clusterByClause
+@init { msgs.push("cluster by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_CLUSTER KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_CLUSTER KW_BY
+    expression
+    ((COMMA))*
+    ;
+
+partitionByClause
+@init  { msgs.push("partition by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_PARTITION KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_PARTITION KW_BY
+    expression ((COMMA))*
+    ;
+
+distributeByClause
+@init { msgs.push("distribute by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_DISTRIBUTE KW_BY
+    LPAREN expression (COMMA expression)* RPAREN
+    |
+    KW_DISTRIBUTE KW_BY
+    expression ((COMMA))*
+    ;
+
+sortByClause
+@init { msgs.push("sort by clause"); }
+@after { msgs.pop(); }
+    :
+    KW_SORT KW_BY
+    LPAREN columnRefOrder
+    ( COMMA columnRefOrder)* RPAREN 
+    |
+    KW_SORT KW_BY
+    columnRefOrder
+    ( (COMMA))*
+    ;
+
+// fun(par1, par2, par3)
+function
+@init { msgs.push("function specification"); }
+@after { msgs.pop(); }
+    :
+    functionName
+    LPAREN
+      (
+        (star=STAR)
+        | (dist=KW_DISTINCT)? (selectExpression (COMMA selectExpression)*)?
+      )
+    RPAREN 
+    ;
+
+functionName
+@init { msgs.push("function name"); }
+@after { msgs.pop(); }
+    : // Keyword IF is also a function name
+    KW_IF | KW_ARRAY | KW_MAP | KW_STRUCT | KW_UNIONTYPE | identifier
+    ;
+
+castExpression
+@init { msgs.push("cast expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CAST
+    LPAREN
+          expression
+          KW_AS
+          primitiveType
+    RPAREN 
+    ;
+
+caseExpression
+@init { msgs.push("case expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CASE expression
+    (KW_WHEN expression KW_THEN expression)+
+    (KW_ELSE expression)?
+    KW_END 
+    ;
+
+whenExpression
+@init { msgs.push("case expression"); }
+@after { msgs.pop(); }
+    :
+    KW_CASE
+     ( KW_WHEN expression KW_THEN expression)+
+    (KW_ELSE expression)?
+    KW_END 
+    ;
+
+constant
+@init { msgs.push("constant"); }
+@after { msgs.pop(); }
+    :
+    Number
+    | StringLiteral
+    | stringLiteralSequence
+    | BigintLiteral
+    | SmallintLiteral
+    | TinyintLiteral
+    | DecimalLiteral
+    | charSetStringLiteral
+    | booleanValue
+    ;
+
+stringLiteralSequence
+    :
+    StringLiteral StringLiteral+ 
+    ;
+
+charSetStringLiteral
+@init { msgs.push("character string literal"); }
+@after { msgs.pop(); }
+    :
+    csName=CharSetName csLiteral=CharSetLiteral 
+    ;
+
+expression
+@init { msgs.push("expression specification"); }
+@after { msgs.pop(); }
+    :
+    precedenceOrExpression
+    ;
+
+atomExpression
+    :
+    KW_NULL 
+    | constant
+    | function
+    | castExpression
+    | caseExpression
+    | whenExpression
+    | tableOrColumn
+    | LPAREN expression RPAREN
+    ;
+
+
+precedenceFieldExpression
+    :
+    atomExpression ((LSQUARE expression RSQUARE) | (DOT identifier))*
+    ;
+
+precedenceUnaryOperator
+    :
+    PLUS | MINUS | TILDE
+    ;
+
+nullCondition
+    :
+    KW_NULL     
+    | KW_NOT KW_NULL     
+    ;
+
+precedenceUnaryPrefixExpression
+    :
+    (precedenceUnaryOperator)* precedenceFieldExpression
+    ;
+
+precedenceUnarySuffixExpression
+    : precedenceUnaryPrefixExpression (a=KW_IS nullCondition)?
+    ;
+
+
+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
+    ;
+
+precedenceEqualExpression
+    :
+    (left=precedenceBitwiseOrExpression     
+    )
+    (
+       (KW_NOT precedenceEqualNegatableOperator notExpr=precedenceBitwiseOrExpression) 
+    | (precedenceEqualOperator equalExpr=precedenceBitwiseOrExpression)
+    | (KW_NOT KW_IN expressions) 
+    | (KW_IN expressions) 
+    | ( KW_NOT KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) ) 
+    | ( KW_BETWEEN (min=precedenceBitwiseOrExpression) KW_AND (max=precedenceBitwiseOrExpression) )
+    )*
+    ;
+
+expressions
+    :
+    LPAREN expression (COMMA expression)* RPAREN 
+    ;
+
+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
+    ;
+
+tableOrPartition
+   :
+   tableName partitionSpec? 
+   ;
+
+partitionSpec
+    :
+    KW_PARTITION
+     LPAREN partitionVal (COMMA  partitionVal )* RPAREN 
+    ;
+
+partitionVal
+    :
+    identifier (EQUAL constant)? 
+    ;
+
+dropPartitionSpec
+    :
+    KW_PARTITION
+     LPAREN dropPartitionVal (COMMA  dropPartitionVal )* RPAREN 
+    ;
+
+dropPartitionVal
+    :
+    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
+    | StringLiteral
+    | identifier
+    ;
+
+identifier
+    :
+    Identifier
+    | nonReserved 
+    ;
+    
+nonReserved
+    :
+    KW_TRUE | KW_FALSE | KW_LIKE | KW_EXISTS | KW_ASC | KW_DESC | KW_ORDER | KW_GROUP | KW_BY | KW_AS | KW_INSERT | KW_OVERWRITE | KW_OUTER | KW_LEFT | KW_RIGHT | KW_FULL | KW_PARTITION | KW_PARTITIONS | KW_TABLE | KW_TABLES | KW_COLUMNS | KW_INDEX | KW_INDEXES | KW_REBUILD | KW_FUNCTIONS | KW_SHOW | KW_MSCK | KW_REPAIR | KW_DIRECTORY | KW_LOCAL | KW_USING | KW_CLUSTER | KW_DISTRIBUTE | KW_SORT | KW_UNION | KW_LOAD | KW_EXPORT | KW_IMPORT | KW_DATA | KW_INPATH | KW_IS | KW_NULL | KW_CREATE | KW_EXTERNAL | KW_ALTER | KW_CHANGE | KW_FIRST | KW_AFTER | KW_DESCRIBE | KW_DROP | KW_RENAME | KW_IGNORE | KW_PROTECTION | KW_TO | KW_COMMENT | KW_BOOLEAN | KW_TINYINT | KW_SMALLINT | KW_INT | KW_BIGINT | KW_FLOAT | KW_DOUBLE | KW_DATE | KW_DATETIME | KW_TIMESTAMP | KW_DECIMAL | KW_STRING | KW_ARRAY | KW_STRUCT | KW_UNIONTYPE | KW_PARTITIONED | KW_CLUSTERED | KW_SORTED | KW_INTO | KW_BUCKETS | KW_ROW | KW_ROWS | KW_FORMAT | KW_DELIMITED | KW_FIELDS | KW_TERMINATED | KW_ESCAPED | KW_COLLECTION | 
 KW_ITEMS | KW_KEYS | KW_KEY_TYPE | KW_LINES | KW_STORED | KW_FILEFORMAT | KW_SEQUENCEFILE | KW_TEXTFILE | KW_RCFILE | KW_ORCFILE | KW_INPUTFORMAT | KW_OUTPUTFORMAT | KW_INPUTDRIVER | KW_OUTPUTDRIVER | KW_OFFLINE | KW_ENABLE | KW_DISABLE | KW_READONLY | KW_NO_DROP | KW_LOCATION | KW_BUCKET | KW_OUT | KW_OF | KW_PERCENT | KW_ADD | KW_REPLACE | KW_RLIKE | KW_REGEXP | KW_TEMPORARY | KW_EXPLAIN | KW_FORMATTED | KW_PRETTY | KW_DEPENDENCY | KW_SERDE | KW_WITH | KW_DEFERRED | KW_SERDEPROPERTIES | KW_DBPROPERTIES | KW_LIMIT | KW_SET | KW_UNSET | KW_TBLPROPERTIES | KW_IDXPROPERTIES | KW_VALUE_TYPE | KW_ELEM_TYPE | KW_MAPJOIN | KW_STREAMTABLE | KW_HOLD_DDLTIME | KW_CLUSTERSTATUS | KW_UTC | KW_UTCTIMESTAMP | KW_LONG | KW_DELETE | KW_PLUS | KW_MINUS | KW_FETCH | KW_INTERSECT | KW_VIEW | KW_IN | KW_DATABASES | KW_MATERIALIZED | KW_SCHEMA | KW_SCHEMAS | KW_GRANT | KW_REVOKE | KW_SSL | KW_UNDO | KW_LOCK | KW_LOCKS | KW_UNLOCK | KW_SHARED | KW_EXCLUSIVE | KW_PROCEDURE | KW_UNSIGNED | KW_WHILE | KW_R
 EAD | KW_READS | KW_PURGE | KW_RANGE | KW_ANALYZE | KW_BEFORE | KW_BETWEEN | KW_BOTH | KW_BINARY | KW_CONTINUE | KW_CURSOR | KW_TRIGGER | KW_RECORDREADER | KW_RECORDWRITER | KW_SEMI | KW_LATERAL | KW_TOUCH | KW_ARCHIVE | KW_UNARCHIVE | KW_COMPUTE | KW_STATISTICS | KW_USE | KW_OPTION | KW_CONCATENATE | KW_SHOW_DATABASE | KW_UPDATE | KW_RESTRICT | KW_CASCADE | KW_SKEWED | KW_ROLLUP | KW_CUBE | KW_DIRECTORIES | KW_FOR | KW_GROUPING | KW_SETS | KW_TRUNCATE | KW_NOSCAN | KW_USER | KW_ROLE | KW_INNER
+    ;
+
+//-----------------------------------------------------------------------------------
+
+// starting rule
+statement
+	: explainStatement EOF
+	| execStatement EOF
+	;
+
+explainStatement
+@init { msgs.push("explain statement"); }
+@after { msgs.pop(); }
+	: KW_EXPLAIN (explainOptions=KW_EXTENDED|explainOptions=KW_FORMATTED|explainOptions=KW_DEPENDENCY)? execStatement
+	;
+
+execStatement
+@init { msgs.push("statement"); }
+@after { msgs.pop(); }
+    : queryStatementExpression
+    | loadStatement
+    | exportStatement
+    | importStatement
+    | ddlStatement
+    ;
+
+loadStatement
+@init { msgs.push("load statement"); }
+@after { msgs.pop(); }
+    : KW_LOAD KW_DATA (islocal=KW_LOCAL)? KW_INPATH (path=StringLiteral) (isoverwrite=KW_OVERWRITE)? KW_INTO KW_TABLE (tab=tableOrPartition)
+    ;
+
+exportStatement
+@init { msgs.push("export statement"); }
+@after { msgs.pop(); }
+    : KW_EXPORT KW_TABLE (tab=tableOrPartition) KW_TO (path=StringLiteral)
+    ;
+
+importStatement
+@init { msgs.push("import statement"); }
+@after { msgs.pop(); }
+	: KW_IMPORT ((ext=KW_EXTERNAL)? KW_TABLE (tab=tableOrPartition))? KW_FROM (path=StringLiteral) tableLocation?
+    ;
+
+ddlStatement
+@init { msgs.push("ddl statement"); }
+@after { msgs.pop(); }
+    : createDatabaseStatement
+    | switchDatabaseStatement
+    | dropDatabaseStatement
+    | createTableStatement
+    | dropTableStatement
+    | truncateTableStatement
+    | alterStatement
+    | descStatement
+    | showStatement
+    | metastoreCheck
+    | createViewStatement
+    | dropViewStatement
+    | createFunctionStatement
+    | createIndexStatement
+    | dropIndexStatement
+    | dropFunctionStatement
+    | analyzeStatement
+    | lockStatement
+    | unlockStatement
+    | createRoleStatement
+    | dropRoleStatement
+    | grantPrivileges
+    | revokePrivileges
+    | showGrants
+    | showRoleGrants
+    | grantRole
+    | revokeRole
+    ;
+
+ifExists
+@init { msgs.push("if exists clause"); }
+@after { msgs.pop(); }
+    : KW_IF KW_EXISTS
+    ;
+
+restrictOrCascade
+@init { msgs.push("restrict or cascade clause"); }
+@after { msgs.pop(); }
+    : KW_RESTRICT
+    | KW_CASCADE
+    ;
+
+ifNotExists
+@init { msgs.push("if not exists clause"); }
+@after { msgs.pop(); }
+    : KW_IF KW_NOT KW_EXISTS
+    ;
+
+storedAsDirs
+@init { msgs.push("stored as directories"); }
+@after { msgs.pop(); }
+    : KW_STORED KW_AS KW_DIRECTORIES
+    ;
+
+orReplace
+@init { msgs.push("or replace clause"); }
+@after { msgs.pop(); }
+    : KW_OR KW_REPLACE
+    ;
+
+ignoreProtection
+@init { msgs.push("ignore protection clause"); }
+@after { msgs.pop(); }
+        : KW_IGNORE KW_PROTECTION
+        ;
+
+createDatabaseStatement
+@init { msgs.push("create database statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE (KW_DATABASE|KW_SCHEMA)
+        ifNotExists?
+        name=identifier
+        databaseComment?
+        dbLocation?
+        (KW_WITH KW_DBPROPERTIES dbprops=dbProperties)?
+    ;
+
+dbLocation
+@init { msgs.push("database location specification"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+dbProperties
+@init { msgs.push("dbproperties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN dbPropertiesList RPAREN 
+    ;
+
+dbPropertiesList
+@init { msgs.push("database properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    ;
+
+
+switchDatabaseStatement
+@init { msgs.push("switch database statement"); }
+@after { msgs.pop(); }
+    : KW_USE identifier
+    ;
+
+dropDatabaseStatement
+@init { msgs.push("drop database statement"); }
+@after { msgs.pop(); }
+    : KW_DROP (KW_DATABASE|KW_SCHEMA) ifExists? identifier restrictOrCascade?
+    ;
+
+databaseComment
+@init { msgs.push("database's comment"); }
+@after { msgs.pop(); }
+    : KW_COMMENT comment=StringLiteral
+    ;
+
+createTableStatement
+@init { msgs.push("create table statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE (ext=KW_EXTERNAL)? KW_TABLE ifNotExists? name=tableName
+      (  like=KW_LIKE likeName=tableName
+         tableLocation?
+         tablePropertiesPrefixed?
+       | (LPAREN columnNameTypeList RPAREN)?
+         tableComment?
+         tablePartition?
+         tableBuckets?
+         tableSkewed?
+         tableRowFormat?
+         tableFileFormat?
+         tableLocation?
+         tablePropertiesPrefixed?
+         (KW_AS selectStatement)?
+      )
+    ;
+
+truncateTableStatement
+@init { msgs.push("truncate table statement"); }
+@after { msgs.pop(); }
+    : KW_TRUNCATE KW_TABLE tablePartitionPrefix 
+;
+
+createIndexStatement
+@init { msgs.push("create index statement");}
+@after {msgs.pop();}
+    : 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?
+    ;
+
+indexComment
+@init { msgs.push("comment on an index");}
+@after {msgs.pop();}
+        :
+                KW_COMMENT comment=StringLiteral  
+        ;
+
+autoRebuild
+@init { msgs.push("auto rebuild index");}
+@after {msgs.pop();}
+    : KW_WITH KW_DEFERRED KW_REBUILD
+    ;
+
+indexTblName
+@init { msgs.push("index table name");}
+@after {msgs.pop();}
+    : KW_IN KW_TABLE indexTbl=tableName
+    ;
+
+indexPropertiesPrefixed
+@init { msgs.push("table properties with prefix"); }
+@after { msgs.pop(); }
+    :
+        KW_IDXPROPERTIES indexProperties
+    ;
+
+indexProperties
+@init { msgs.push("index properties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN indexPropertiesList RPAREN 
+    ;
+
+indexPropertiesList
+@init { msgs.push("index properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    ;
+
+dropIndexStatement
+@init { msgs.push("drop index statement");}
+@after {msgs.pop();}
+    : KW_DROP KW_INDEX ifExists? indexName=identifier KW_ON tab=tableName
+    ;
+
+dropTableStatement
+@init { msgs.push("drop statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_TABLE ifExists? tableName 
+    ;
+
+alterStatement
+@init { msgs.push("alter statement"); }
+@after { msgs.pop(); }
+    : 
+    KW_ALTER
+        (
+            KW_TABLE alterTableStatementSuffix
+        |
+            KW_VIEW alterViewStatementSuffix
+        |
+            KW_INDEX alterIndexStatementSuffix
+        |
+            KW_DATABASE alterDatabaseStatementSuffix
+        )
+    ;
+
+alterTableStatementSuffix
+@init { msgs.push("alter table statement"); }
+@after { msgs.pop(); }
+    : alterStatementSuffixRename
+    | alterStatementSuffixAddCol
+    | alterStatementSuffixRenameCol
+    | alterStatementSuffixDropPartitions
+    | alterStatementSuffixAddPartitions
+    | alterStatementSuffixTouch
+    | alterStatementSuffixArchive
+    | alterStatementSuffixUnArchive
+    | alterStatementSuffixProperties
+    | alterTblPartitionStatement
+    | alterStatementSuffixSkewedby
+    ;
+
+alterViewStatementSuffix
+@init { msgs.push("alter view statement"); }
+@after { msgs.pop(); }
+    : alterViewSuffixProperties
+    | alterStatementSuffixRename
+    | alterStatementSuffixAddPartitions
+    | alterStatementSuffixDropPartitions
+    | name=tableName KW_AS selectStatement
+    ;
+
+alterIndexStatementSuffix
+@init { msgs.push("alter index statement"); }
+@after { msgs.pop(); }
+    : indexName=identifier
+      (KW_ON tableNameId=identifier)
+      partitionSpec?
+    (
+      KW_REBUILD
+    |
+      KW_SET KW_IDXPROPERTIES
+      indexProperties
+    )
+    ;
+
+alterDatabaseStatementSuffix
+@init { msgs.push("alter database statement"); }
+@after { msgs.pop(); }
+    : alterDatabaseSuffixProperties
+    ;
+
+alterDatabaseSuffixProperties
+@init { msgs.push("alter database properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_DBPROPERTIES dbProperties
+    ;
+
+alterStatementSuffixRename
+@init { msgs.push("rename statement"); }
+@after { msgs.pop(); }
+    : oldName=identifier KW_RENAME KW_TO newName=identifier
+    ;
+
+alterStatementSuffixAddCol
+@init { msgs.push("add column statement"); }
+@after { msgs.pop(); }
+    : identifier (add=KW_ADD | replace=KW_REPLACE) KW_COLUMNS LPAREN columnNameTypeList RPAREN
+    ;
+
+alterStatementSuffixRenameCol
+@init { msgs.push("rename column name"); }
+@after { msgs.pop(); }
+    : identifier KW_CHANGE KW_COLUMN? oldName=identifier newName=identifier colType (KW_COMMENT comment=StringLiteral)? alterStatementChangeColPosition?
+    ;
+
+alterStatementChangeColPosition
+    : first=KW_FIRST|KW_AFTER afterCol=identifier
+    ;
+
+alterStatementSuffixAddPartitions
+@init { msgs.push("add partition statement"); }
+@after { msgs.pop(); }
+    : identifier KW_ADD ifNotExists? partitionSpec partitionLocation? (partitionSpec partitionLocation?)*
+    ;
+
+alterStatementSuffixTouch
+@init { msgs.push("touch statement"); }
+@after { msgs.pop(); }
+    : identifier KW_TOUCH (partitionSpec)*
+    ;
+
+alterStatementSuffixArchive
+@init { msgs.push("archive statement"); }
+@after { msgs.pop(); }
+    : identifier KW_ARCHIVE (partitionSpec)*
+    ;
+
+alterStatementSuffixUnArchive
+@init { msgs.push("unarchive statement"); }
+@after { msgs.pop(); }
+    : identifier KW_UNARCHIVE (partitionSpec)*
+    ;
+
+partitionLocation
+@init { msgs.push("partition location"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+alterStatementSuffixDropPartitions
+@init { msgs.push("drop partition statement"); }
+@after { msgs.pop(); }
+    : identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
+    ;
+
+alterStatementSuffixProperties
+@init { msgs.push("alter properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
+    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    ;
+
+alterViewSuffixProperties
+@init { msgs.push("alter view properties statement"); }
+@after { msgs.pop(); }
+    : name=identifier KW_SET KW_TBLPROPERTIES tableProperties
+    | name=identifier KW_UNSET KW_TBLPROPERTIES ifExists? tableProperties
+    ;
+
+alterStatementSuffixSerdeProperties
+@init { msgs.push("alter serdes statement"); }
+@after { msgs.pop(); }
+    : KW_SET KW_SERDE serdeName=StringLiteral (KW_WITH KW_SERDEPROPERTIES tableProperties)?
+    | KW_SET KW_SERDEPROPERTIES tableProperties
+    ;
+
+tablePartitionPrefix
+@init {msgs.push("table partition prefix");}
+@after {msgs.pop();}
+  :name=identifier partitionSpec?
+  ;
+
+alterTblPartitionStatement
+@init {msgs.push("alter table partition statement");}
+@after {msgs.pop();}
+  : tablePartitionPrefix alterTblPartitionStatementSuffix
+  |Identifier KW_PARTITION KW_COLUMN LPAREN columnNameType RPAREN
+  ;
+
+alterTblPartitionStatementSuffix
+@init {msgs.push("alter table partition statement suffix");}
+@after {msgs.pop();}
+  : alterStatementSuffixFileFormat
+  | alterStatementSuffixLocation
+  | alterStatementSuffixProtectMode
+  | alterStatementSuffixMergeFiles
+  | alterStatementSuffixSerdeProperties
+  | alterStatementSuffixRenamePart
+  | alterStatementSuffixBucketNum
+  | alterTblPartitionStatementSuffixSkewedLocation
+  | alterStatementSuffixClusterbySortby
+  ;
+
+alterStatementSuffixFileFormat
+@init {msgs.push("alter fileformat statement"); }
+@after {msgs.pop();}
+	: KW_SET KW_FILEFORMAT fileFormat
+	;
+
+alterStatementSuffixClusterbySortby
+@init {msgs.push("alter partition cluster by sort by statement");}
+@after {msgs.pop();}
+  : KW_NOT KW_CLUSTERED 
+  | KW_NOT KW_SORTED 
+  | tableBuckets 
+  ;
+
+alterTblPartitionStatementSuffixSkewedLocation
+@init {msgs.push("alter partition skewed location");}
+@after {msgs.pop();}
+  : KW_SET KW_SKEWED KW_LOCATION skewedLocations
+  ;
+  
+skewedLocations
+@init { msgs.push("skewed locations"); }
+@after { msgs.pop(); }
+    :
+      LPAREN skewedLocationsList RPAREN 
+    ;
+
+skewedLocationsList
+@init { msgs.push("skewed locations list"); }
+@after { msgs.pop(); }
+    :
+      skewedLocationMap (COMMA skewedLocationMap)* 
+    ;
+
+skewedLocationMap
+@init { msgs.push("specifying skewed location map"); }
+@after { msgs.pop(); }
+    :
+      key=skewedValueLocationElement EQUAL value=StringLiteral 
+    ;
+
+alterStatementSuffixLocation
+@init {msgs.push("alter location");}
+@after {msgs.pop();}
+  : KW_SET KW_LOCATION newLoc=StringLiteral
+  ;
+
+	
+alterStatementSuffixSkewedby
+@init {msgs.push("alter skewed by statement");}
+@after{msgs.pop();}
+	:name=identifier tableSkewed
+	|
+	name=identifier KW_NOT KW_SKEWED
+	|
+	name=identifier KW_NOT storedAsDirs
+	;
+
+alterStatementSuffixProtectMode
+@init { msgs.push("alter partition protect mode statement"); }
+@after { msgs.pop(); }
+    : alterProtectMode
+    ;
+
+alterStatementSuffixRenamePart
+@init { msgs.push("alter table rename partition statement"); }
+@after { msgs.pop(); }
+    : KW_RENAME KW_TO partitionSpec
+    ;
+
+alterStatementSuffixMergeFiles
+@init { msgs.push(""); }
+@after { msgs.pop(); }
+    : KW_CONCATENATE
+    ;
+
+alterProtectMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_ENABLE alterProtectModeMode  
+    | KW_DISABLE alterProtectModeMode  
+    ;
+
+alterProtectModeMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_OFFLINE  
+    | KW_NO_DROP KW_CASCADE? 
+    | KW_READONLY  
+    ;
+
+alterStatementSuffixBucketNum
+@init { msgs.push(""); }
+@after { msgs.pop(); }
+    : KW_INTO num=Number KW_BUCKETS
+    ;
+
+fileFormat
+@init { msgs.push("file format specification"); }
+@after { msgs.pop(); }
+    : KW_SEQUENCEFILE  
+    | KW_TEXTFILE  
+    | KW_RCFILE  
+    | KW_ORCFILE 
+    | KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
+    | genericSpec=identifier 
+    ;
+
+tabTypeExpr
+@init { msgs.push("specifying table types"); }
+@after { msgs.pop(); }
+
+   : 
+   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))*
+   ;
+
+descTabTypeExpr
+@init { msgs.push("specifying describe table types"); }
+@after { msgs.pop(); }
+
+   : 
+   identifier (DOT (KW_ELEM_TYPE | KW_KEY_TYPE | KW_VALUE_TYPE | identifier))* identifier?
+   ;
+
+partTypeExpr
+@init { msgs.push("specifying table partitions"); }
+@after { msgs.pop(); }
+    :  tabTypeExpr partitionSpec? 
+    ;
+
+descPartTypeExpr
+@init { msgs.push("specifying describe table partitions"); }
+@after { msgs.pop(); }
+    :  descTabTypeExpr partitionSpec? 
+    ;
+
+descStatement
+@init { msgs.push("describe statement"); }
+@after { msgs.pop(); }
+    : (KW_DESCRIBE|KW_DESC) (descOptions=KW_FORMATTED|descOptions=KW_EXTENDED|descOptions=KW_PRETTY)? (parttype=descPartTypeExpr) 
+    | (KW_DESCRIBE|KW_DESC) KW_FUNCTION KW_EXTENDED? (name=descFuncNames) 
+    | (KW_DESCRIBE|KW_DESC) KW_DATABASE KW_EXTENDED? (dbName=identifier) 
+    ;
+
+analyzeStatement
+@init { msgs.push("analyze statement"); }
+@after { msgs.pop(); }
+    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) | (KW_FOR KW_COLUMNS statsColumnName=columnNameList))? 
+    ;
+
+showStatement
+@init { msgs.push("show statement"); }
+@after { msgs.pop(); }
+    : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? 
+    | KW_SHOW KW_TABLES ((KW_FROM|KW_IN) db_name=identifier)? (KW_LIKE showStmtIdentifier|showStmtIdentifier)?  
+    | KW_SHOW KW_COLUMNS (KW_FROM|KW_IN) tabname=tableName ((KW_FROM|KW_IN) db_name=identifier)? 
+    | KW_SHOW KW_FUNCTIONS showStmtIdentifier?  
+    | KW_SHOW KW_PARTITIONS identifier partitionSpec? 
+    | KW_SHOW KW_CREATE KW_TABLE tabName=tableName 
+    | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
+    | KW_SHOW KW_TBLPROPERTIES tblName=identifier (LPAREN prptyName=StringLiteral RPAREN)? 
+    | KW_SHOW KW_LOCKS (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? 
+    | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
+    ;
+
+lockStatement
+@init { msgs.push("lock statement"); }
+@after { msgs.pop(); }
+    : KW_LOCK KW_TABLE tableName partitionSpec? lockMode 
+    ;
+
+lockMode
+@init { msgs.push("lock mode"); }
+@after { msgs.pop(); }
+    : KW_SHARED | KW_EXCLUSIVE
+    ;
+
+unlockStatement
+@init { msgs.push("unlock statement"); }
+@after { msgs.pop(); }
+    : KW_UNLOCK KW_TABLE tableName partitionSpec?  
+    ;
+
+createRoleStatement
+@init { msgs.push("create role"); }
+@after { msgs.pop(); }
+    : KW_CREATE KW_ROLE roleName=identifier
+    ;
+
+dropRoleStatement
+@init {msgs.push("drop role");}
+@after {msgs.pop();}
+    : KW_DROP KW_ROLE roleName=identifier
+    ;
+
+grantPrivileges
+@init {msgs.push("grant privileges");}
+@after {msgs.pop();}
+    : KW_GRANT privList=privilegeList
+      privilegeObject?
+      KW_TO principalSpecification
+      (KW_WITH withOption)?
+    ;
+
+revokePrivileges
+@init {msgs.push("revoke privileges");}
+@afer {msgs.pop();}
+    : KW_REVOKE privilegeList privilegeObject? KW_FROM principalSpecification
+    ;
+
+grantRole
+@init {msgs.push("grant role");}
+@after {msgs.pop();}
+    : KW_GRANT KW_ROLE identifier (COMMA identifier)* KW_TO principalSpecification
+    ;
+
+revokeRole
+@init {msgs.push("revoke role");}
+@after {msgs.pop();}
+    : KW_REVOKE KW_ROLE identifier (COMMA identifier)* KW_FROM principalSpecification
+    ;
+
+showRoleGrants
+@init {msgs.push("show role grants");}
+@after {msgs.pop();}
+    : KW_SHOW KW_ROLE KW_GRANT principalName
+    ;
+
+showGrants
+@init {msgs.push("show grants");}
+@after {msgs.pop();}
+    : KW_SHOW KW_GRANT principalName privilegeIncludeColObject?
+    ;
+
+privilegeIncludeColObject
+@init {msgs.push("privilege object including columns");}
+@after {msgs.pop();}
+    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier (LPAREN cols=columnNameList RPAREN)? partitionSpec?
+    ;
+
+privilegeObject
+@init {msgs.push("privilege subject");}
+@after {msgs.pop();}
+    : KW_ON (table=KW_TABLE|KW_DATABASE) identifier partitionSpec?
+    ;
+
+privilegeList
+@init {msgs.push("grant privilege list");}
+@after {msgs.pop();}
+    : privlegeDef (COMMA privlegeDef)*
+    ;
+
+privlegeDef
+@init {msgs.push("grant privilege");}
+@after {msgs.pop();}
+    : privilegeType (LPAREN cols=columnNameList RPAREN)?
+    ;
+
+privilegeType
+@init {msgs.push("privilege type");}
+@after {msgs.pop();}
+    : KW_ALL 
+    | KW_ALTER 
+    | KW_UPDATE 
+    | KW_CREATE 
+    | KW_DROP 
+    | KW_INDEX 
+    | KW_LOCK 
+    | KW_SELECT 
+    | KW_SHOW_DATABASE 
+    ;
+
+principalSpecification
+@init { msgs.push("user/group/role name list"); }
+@after { msgs.pop(); }
+    : principalName (COMMA principalName)* 
+    ;
+
+principalName
+@init {msgs.push("user|group|role name");}
+@after {msgs.pop();}
+    : KW_USER identifier 
+    | KW_GROUP identifier 
+    | KW_ROLE identifier 
+    ;
+
+withOption
+@init {msgs.push("grant with option");}
+@after {msgs.pop();}
+    : KW_GRANT KW_OPTION
+    ;
+
+metastoreCheck
+@init { msgs.push("metastore check statement"); }
+@after { msgs.pop(); }
+    : KW_MSCK (repair=KW_REPAIR)? (KW_TABLE table=identifier partitionSpec? (COMMA partitionSpec)*)?
+    ;
+
+createFunctionStatement
+@init { msgs.push("create function statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE KW_TEMPORARY KW_FUNCTION identifier KW_AS StringLiteral
+    ;
+
+dropFunctionStatement
+@init { msgs.push("drop temporary function statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_TEMPORARY KW_FUNCTION ifExists? identifier
+    ;
+
+createViewStatement
+@init {
+    msgs.push("create view statement");
+}
+@after { msgs.pop(); }
+    : KW_CREATE (orReplace)? KW_VIEW (ifNotExists)? name=tableName
+        (LPAREN columnNameCommentList RPAREN)? tableComment? viewPartition?
+        tablePropertiesPrefixed?
+        KW_AS
+        selectStatement
+    ;
+
+viewPartition
+@init { msgs.push("view partition specification"); }
+@after { msgs.pop(); }
+    : KW_PARTITIONED KW_ON LPAREN columnNameList RPAREN
+    ;
+
+dropViewStatement
+@init { msgs.push("drop view statement"); }
+@after { msgs.pop(); }
+    : KW_DROP KW_VIEW ifExists? viewName 
+    ;
+
+showStmtIdentifier
+@init { msgs.push("identifier for show statement"); }
+@after { msgs.pop(); }
+    : identifier
+    | StringLiteral
+    ;
+
+tableComment
+@init { msgs.push("table's comment"); }
+@after { msgs.pop(); }
+    :
+      KW_COMMENT comment=StringLiteral  
+    ;
+
+tablePartition
+@init { msgs.push("table partition specification"); }
+@after { msgs.pop(); }
+    : KW_PARTITIONED KW_BY LPAREN columnNameTypeList RPAREN
+    ;
+
+tableBuckets
+@init { msgs.push("table buckets specification"); }
+@after { msgs.pop(); }
+    :
+      KW_CLUSTERED KW_BY LPAREN bucketCols=columnNameList RPAREN (KW_SORTED KW_BY LPAREN sortCols=columnNameOrderList RPAREN)? KW_INTO num=Number KW_BUCKETS
+    ;
+
+tableSkewed
+@init { msgs.push("table skewed specification"); }
+@after { msgs.pop(); }
+    :
+     KW_SKEWED KW_BY LPAREN skewedCols=columnNameList RPAREN KW_ON LPAREN (skewedValues=skewedValueElement) RPAREN (storedAsDirs)?
+    ;
+
+rowFormat
+@init { msgs.push("serde specification"); }
+@after { msgs.pop(); }
+    : rowFormatSerde 
+    | rowFormatDelimited 
+    ;
+
+recordReader
+@init { msgs.push("record reader specification"); }
+@after { msgs.pop(); }
+    : KW_RECORDREADER StringLiteral 
+    ;
+
+recordWriter
+@init { msgs.push("record writer specification"); }
+@after { msgs.pop(); }
+    : KW_RECORDWRITER StringLiteral 
+    ;
+
+rowFormatSerde
+@init { msgs.push("serde format specification"); }
+@after { msgs.pop(); }
+    : KW_ROW KW_FORMAT KW_SERDE name=StringLiteral (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
+    ;
+
+rowFormatDelimited
+@init { msgs.push("serde properties specification"); }
+@after { msgs.pop(); }
+    :
+      KW_ROW KW_FORMAT KW_DELIMITED tableRowFormatFieldIdentifier? tableRowFormatCollItemsIdentifier? tableRowFormatMapKeysIdentifier? tableRowFormatLinesIdentifier?
+    ;
+
+tableRowFormat
+@init { msgs.push("table row format specification"); }
+@after { msgs.pop(); }
+    :
+      rowFormatDelimited
+    | rowFormatSerde
+    ;
+
+tablePropertiesPrefixed
+@init { msgs.push("table properties with prefix"); }
+@after { msgs.pop(); }
+    :
+        KW_TBLPROPERTIES tableProperties
+    ;
+
+tableProperties
+@init { msgs.push("table properties"); }
+@after { msgs.pop(); }
+    :
+      LPAREN tablePropertiesList RPAREN 
+    ;
+
+tablePropertiesList
+@init { msgs.push("table properties list"); }
+@after { msgs.pop(); }
+    :
+      keyValueProperty (COMMA keyValueProperty)* 
+    |
+      keyProperty (COMMA keyProperty)* 
+    ;
+
+keyValueProperty
+@init { msgs.push("specifying key/value property"); }
+@after { msgs.pop(); }
+    :
+      key=StringLiteral EQUAL value=StringLiteral 
+    ;
+
+keyProperty
+@init { msgs.push("specifying key property"); }
+@after { msgs.pop(); }
+    :
+      key=StringLiteral 
+    ;
+
+tableRowFormatFieldIdentifier
+@init { msgs.push("table row format's field separator"); }
+@after { msgs.pop(); }
+    :
+      KW_FIELDS KW_TERMINATED KW_BY fldIdnt=StringLiteral (KW_ESCAPED KW_BY fldEscape=StringLiteral)?
+    ;
+
+tableRowFormatCollItemsIdentifier
+@init { msgs.push("table row format's column separator"); }
+@after { msgs.pop(); }
+    :
+      KW_COLLECTION KW_ITEMS KW_TERMINATED KW_BY collIdnt=StringLiteral
+    ;
+
+tableRowFormatMapKeysIdentifier
+@init { msgs.push("table row format's map key separator"); }
+@after { msgs.pop(); }
+    :
+      KW_MAP KW_KEYS KW_TERMINATED KW_BY mapKeysIdnt=StringLiteral
+    ;
+
+tableRowFormatLinesIdentifier
+@init { msgs.push("table row format's line separator"); }
+@after { msgs.pop(); }
+    :
+      KW_LINES KW_TERMINATED KW_BY linesIdnt=StringLiteral
+    ;
+
+tableFileFormat
+@init { msgs.push("table file format specification"); }
+@after { msgs.pop(); }
+    :
+      KW_STORED KW_AS KW_SEQUENCEFILE  
+      | KW_STORED KW_AS KW_TEXTFILE  
+      | KW_STORED KW_AS KW_RCFILE  
+      | KW_STORED KW_AS KW_ORCFILE 
+      | KW_STORED KW_AS KW_INPUTFORMAT inFmt=StringLiteral KW_OUTPUTFORMAT outFmt=StringLiteral (KW_INPUTDRIVER inDriver=StringLiteral KW_OUTPUTDRIVER outDriver=StringLiteral)?
+      | KW_STORED KW_BY storageHandler=StringLiteral
+         (KW_WITH KW_SERDEPROPERTIES serdeprops=tableProperties)?
+      | KW_STORED KW_AS genericSpec=identifier
+    ;
+
+tableLocation
+@init { msgs.push("table location specification"); }
+@after { msgs.pop(); }
+    :
+      KW_LOCATION locn=StringLiteral 
+    ;
+
+columnNameTypeList
+@init { msgs.push("column name type list"); }
+@after { msgs.pop(); }
+    : columnNameType (COMMA columnNameType)* 
+    ;
+
+columnNameColonTypeList
+@init { msgs.push("column name type list"); }
+@after { msgs.pop(); }
+    : columnNameColonType (COMMA columnNameColonType)* 
+    ;
+
+columnNameList
+@init { msgs.push("column name list"); }
+@after { msgs.pop(); }
+    : columnName (COMMA columnName)* 
+    ;
+
+columnName
+@init { msgs.push("column name"); }
+@after { msgs.pop(); }
+    :
+      identifier
+    ;
+
+columnNameOrderList
+@init { msgs.push("column name order list"); }
+@after { msgs.pop(); }
+    : columnNameOrder (COMMA columnNameOrder)* 
+    ;
+
+skewedValueElement
+@init { msgs.push("skewed value element"); }
+@after { msgs.pop(); }
+    : 
+      skewedColumnValues
+     | skewedColumnValuePairList
+    ;
+
+skewedColumnValuePairList
+@init { msgs.push("column value pair list"); }
+@after { msgs.pop(); }
+    : skewedColumnValuePair (COMMA skewedColumnValuePair)* 
+    ;
+
+skewedColumnValuePair
+@init { msgs.push("column value pair"); }
+@after { msgs.pop(); }
+    : 
+      LPAREN colValues=skewedColumnValues RPAREN 
+    ;
+
+skewedColumnValues
+@init { msgs.push("column values"); }
+@after { msgs.pop(); }
+    : skewedColumnValue (COMMA skewedColumnValue)* 
+    ;
+
+skewedColumnValue
+@init { msgs.push("column value"); }
+@after { msgs.pop(); }
+    :
+      constant
+    ;
+
+skewedValueLocationElement
+@init { msgs.push("skewed value location element"); }
+@after { msgs.pop(); }
+    : 
+      skewedColumnValue
+     | skewedColumnValuePair
+    ;
+    
+columnNameOrder
+@init { msgs.push("column name order"); }
+@after { msgs.pop(); }
+    : identifier (asc=KW_ASC | desc=KW_DESC)?
+    ;
+
+columnNameCommentList
+@init { msgs.push("column name comment list"); }
+@after { msgs.pop(); }
+    : columnNameComment (COMMA columnNameComment)* 
+    ;
+
+columnNameComment
+@init { msgs.push("column name comment"); }
+@after { msgs.pop(); }
+    : colName=identifier (KW_COMMENT comment=StringLiteral)?
+    ;
+
+columnRefOrder
+@init { msgs.push("column order"); }
+@after { msgs.pop(); }
+    : expression (asc=KW_ASC | desc=KW_DESC)?
+    ;
+
+columnNameType
+@init { msgs.push("column specification"); }
+@after { msgs.pop(); }
+    : colName=identifier colType (KW_COMMENT comment=StringLiteral)?
+    ;
+
+columnNameColonType
+@init { msgs.push("column specification"); }
+@after { msgs.pop(); }
+    : colName=identifier COLON colType (KW_COMMENT comment=StringLiteral)?
+    ;
+
+colType
+@init { msgs.push("column type"); }
+@after { msgs.pop(); }
+    : type
+    ;
+
+colTypeList
+@init { msgs.push("column type list"); }
+@after { msgs.pop(); }
+    : colType (COMMA colType)* 
+    ;
+
+type
+    : primitiveType
+    | listType
+    | structType
+    | mapType
+    | unionType;
+
+primitiveType
+@init { msgs.push("primitive type specification"); }
+@after { msgs.pop(); }
+    : KW_TINYINT       
+    | KW_SMALLINT      
+    | KW_INT           
+    | KW_BIGINT        
+    | KW_BOOLEAN       
+    | KW_FLOAT         
+    | KW_DOUBLE        
+    | KW_DATE          
+    | KW_DATETIME      
+    | KW_TIMESTAMP     
+    | KW_STRING        
+    | KW_BINARY        
+    | KW_DECIMAL       
+    ;
+
+listType
+@init { msgs.push("list type"); }
+@after { msgs.pop(); }
+    : KW_ARRAY LESSTHAN type GREATERTHAN   
+    ;
+
+structType
+@init { msgs.push("struct type"); }
+@after { msgs.pop(); }
+    : KW_STRUCT LESSTHAN columnNameColonTypeList GREATERTHAN 
+    ;
+
+mapType
+@init { msgs.push("map type"); }
+@after { msgs.pop(); }
+    : KW_MAP LESSTHAN left=primitiveType COMMA right=type GREATERTHAN
+    ;
+
+unionType
+@init { msgs.push("uniontype type"); }
+@after { msgs.pop(); }
+    : KW_UNIONTYPE LESSTHAN colTypeList GREATERTHAN 
+    ;
+
+queryOperator
+@init { msgs.push("query operator"); }
+@after { msgs.pop(); }
+    : KW_UNION KW_ALL 
+    ;
+
+// select statement select ... from ... where ... group by ... order by ...
+queryStatementExpression
+    : 
+    queryStatement (queryOperator queryStatement)*
+    ;
+
+queryStatement
+    :
+    fromClause
+    ( b+=body )+ 
+    | regular_body
+    ;
+
+regular_body
+   :
+   insertClause
+   selectClause
+   fromClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   |
+   selectStatement
+   ;
+
+selectStatement
+   :
+   selectClause
+   fromClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   ;
+
+
+body
+   :
+   insertClause
+   selectClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   |
+   selectClause
+   whereClause?
+   groupByClause?
+   havingClause?
+   orderByClause?
+   clusterByClause?
+   distributeByClause?
+   sortByClause?
+   window_clause?
+   limitClause? 
+   ;
+
+insertClause
+@init { msgs.push("insert clause"); }
+@after { msgs.pop(); }
+   :
+     KW_INSERT KW_OVERWRITE destination ifNotExists? 
+   | KW_INSERT KW_INTO KW_TABLE tableOrPartition
+   ;
+
+destination
+@init { msgs.push("destination specification"); }
+@after { msgs.pop(); }
+   :
+     KW_LOCAL KW_DIRECTORY StringLiteral tableRowFormat? tableFileFormat? 
+   | KW_DIRECTORY StringLiteral 
+   | KW_TABLE tableOrPartition 
+   ;
+
+limitClause
+@init { msgs.push("limit clause"); }
+@after { msgs.pop(); }
+   :
+   KW_LIMIT num=Number 
+   ;
+
+    
\ No newline at end of file


[5/5] git commit: TAJO-622: Add TM mark and navigation links required for TLP project.

Posted by hy...@apache.org.
TAJO-622: Add TM mark and navigation links required for TLP project.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tajo/commit/7f5d5eca
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/7f5d5eca
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/7f5d5eca

Branch: refs/heads/branch-0.8.0
Commit: 7f5d5eca5cd6407a6151815a28b9ad3d403e097d
Parents: e8da943
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sun Feb 23 14:14:41 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sun Feb 23 15:40:48 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +++
 tajo-project/src/site/apt/index.apt             |  16 ++++++++++++----
 .../src/site/resources/images/Tajo_logo.png     | Bin 13672 -> 14487 bytes
 tajo-project/src/site/site.xml                  |   7 +++++++
 4 files changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/7f5d5eca/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b5a2efd..bfbe617 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -463,6 +463,9 @@ Release 0.8.0 - unreleased
 
   TASKS
 
+    TAJO-622: Add TM mark and navigation links required for TLP project.
+    (hyunsik)
+
     TAJO-578: Update configuration for tajo-site.xml. (jaehwa)
 
     TAJO-530: Fix warnings in tajo-catalog. (jaehwa)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/7f5d5eca/tajo-project/src/site/apt/index.apt
----------------------------------------------------------------------
diff --git a/tajo-project/src/site/apt/index.apt b/tajo-project/src/site/apt/index.apt
index fee2130..00a6c5c 100644
--- a/tajo-project/src/site/apt/index.apt
+++ b/tajo-project/src/site/apt/index.apt
@@ -14,14 +14,14 @@
 ~~ See the License for the specific language governing permissions and
 ~~ limitations under the License.
 
-Tajo - Big Data Warehouse System on Hadoop
+Apache Tajo (TM) - An open source big data warehouse system in Hadoop
 
-	Tajo is a distributed data warehouse system on Hadoop that provides
+	Apache Tajo (TM) is an advanced open source data warehouse system on Hadoop that provides
 	low-latency and scalable ad-hoc queries and ETL on large-data sets stored on HDFS and other data sources.
 
 Features
 
-  * <<Scalability and low latency>>
+  * <<Interactive and Batch Queries>>
 
     * Fully distributed SQL query processing on large data sets stored in HDFS and other data sources
 
@@ -33,6 +33,12 @@ Features
 
     * Dynamic scheduling support that handles struggling and heterogeneous cluster nodes
 
+  * <<Query Optimization>>
+
+    * Cost-based optimization for bushy join trees
+
+    * Progressive query optimization for reoptimizing running queries
+
   * <<ETL>>
 
     * ETL features that transform one data format to another data format
@@ -66,6 +72,8 @@ Features
 
 News
 
+  * <<[2014-01-02]>> Keuntae Park was invited to become a new committer.
+
   * <<[2013-11-20]>> Tajo 0.2.0-incubating Released. Now available for {{{http://apache.org/dyn/closer.cgi/incubator/tajo/tajo-0.2.0-incubating/}download}}!
 
   * <<[2013-10-15]>> Tajo was presented at {{{http://www.meetup.com/hadoop/events/146077932/}Bay Area Hadoop User Group - LinkedIn Special Event}}.
@@ -86,7 +94,7 @@ Downloads
 
 Disclaimer
 
-  Apache Tajo is an effort undergoing incubation at The Apache Software Foundation (ASF)
+  Apache Tajo (TM) is an effort undergoing incubation at The Apache Software Foundation (ASF)
   sponsored by the Apache Incubator PMC. Incubation is required of all newly accepted projects
   until a further review indicates that the infrastructure, communications, and decision making
   process have stabilized in a manner consistent with other successful ASF projects.

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/7f5d5eca/tajo-project/src/site/resources/images/Tajo_logo.png
----------------------------------------------------------------------
diff --git a/tajo-project/src/site/resources/images/Tajo_logo.png b/tajo-project/src/site/resources/images/Tajo_logo.png
index a52b9b9..61e5eca 100644
Binary files a/tajo-project/src/site/resources/images/Tajo_logo.png and b/tajo-project/src/site/resources/images/Tajo_logo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/7f5d5eca/tajo-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/tajo-project/src/site/site.xml b/tajo-project/src/site/site.xml
index 0ef2cb6..e89dcca 100644
--- a/tajo-project/src/site/site.xml
+++ b/tajo-project/src/site/site.xml
@@ -108,6 +108,13 @@
       <item name="Coding Style" href="http://wiki.apache.org/tajo/CodingStyle" />
     </menu>
 
+    <menu name="Apache">
+      <item name="Web Site" href="http://www.apache.org/" />
+      <item name="Sponsorship" href="http://www.apache.org/foundation/sponsorship.html" />
+      <item name="Thanks" href="http://www.apache.org/foundation/thanks.html" />
+      <item name="Security" href="http://www.apache.org/security/" />
+    </menu>
+
     <footer>
       <div class="row span12">
         Apache Tajo, Apache Hadoop, Apache, the Apache feather logo, and the Apache incubator logo are