You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ke...@apache.org on 2012/10/14 04:04:44 UTC

svn commit: r1397994 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/ java/org/apache/hadoop/hive/ql/parse/ test/queries/clientnegative/ test/results/clientnegative/

Author: kevinwilfong
Date: Sun Oct 14 02:04:43 2012
New Revision: 1397994

URL: http://svn.apache.org/viewvc?rev=1397994&view=rev
Log:
HIVE-3573. Revert HIVE-3268. (njain via kevinwilfong)

Removed:
    hive/trunk/ql/src/test/queries/clientnegative/expr_clusterby1.q
    hive/trunk/ql/src/test/queries/clientnegative/expr_distributeby1.q
    hive/trunk/ql/src/test/queries/clientnegative/expr_distributeby_sortby_1.q
    hive/trunk/ql/src/test/queries/clientnegative/expr_orderby1.q
    hive/trunk/ql/src/test/queries/clientnegative/expr_sortby1.q
    hive/trunk/ql/src/test/results/clientnegative/expr_clusterby1.q.out
    hive/trunk/ql/src/test/results/clientnegative/expr_distributeby1.q.out
    hive/trunk/ql/src/test/results/clientnegative/expr_distributeby_sortby_1.q.out
    hive/trunk/ql/src/test/results/clientnegative/expr_orderby1.q.out
    hive/trunk/ql/src/test/results/clientnegative/expr_sortby1.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java?rev=1397994&r1=1397993&r2=1397994&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java Sun Oct 14 02:04:43 2012
@@ -233,15 +233,6 @@ public enum ErrorMsg {
     "If you really want to perform the operation, either remove the " +
     "mapjoin hint from your query or set hive.enforce.bucketmapjoin to false."),
 
-  EXPRESSIONS_NOT_ALLOWED_CLUSTERBY(10137,
-    "Expressions are not allowed in a cluster by clause. Use a column alias instead"),
-  EXPRESSIONS_NOT_ALLOWED_DISTRIBUTEBY(10138,
-    "Expressions are not allowed in a distribute by clause. Use a column alias instead"),
-  EXPRESSIONS_NOT_ALLOWED_ORDERBY(10139,
-    "Expressions are not allowed in an order by clause. Use a column alias instead"),
-  EXPRESSIONS_NOT_ALLOWED_SORTBY(10140,
-    "Expressions are not allowed in a sort by clause. Use a column alias instead"),
-
   BUCKETED_TABLE_METADATA_INCORRECT(10141,
    "Bucketed table metadata is not correct. " +
     "Fix the metadata or don't use bucketed mapjoin, by setting " +

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java?rev=1397994&r1=1397993&r2=1397994&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java Sun Oct 14 02:04:43 2012
@@ -490,11 +490,4 @@ public class QBParseInfo {
   public HashMap<String, TableSample> getNameToSample() {
     return nameToSample;
   }
-
-  protected static enum ClauseType {
-    CLUSTER_BY_CLAUSE,
-    DISTRIBUTE_BY_CLAUSE,
-    ORDER_BY_CLAUSE,
-    SORT_BY_CLAUSE
-  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1397994&r1=1397993&r2=1397994&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Sun Oct 14 02:04:43 2012
@@ -114,7 +114,6 @@ import org.apache.hadoop.hive.ql.optimiz
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalOptimizer;
 import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.tableSpec.SpecType;
-import org.apache.hadoop.hive.ql.parse.QBParseInfo.ClauseType;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
@@ -6278,93 +6277,6 @@ public class SemanticAnalyzer extends Ba
     return curr;
   }
 
-  // Expressions are not allowed currently in cluster/distribute/order/sort by.
-  // It would be good to support them in the future, but till then it is better
-  // to throw a good semantic error instead of some crpytic error.
-  private void checkExpression(ASTNode input,
-    ClauseType clauseType) throws SemanticException {
-    int childCount = input.getChildCount();
-
-    // Columns can only exist at the top
-    if (input.getType() == HiveParser.TOK_TABLE_OR_COL) {
-      switch (clauseType) {
-        case CLUSTER_BY_CLAUSE:
-          throw new
-            SemanticException(ErrorMsg.EXPRESSIONS_NOT_ALLOWED_CLUSTERBY.getMsg());
-        case DISTRIBUTE_BY_CLAUSE:
-          throw new
-            SemanticException(ErrorMsg.EXPRESSIONS_NOT_ALLOWED_DISTRIBUTEBY.getMsg());
-        case ORDER_BY_CLAUSE:
-          throw new
-            SemanticException(ErrorMsg.EXPRESSIONS_NOT_ALLOWED_ORDERBY.getMsg());
-        case SORT_BY_CLAUSE:
-          throw new
-            SemanticException(ErrorMsg.EXPRESSIONS_NOT_ALLOWED_SORTBY.getMsg());
-      }
-    }
-
-    if (childCount > 0) {
-      for (int pos = 0; pos < childCount; pos++) {
-        ASTNode exprChild = (ASTNode) input.getChild(pos);
-        checkExpression(exprChild, clauseType);
-      }
-    }
-  }
-
-  private void validateExpressionSkipParent(ASTNode inputExpr,
-    ClauseType clauseType) throws SemanticException {
-    int childCount = inputExpr.getChildCount();
-    if (childCount > 0) {
-      for (int pos = 0; pos < childCount; pos++) {
-        checkExpression((ASTNode)inputExpr.getChild(pos), clauseType);
-      }
-    }
-  }
-
-  private void validateExpressionHandleTableQualifier(ASTNode inputExpr,
-    ClauseType clauseType) throws SemanticException {
-    // If the expression is tab.column, go to the columns
-    // Same for value[3]
-    if ((inputExpr.getType() == HiveParser.DOT) ||
-        (inputExpr.getType() == HiveParser.LSQUARE)) {
-      for (int pos = 0; pos < inputExpr.getChildCount(); pos++) {
-        validateExpressionHandleTableQualifier((ASTNode)inputExpr.getChild(pos), clauseType);
-      }
-    } else {
-      validateExpressionSkipParent(inputExpr, clauseType);
-    }
-  }
-
-  // Validate that the expression only consists of constants and columns.
-  // Expressions are not allowed in the cluster/distribute/order/sort by list
-  private void validateExpression(ASTNode expr,
-    ClauseType clauseType) throws SemanticException {
-
-    boolean isGrandChild = true;
-    // The first level of children is whether it is ascending/descending
-    // for order by and sort by
-    if ((clauseType == ClauseType.DISTRIBUTE_BY_CLAUSE) ||
-        (clauseType == ClauseType.CLUSTER_BY_CLAUSE)) {
-      isGrandChild = false;
-    }
-
-    int ccount = expr.getChildCount();
-    for (int i = 0; i < ccount; ++i) {
-      ASTNode cl = (ASTNode) expr.getChild(i);
-      if (isGrandChild == false) {
-        validateExpressionHandleTableQualifier(cl, clauseType);
-      } else {
-        int grandChildCount = cl.getChildCount();
-        if (grandChildCount > 0) {
-          for (int childPos = 0; childPos < grandChildCount; childPos++) {
-            validateExpressionHandleTableQualifier(
-              (ASTNode)cl.getChild(childPos), clauseType);
-          }
-        }
-      }
-    }
-  }
-
   private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb)
       throws SemanticException {
 
@@ -6391,26 +6303,18 @@ public class SemanticAnalyzer extends Ba
     // order by or a sort by clause. For each of the above clause types, check
     // if the clause contains any expression.
     if (qbp.getClusterByForClause(dest) != null) {
-      validateExpression(qbp.getClusterByForClause(dest),
-        ClauseType.CLUSTER_BY_CLAUSE);
       genReduceSink = true;
     }
 
     if (qbp.getDistributeByForClause(dest) != null) {
-      validateExpression(qbp.getDistributeByForClause(dest),
-        ClauseType.DISTRIBUTE_BY_CLAUSE);
       genReduceSink = true;
     }
 
     if (qbp.getOrderByForClause(dest) != null) {
-      validateExpression(qbp.getOrderByForClause(dest),
-        ClauseType.ORDER_BY_CLAUSE);
       genReduceSink = true;
     }
 
     if (qbp.getSortByForClause(dest) != null) {
-      validateExpression(qbp.getSortByForClause(dest),
-        ClauseType.SORT_BY_CLAUSE);
       genReduceSink = true;
     }