Author: hashutosh
Date: Sat Nov 23 18:21:01 2013
New Revision: 1544857

URL: http://svn.apache.org/r1544857
Log:
HIVE-5614 : Subquery support: allow subquery expressions in having clause 
(Harish Butani via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/test/queries/clientpositive/subquery_exists_having.q
    hive/trunk/ql/src/test/queries/clientpositive/subquery_in_having.q
    hive/trunk/ql/src/test/queries/clientpositive/subquery_notexists_having.q
    hive/trunk/ql/src/test/queries/clientpositive/subquery_notin_having.q
    hive/trunk/ql/src/test/results/clientpositive/subquery_exists_having.q.out
    hive/trunk/ql/src/test/results/clientpositive/subquery_in_having.q.out
    
hive/trunk/ql/src/test/results/clientpositive/subquery_notexists_having.q.out
    hive/trunk/ql/src/test/results/clientpositive/subquery_notin_having.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/trunk/ql/src/test/results/clientpositive/subquery_multiinsert.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java?rev=1544857&r1=1544856&r2=1544857&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QB.java Sat Nov 23 
18:21:01 2013
@@ -69,6 +69,14 @@ public class QB {
    * If this QB represents a SubQuery predicate then this will point to the 
SubQuery object.
    */
   private QBSubQuery subQueryPredicateDef;
+  
+       /*
+        * used to give a unique name to each SubQuery QB Currently there can 
be at
+        * most 2 SubQueries in a Query: 1 in the Where clause, and 1 in the 
Having
+        * clause.
+        */
+       private int numSubQueryPredicates;
+  
 
   // results
 
@@ -320,5 +328,13 @@ public class QB {
   protected QBSubQuery getSubQueryPredicateDef() {
     return subQueryPredicateDef;
   }
+  
+       protected int getNumSubQueryPredicates() {
+               return numSubQueryPredicates;
+       }
+
+       protected int incrNumSubQueryPredicates() {
+               return ++numSubQueryPredicates;
+       }
 
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java?rev=1544857&r1=1544856&r2=1544857&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java 
(original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java Sat 
Nov 23 18:21:01 2013
@@ -4,11 +4,14 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import 
org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory.DefaultExprProcessor;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -124,30 +127,56 @@ public class QBSubQuery {
     public abstract ExprType combine(ExprType other);
   }
 
+  /*
+   * This class captures the information about a 
+   * conjunct in the where clause of the SubQuery.
+   * For a equality predicate it capture for each side:
+   * - the AST
+   * - the type of Expression (basically what columns are referenced)
+   * - for Expressions that refer the parent it captures the 
+   *   parent's ColumnInfo. In case of outer Aggregation expressions
+   *   we need this to introduce a new mapping in the OuterQuery
+   *   RowResolver. A join condition must use qualified column references,
+   *   so we generate a new name for the aggr expression and use it in the 
+   *   joining condition.
+   *   For e.g.
+   *   having exists ( select x from R2 where y = min(R1.z) )
+   *   where the expression 'min(R1.z)' is from the outer Query.
+   *   We give this expression a new name like 'R1._gby_sq_col_1'
+   *   and use the join condition: R1._gby_sq_col_1 = R2.y
+   */
   static class Conjunct {
     private final ASTNode leftExpr;
     private final ASTNode rightExpr;
     private final ExprType leftExprType;
     private final ExprType rightExprType;
+    private final ColumnInfo leftOuterColInfo;
+    private final ColumnInfo rightOuterColInfo;
 
-    public Conjunct(ASTNode leftExpr, ASTNode rightExpr, ExprType leftExprType,
-        ExprType rightExprType) {
+   Conjunct(ASTNode leftExpr, 
+        ASTNode rightExpr, 
+        ExprType leftExprType,
+        ExprType rightExprType,
+        ColumnInfo leftOuterColInfo,
+        ColumnInfo rightOuterColInfo) {
       super();
       this.leftExpr = leftExpr;
       this.rightExpr = rightExpr;
       this.leftExprType = leftExprType;
       this.rightExprType = rightExprType;
+      this.leftOuterColInfo = leftOuterColInfo;
+      this.rightOuterColInfo = rightOuterColInfo;
     }
-    public ASTNode getLeftExpr() {
+    ASTNode getLeftExpr() {
       return leftExpr;
     }
-    public ASTNode getRightExpr() {
+    ASTNode getRightExpr() {
       return rightExpr;
     }
-    public ExprType getLeftExprType() {
+    ExprType getLeftExprType() {
       return leftExprType;
     }
-    public ExprType getRightExprType() {
+    ExprType getRightExprType() {
       return rightExprType;
     }
 
@@ -173,16 +202,28 @@ public class QBSubQuery {
       }
       return leftExprType.combine(rightExprType) == ExprType.REFERS_PARENT;
     }
+    ColumnInfo getLeftOuterColInfo() {
+      return leftOuterColInfo;
+    }
+    ColumnInfo getRightOuterColInfo() {
+      return rightOuterColInfo;
+    }
   }
 
   class ConjunctAnalyzer {
     RowResolver parentQueryRR;
+    boolean forHavingClause;
+    String parentQueryNewAlias;
     NodeProcessor defaultExprProcessor;
     Stack<Node> stack;
 
-    ConjunctAnalyzer(RowResolver parentQueryRR) {
+    ConjunctAnalyzer(RowResolver parentQueryRR,
+               boolean forHavingClause,
+               String parentQueryNewAlias) {
       this.parentQueryRR = parentQueryRR;
       defaultExprProcessor = new DefaultExprProcessor();
+      this.forHavingClause = forHavingClause;
+      this.parentQueryNewAlias = parentQueryNewAlias;
       stack = new Stack<Node>();
     }
 
@@ -195,25 +236,34 @@ public class QBSubQuery {
      * 3. All other expressions have a Type based on their children.
      *    An Expr w/o children is assumed to refer to neither.
      */
-    private ExprType analyzeExpr(ASTNode expr) {
-      ExprNodeDesc exprNode;
+    private ObjectPair<ExprType,ColumnInfo> analyzeExpr(ASTNode expr) {
+      ColumnInfo cInfo = null;
+      if ( forHavingClause ) {
+       try {
+         cInfo = parentQueryRR.getExpression(expr);
+               if ( cInfo != null) {
+                   return ObjectPair.create(ExprType.REFERS_PARENT, cInfo);
+           }
+       } catch(SemanticException se) {
+       }
+      }
       if ( expr.getType() == HiveParser.DOT) {
         ASTNode dot = firstDot(expr);
-        exprNode = resolveDot(dot);
-        if ( exprNode != null ) {
-          return ExprType.REFERS_PARENT;
+        cInfo = resolveDot(dot);
+        if ( cInfo != null ) {
+          return ObjectPair.create(ExprType.REFERS_PARENT, cInfo);
         }
-        return ExprType.REFERS_SUBQUERY;
+        return ObjectPair.create(ExprType.REFERS_SUBQUERY, null);
       } else if ( expr.getType() == HiveParser.TOK_TABLE_OR_COL ) {
-        return ExprType.REFERS_SUBQUERY;
+        return ObjectPair.create(ExprType.REFERS_SUBQUERY, null);
       } else {
         ExprType exprType = ExprType.REFERS_NONE;
         int cnt = expr.getChildCount();
         for(int i=0; i < cnt; i++) {
           ASTNode child = (ASTNode) expr.getChild(i);
-          exprType = exprType.combine(analyzeExpr(child));
+          exprType = exprType.combine(analyzeExpr(child).getFirst());
         }
-        return exprType;
+        return ObjectPair.create(exprType, null);
       }
     }
 
@@ -234,13 +284,17 @@ public class QBSubQuery {
       if ( type == HiveParser.EQUAL ) {
         ASTNode left = (ASTNode) conjunct.getChild(0);
         ASTNode right = (ASTNode) conjunct.getChild(1);
-        ExprType leftType = analyzeExpr(left);
-        ExprType rightType = analyzeExpr(right);
+        ObjectPair<ExprType,ColumnInfo> leftInfo = analyzeExpr(left);
+        ObjectPair<ExprType,ColumnInfo> rightInfo = analyzeExpr(right);
 
-        return new Conjunct(left, right, leftType, rightType);
+        return new Conjunct(left, right, 
+            leftInfo.getFirst(), rightInfo.getFirst(),
+            leftInfo.getSecond(), rightInfo.getSecond());
       } else {
-        ExprType sqExprType = analyzeExpr(conjunct);
-        return new Conjunct(conjunct, null, sqExprType, null);
+        ObjectPair<ExprType,ColumnInfo> sqExprInfo = analyzeExpr(conjunct);
+        return new Conjunct(conjunct, null, 
+            sqExprInfo.getFirst(), null,
+            sqExprInfo.getSecond(), sqExprInfo.getSecond());
       }
     }
 
@@ -248,16 +302,20 @@ public class QBSubQuery {
      * Try to resolve a qualified name as a column reference on the Parent 
Query's RowResolver.
      * Apply this logic on the leftmost(first) dot in an AST tree.
      */
-    protected ExprNodeDesc resolveDot(ASTNode node) {
+    protected ColumnInfo resolveDot(ASTNode node) {
       try {
         TypeCheckCtx tcCtx = new TypeCheckCtx(parentQueryRR);
         String str = 
BaseSemanticAnalyzer.unescapeIdentifier(node.getChild(1).getText());
         ExprNodeDesc idDesc = new 
ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, str);
-        return (ExprNodeDesc)
-            defaultExprProcessor.process(node, stack, tcCtx, (Object) null, 
idDesc);
+         ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc)
+             defaultExprProcessor.process(node, stack, tcCtx, (Object) null, 
idDesc);
+         if ( colDesc != null ) {
+           String[] qualName = 
parentQueryRR.reverseLookup(colDesc.getColumn());
+           return parentQueryRR.get(qualName[0], qualName[1]);
+         }
       } catch(SemanticException se) {
-        return null;
       }
+      return null;
     }
 
     /*
@@ -295,6 +353,8 @@ public class QBSubQuery {
   private int numOfCorrelationExprsAddedToSQSelect;
 
   private boolean groupbyAddedToSQ;
+  
+  private int numOuterCorrExprsForHaving;
 
   public QBSubQuery(String outerQueryId,
       int sqIdx,
@@ -311,6 +371,7 @@ public class QBSubQuery {
     this.sqIdx = sqIdx;
     this.alias = "sq_" + this.sqIdx;
     this.numCorrExprsinSQ = 0;
+    this.numOuterCorrExprsForHaving = 0;
     String s = ctx.getTokenRewriteStream().toString(
         originalSQAST.getTokenStartIndex(), originalSQAST.getTokenStopIndex());
     originalSQASTOrigin = new ASTNodeOrigin("SubQuery", alias, s, alias, 
originalSQAST);
@@ -328,7 +389,9 @@ public class QBSubQuery {
     return operator;
   }
 
-  void validateAndRewriteAST(RowResolver outerQueryRR) throws 
SemanticException {
+  void validateAndRewriteAST(RowResolver outerQueryRR,
+                 boolean forHavingClause,
+                 String outerQueryAlias) throws SemanticException {
 
     ASTNode selectClause = (ASTNode) subQueryAST.getChild(1).getChild(1);
 
@@ -359,7 +422,7 @@ public class QBSubQuery {
       containsAggregationExprs = containsAggregationExprs | ( r == 1 );
     }
 
-    rewrite(outerQueryRR);
+    rewrite(outerQueryRR, forHavingClause, outerQueryAlias);
 
     SubQueryUtils.setOriginDeep(subQueryAST, originalSQASTOrigin);
 
@@ -418,14 +481,28 @@ public class QBSubQuery {
     }
   }
 
-  void buildJoinCondition(RowResolver outerQueryRR, RowResolver sqRR) throws 
SemanticException {
+  void buildJoinCondition(RowResolver outerQueryRR, RowResolver sqRR,
+                 boolean forHavingClause,
+                 String outerQueryAlias) throws SemanticException {
     ASTNode parentQueryJoinCond = null;
 
     if ( parentQueryExpression != null ) {
+      
+      ColumnInfo outerQueryCol = null;
+      try {
+        outerQueryCol = outerQueryRR.getExpression(parentQueryExpression);
+      } catch(SemanticException se) {
+      }
+      
       parentQueryJoinCond = SubQueryUtils.buildOuterQryToSQJoinCond(
         getOuterQueryExpression(),
         alias,
         sqRR);
+      
+      if ( outerQueryCol != null ) {
+        rewriteCorrConjunctForHaving(parentQueryJoinCond, true, 
+            outerQueryAlias, outerQueryRR, outerQueryCol);
+      }
     }
     joinConditionAST = SubQueryUtils.andAST(parentQueryJoinCond, 
joinConditionAST);
     setJoinType();
@@ -494,8 +571,25 @@ public class QBSubQuery {
    *       expression to its GroupBy; add it to the front of the GroupBy.
    *   - If predicate is not correlated, let it remain in the SubQuery
    *     where clause.
+   * Additional things for Having clause:
+   * - A correlation predicate may refer to an aggregation expression.
+   * - This introduces 2 twists to the rewrite:
+   *   a. When analyzing equality predicates we need to analyze each side 
+   *      to see if it is an aggregation expression from the Outer Query.
+   *      So for e.g. this is a valid correlation predicate:
+   *         R2.x = min(R1.y)
+   *      Where R1 is an outer table reference, and R2 is a SubQuery table 
reference.
+   *   b. When hoisting the correlation predicate to a join predicate, we need 
to
+   *      rewrite it to be in the form the Join code allows: so the predicte 
needs
+   *      to contain a qualified column references.
+   *      We handle this by generating a new name for the aggregation 
expression,
+   *      like R1._gby_sq_col_1 and adding this mapping to the Outer Query's
+   *      Row Resolver. Then we construct a joining predicate using this new 
+   *      name; so in our e.g. the condition would be: R2.x = R1._gby_sq_col_1
    */
-  private void rewrite(RowResolver parentQueryRR) throws SemanticException {
+  private void rewrite(RowResolver parentQueryRR,
+                 boolean forHavingClause,
+                 String outerQueryAlias) throws SemanticException {
     ASTNode selectClause = (ASTNode) subQueryAST.getChild(1).getChild(1);
     ASTNode whereClause = null;
     if ( subQueryAST.getChild(1).getChildCount() > 2 &&
@@ -511,7 +605,8 @@ public class QBSubQuery {
     List<ASTNode> conjuncts = new ArrayList<ASTNode>();
     SubQueryUtils.extractConjuncts(searchCond, conjuncts);
 
-    ConjunctAnalyzer conjunctAnalyzer = new ConjunctAnalyzer(parentQueryRR);
+    ConjunctAnalyzer conjunctAnalyzer = new ConjunctAnalyzer(parentQueryRR,
+               forHavingClause, outerQueryAlias);
     ASTNode sqNewSearchCond = null;
 
     for(ASTNode conjunctAST : conjuncts) {
@@ -545,6 +640,10 @@ public class QBSubQuery {
         ASTNode sqExprForCorr = SubQueryUtils.createColRefAST(alias, 
exprAlias);
 
         if ( conjunct.getLeftExprType().refersSubQuery() ) {
+          if ( forHavingClause && conjunct.getRightOuterColInfo() != null ) {
+            rewriteCorrConjunctForHaving(conjunctAST, false, outerQueryAlias, 
+                parentQueryRR, conjunct.getRightOuterColInfo());
+          }
           ASTNode joinPredciate = SubQueryUtils.alterCorrelatedPredicate(
               conjunctAST, sqExprForCorr, true);
           joinConditionAST = SubQueryUtils.andAST(joinConditionAST, 
joinPredciate);
@@ -557,6 +656,10 @@ public class QBSubQuery {
             SubQueryUtils.addGroupExpressionToFront(gBy, 
conjunct.getLeftExpr());
           }
         } else {
+          if ( forHavingClause && conjunct.getLeftOuterColInfo() != null ) {
+            rewriteCorrConjunctForHaving(conjunctAST, true, outerQueryAlias, 
+                parentQueryRR, conjunct.getLeftOuterColInfo());
+          }
           ASTNode joinPredciate = SubQueryUtils.alterCorrelatedPredicate(
               conjunctAST, sqExprForCorr, false);
           joinConditionAST = SubQueryUtils.andAST(joinConditionAST, 
joinPredciate);
@@ -642,4 +745,21 @@ public class QBSubQuery {
   public int getNumOfCorrelationExprsAddedToSQSelect() {
     return numOfCorrelationExprsAddedToSQSelect;
   }
+  
+  private void rewriteCorrConjunctForHaving(ASTNode conjunctASTNode,
+      boolean refersLeft,
+      String outerQueryAlias,
+      RowResolver outerQueryRR,
+      ColumnInfo outerQueryCol) {
+    
+    String newColAlias = "_gby_sq_col_" + numOuterCorrExprsForHaving++;
+    ASTNode outerExprForCorr = SubQueryUtils.createColRefAST(outerQueryAlias, 
newColAlias);
+    if ( refersLeft ) {
+      conjunctASTNode.setChild(0, outerExprForCorr);
+    } else {
+      conjunctASTNode.setChild(1, outerExprForCorr);
+    }
+    outerQueryRR.put(outerQueryAlias, newColAlias, outerQueryCol);
+  }
+      
 }

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=1544857&r1=1544856&r2=1544857&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 
Sat Nov 23 18:21:01 2013
@@ -1897,7 +1897,8 @@ public class SemanticAnalyzer extends Ba
   }
 
   @SuppressWarnings("nls")
-  private Operator genHavingPlan(String dest, QB qb, Operator input)
+  private Operator genHavingPlan(String dest, QB qb, Operator input,
+      Map<String, Operator> aliasToOpInfo)
       throws SemanticException {
 
     ASTNode havingExpr = qb.getParseInfo().getHavingForClause(dest);
@@ -1912,21 +1913,24 @@ public class SemanticAnalyzer extends Ba
     }
     ASTNode condn = (ASTNode) havingExpr.getChild(0);
 
-    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new FilterDesc(genExprNodeDesc(condn, inputRR), false), new RowSchema(
-            inputRR.getColumnInfos()), input), inputRR);
-
+    /*
+     * Now a having clause can contain a SubQuery predicate;
+     * so we invoke genFilterPlan to handle SubQuery algebraic transformation,
+     * just as is done for SubQuery predicates appearing in the Where Clause.
+     */
+    Operator output = genFilterPlan(condn, qb, input, aliasToOpInfo, true);
+    output = putOpInsertMap(output, inputRR);
     return output;
   }
 
   @SuppressWarnings("nls")
-  private Operator genFilterPlan(String dest, QB qb, Operator input,
-      Map<String, Operator> aliasToOpInfo)
+  private Operator genFilterPlan(ASTNode searchCond, QB qb, Operator input,
+      Map<String, Operator> aliasToOpInfo,
+      boolean forHavingClause)
       throws SemanticException {
 
     OpParseContext inputCtx = opParseCtx.get(input);
     RowResolver inputRR = inputCtx.getRowResolver();
-    ASTNode whereExpr = qb.getParseInfo().getWhrForClause(dest);
 
     /*
      * Handling of SubQuery Expressions:
@@ -1949,8 +1953,16 @@ public class SemanticAnalyzer extends Ba
      *     --> ===CONTINUE_FILTER_PROCESSING===
      *   endif
      * endif
+     * 
+     * Support for Sub Queries in Having Clause:
+     * - By and large this works the same way as SubQueries in the Where 
Clause.
+     * - The one addum is the handling of aggregation expressions from the 
Outer Query
+     *   appearing in correlation clauses. 
+     *   - So such correlating predicates are allowed:
+     *        min(OuterQuert.x) = SubQuery.y
+     *   - this requires special handling when converting to joins. See 
QBSubQuery.rewrite
+     *     method method for detailed comments. 
      */
-    ASTNode searchCond = (ASTNode) whereExpr.getChild(0);
     List<ASTNode> subQueriesInOriginalTree = 
SubQueryUtils.findSubQueries(searchCond);
 
     if ( subQueriesInOriginalTree.size() > 0 ) {
@@ -1982,13 +1994,20 @@ public class SemanticAnalyzer extends Ba
         ASTNode subQueryAST = subQueries.get(i);
         ASTNode originalSubQueryAST = subQueriesInOriginalTree.get(i);
 
-        int sqIdx = i+1;
+        int sqIdx = qb.incrNumSubQueryPredicates();
         clonedSearchCond = 
SubQueryUtils.rewriteParentQueryWhere(clonedSearchCond, subQueryAST);
 
         QBSubQuery subQuery = SubQueryUtils.buildSubQuery(qb.getId(),
             sqIdx, subQueryAST, originalSubQueryAST, ctx);
+        
+        String havingInputAlias = null;
+        
+        if ( forHavingClause ) {
+               havingInputAlias = "gby_sq" + sqIdx;
+               aliasToOpInfo.put(havingInputAlias, input);
+        }
 
-        subQuery.validateAndRewriteAST(inputRR);
+        subQuery.validateAndRewriteAST(inputRR, forHavingClause, 
havingInputAlias);
 
         QB qbSQ = new QB(subQuery.getOuterQueryId(), subQuery.getAlias(), 
true);
         qbSQ.setSubQueryDef(subQuery);
@@ -2014,7 +2033,7 @@ public class SemanticAnalyzer extends Ba
         /*
          * Gen Join between outer Operator and SQ op
          */
-        subQuery.buildJoinCondition(inputRR, sqRR);
+        subQuery.buildJoinCondition(inputRR, sqRR, forHavingClause, 
havingInputAlias);
         QBJoinTree joinTree = genSQJoinTree(qb, subQuery,
             input,
             aliasToOpInfo);
@@ -4410,14 +4429,15 @@ public class SemanticAnalyzer extends Ba
       curr = forwardOp;
 
       if (parseInfo.getWhrForClause(dest) != null) {
-        curr = genFilterPlan(dest, qb, forwardOp, aliasToOpInfo);
+        ASTNode whereExpr = qb.getParseInfo().getWhrForClause(dest);
+        curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, forwardOp, 
aliasToOpInfo, false);
       }
 
       // Generate GroupbyOperator
       Operator groupByOperatorInfo = genGroupByPlanGroupByOperator(parseInfo,
           dest, curr, reduceSinkOperatorInfo, GroupByDesc.Mode.COMPLETE, null);
 
-      curr = genPostGroupByBodyPlan(groupByOperatorInfo, dest, qb);
+      curr = genPostGroupByBodyPlan(groupByOperatorInfo, dest, qb, 
aliasToOpInfo);
     }
 
     return curr;
@@ -7777,7 +7797,8 @@ public class SemanticAnalyzer extends Ba
               curr = inputs.get(dest);
 
               if (qbp.getWhrForClause(dest) != null) {
-                curr = genFilterPlan(dest, qb, curr, aliasToOpInfo);
+                ASTNode whereExpr = qb.getParseInfo().getWhrForClause(dest);
+                curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, 
curr, aliasToOpInfo, false);
               }
 
               if (qbp.getAggregationExprsForClause(dest).size() != 0
@@ -7804,7 +7825,7 @@ public class SemanticAnalyzer extends Ba
                 }
               }
 
-              curr = genPostGroupByBodyPlan(curr, dest, qb);
+              curr = genPostGroupByBodyPlan(curr, dest, qb, aliasToOpInfo);
             }
           } else {
             curr = genGroupByPlan1ReduceMultiGBY(commonGroupByDestGroup, qb, 
input, aliasToOpInfo);
@@ -7830,7 +7851,8 @@ public class SemanticAnalyzer extends Ba
     return inputs;
   }
 
-  private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb)
+  private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb,
+      Map<String, Operator> aliasToOpInfo)
       throws SemanticException {
 
     QBParseInfo qbp = qb.getParseInfo();
@@ -7840,7 +7862,7 @@ public class SemanticAnalyzer extends Ba
       if (getGroupByForClause(qbp, dest).size() == 0) {
         throw new SemanticException("HAVING specified without GROUP BY");
       }
-      curr = genHavingPlan(dest, qb, curr);
+      curr = genHavingPlan(dest, qb, curr, aliasToOpInfo);
     }
 
 

Added: hive/trunk/ql/src/test/queries/clientpositive/subquery_exists_having.q
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/subquery_exists_having.q?rev=1544857&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/subquery_exists_having.q 
(added)
+++ hive/trunk/ql/src/test/queries/clientpositive/subquery_exists_having.q Sat 
Nov 23 18:21:01 2013
@@ -0,0 +1,60 @@
+
+
+-- no agg, corr
+explain
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+-- view test
+create view cv1 as 
+select b.key, count(*) as c
+from src b
+group by b.key
+having exists
+  (select a.key
+  from src a
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+select * from cv1;
+
+-- sq in from
+select *
+from (select b.key, count(*) 
+  from src b 
+  group by b.key
+  having exists 
+    (select a.key 
+    from src a 
+    where a.key = b.key and a.value > 'val_9'
+    )
+) a
+;
+
+-- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having exists ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/subquery_in_having.q
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/subquery_in_having.q?rev=1544857&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/subquery_in_having.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/subquery_in_having.q Sat Nov 
23 18:21:01 2013
@@ -0,0 +1,102 @@
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table 
part;
+
+-- non agg, non corr
+explain
+ select key, count(*) 
+from src 
+group by key
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by 
s1.key )
+;
+
+
+select s1.key, count(*) from src s1 where s1.key > '9' group by s1.key;
+
+select key, count(*) 
+from src 
+group by key
+having count(*) in (select count(*) from src s1 where s1.key = '90' group by 
s1.key )
+;
+
+-- non agg, corr
+explain
+ select key, value, count(*) 
+from src b
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9'  and 
s1.value = b.value group by s1.key )
+;
+
+-- agg, non corr
+explain
+select p_mfgr, avg(p_size)
+from part b
+group by b.p_mfgr
+having b.p_mfgr in 
+   (select p_mfgr 
+    from part
+    group by p_mfgr
+    having max(p_size) - min(p_size) < 20
+   )
+;
+
+-- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having b.key in ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+;
+
+-- where and having
+-- Plan is:
+-- Stage 1: b semijoin sq1:src (subquery in where)
+-- Stage 2: group by Stage 1 o/p
+-- Stage 5: group by on sq2:src (subquery in having)
+-- Stage 6: Stage 2 o/p semijoin Stage 5
+explain
+select key, value, count(*) 
+from src b
+where b.key in (select key from src where src.key > '8')
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by 
s1.key )
+;
+
+set hive.auto.convert.join=true;
+-- Plan is:
+-- Stage  5: group by on sq2:src (subquery in having)
+-- Stage 10: hashtable for sq1:src (subquery in where)
+-- Stage  2: b map-side semijoin Stage 10 o/p
+-- Stage  3: Stage 2 semijoin Stage 5
+-- Stage  9: construct hastable for Stage 5 o/p
+-- Stage  6: Stage 2 map-side semijoin Stage 9
+explain
+select key, value, count(*) 
+from src b
+where b.key in (select key from src where src.key > '8')
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by 
s1.key )
+;
+
+-- non agg, non corr, windowing
+explain
+select p_mfgr, p_name, avg(p_size) 
+from part 
+group by p_mfgr, p_name
+having p_name in 
+  (select first_value(p_name) over(partition by p_mfgr order by p_size) from 
part)
+;

Added: hive/trunk/ql/src/test/queries/clientpositive/subquery_notexists_having.q
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/subquery_notexists_having.q?rev=1544857&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/subquery_notexists_having.q 
(added)
+++ hive/trunk/ql/src/test/queries/clientpositive/subquery_notexists_having.q 
Sat Nov 23 18:21:01 2013
@@ -0,0 +1,46 @@
+
+
+-- no agg, corr
+explain
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+;
+
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+;
+
+
+-- distinct, corr
+explain
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select distinct a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_12'
+  )
+;
+
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select distinct a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_12'
+  )
+;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/subquery_notin_having.q
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/subquery_notin_having.q?rev=1544857&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/subquery_notin_having.q 
(added)
+++ hive/trunk/ql/src/test/queries/clientpositive/subquery_notin_having.q Sat 
Nov 23 18:21:01 2013
@@ -0,0 +1,74 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table 
part;
+
+
+-- non agg, non corr
+explain
+select key, count(*) 
+from src 
+group by key
+having key not in  
+  ( select key  from src s1 
+    where s1.key > '12'
+  )
+;
+
+-- non agg, corr
+explain
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, 
avg(p_retailprice) a from part group by p_mfgr) a 
+  where min(p_retailprice) = l and r - l > 600
+  )
+;
+
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, 
avg(p_retailprice) a from part group by p_mfgr) a 
+  where min(p_retailprice) = l and r - l > 600
+  )
+;
+
+-- agg, non corr
+explain
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from part a
+  group by p_mfgr
+  having max(p_retailprice) - min(p_retailprice) > 600
+  )
+;
+
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from part a
+  group by p_mfgr
+  having max(p_retailprice) - min(p_retailprice) > 600
+  )
+;
\ No newline at end of file

Added: 
hive/trunk/ql/src/test/results/clientpositive/subquery_exists_having.q.out
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/subquery_exists_having.q.out?rev=1544857&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/subquery_exists_having.q.out 
(added)
+++ hive/trunk/ql/src/test/results/clientpositive/subquery_exists_having.q.out 
Sat Nov 23 18:21:01 2013
@@ -0,0 +1,290 @@
+PREHOOK: query: -- no agg, corr
+explain
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+PREHOOK: type: QUERY
+POSTHOOK: query: -- no agg, corr
+explain
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) b)) (TOK_INSERT 
(TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. 
(TOK_TABLE_OR_COL b) key)) (TOK_SELEXPR (TOK_FUNCTIONSTAR count))) (TOK_GROUPBY 
(. (TOK_TABLE_OR_COL b) key)) (TOK_HAVING (TOK_SUBQUERY_EXPR (TOK_SUBQUERY_OP 
exists) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) a)) (TOK_INSERT 
(TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. 
(TOK_TABLE_OR_COL a) key))) (TOK_WHERE (and (= (. (TOK_TABLE_OR_COL a) key) (. 
(TOK_TABLE_OR_COL b) key)) (> (. (TOK_TABLE_OR_COL a) value) 'val_9')))))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        b 
+          TableScan
+            alias: b
+            Select Operator
+              expressions:
+                    expr: key
+                    type: string
+              outputColumnNames: key
+              Group By Operator
+                aggregations:
+                      expr: count()
+                bucketGroup: false
+                keys:
+                      expr: key
+                      type: string
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions:
+                        expr: _col0
+                        type: string
+                  sort order: +
+                  Map-reduce partition columns:
+                        expr: _col0
+                        type: string
+                  tag: -1
+                  value expressions:
+                        expr: _col1
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          keys:
+                expr: KEY._col0
+                type: string
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          File Output Operator
+            compressed: false
+            GlobalTableId: 0
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Alias -> Map Operator Tree:
+        $INTNAME 
+          TableScan
+            Reduce Output Operator
+              key expressions:
+                    expr: _col0
+                    type: string
+              sort order: +
+              Map-reduce partition columns:
+                    expr: _col0
+                    type: string
+              tag: 0
+              value expressions:
+                    expr: _col0
+                    type: string
+                    expr: _col1
+                    type: bigint
+        sq_1:a 
+          TableScan
+            alias: a
+            Filter Operator
+              predicate:
+                  expr: (value > 'val_9')
+                  type: boolean
+              Select Operator
+                expressions:
+                      expr: key
+                      type: string
+                outputColumnNames: _col1
+                Group By Operator
+                  bucketGroup: false
+                  keys:
+                        expr: _col1
+                        type: string
+                  mode: hash
+                  outputColumnNames: _col0
+                  Reduce Output Operator
+                    key expressions:
+                          expr: _col0
+                          type: string
+                    sort order: +
+                    Map-reduce partition columns:
+                          expr: _col0
+                          type: string
+                    tag: 1
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Semi Join 0 to 1
+          condition expressions:
+            0 {VALUE._col0} {VALUE._col1}
+            1 
+          handleSkewJoin: false
+          outputColumnNames: _col0, _col1
+          Filter Operator
+            predicate:
+                expr: (1 = 1)
+                type: boolean
+            Select Operator
+              expressions:
+                    expr: _col0
+                    type: string
+                    expr: _col1
+                    type: bigint
+              outputColumnNames: _col0, _col1
+              File Output Operator
+                compressed: false
+                GlobalTableId: 0
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: 
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+90     3
+92     1
+95     2
+96     1
+97     2
+98     2
+PREHOOK: query: -- view test
+create view cv1 as 
+select b.key, count(*) as c
+from src b
+group by b.key
+having exists
+  (select a.key
+  from src a
+  where a.key = b.key and a.value > 'val_9'
+  )
+PREHOOK: type: CREATEVIEW
+POSTHOOK: query: -- view test
+create view cv1 as 
+select b.key, count(*) as c
+from src b
+group by b.key
+having exists
+  (select a.key
+  from src a
+  where a.key = b.key and a.value > 'val_9'
+  )
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Output: default@cv1
+PREHOOK: query: select * from cv1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cv1
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select * from cv1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cv1
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+90     3
+92     1
+95     2
+96     1
+97     2
+98     2
+PREHOOK: query: -- sq in from
+select *
+from (select b.key, count(*) 
+  from src b 
+  group by b.key
+  having exists 
+    (select a.key 
+    from src a 
+    where a.key = b.key and a.value > 'val_9'
+    )
+) a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- sq in from
+select *
+from (select b.key, count(*) 
+  from src b 
+  group by b.key
+  having exists 
+    (select a.key 
+    from src a 
+    where a.key = b.key and a.value > 'val_9'
+    )
+) a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+90     3
+92     1
+95     2
+96     1
+97     2
+98     2
+PREHOOK: query: -- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having exists ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having exists ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+90     val_90
+92     val_92
+95     val_95
+96     val_96
+97     val_97
+98     val_98


Reply via email to