Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 1631064) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -13833,16 +13833,12 @@ } // 5. Bailout if select involves UDTF - ASTNode udtfExpr = (ASTNode) selExprList.getChild(posn).getChild(0); - GenericUDTF genericUDTF = null; - int udtfExprType = udtfExpr.getType(); - if (udtfExprType == HiveParser.TOK_FUNCTION || udtfExprType == HiveParser.TOK_FUNCTIONSTAR) { - String funcName = TypeCheckProcFactory.DefaultExprProcessor.getFunctionText(udtfExpr, true); + ASTNode expr = (ASTNode) selExprList.getChild(posn).getChild(0); + int exprType = expr.getType(); + if (exprType == HiveParser.TOK_FUNCTION || exprType == HiveParser.TOK_FUNCTIONSTAR) { + String funcName = TypeCheckProcFactory.DefaultExprProcessor.getFunctionText(expr, true); FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName); - if (fi != null) { - genericUDTF = fi.getGenericUDTF(); - } - if (genericUDTF != null) { + if (fi != null && fi.getGenericUDTF() != null) { String msg = String.format("UDTF " + funcName + " is currently not supported in CBO," + " turn off cbo to use UDTF " + funcName); LOG.debug(msg); @@ -13870,7 +13866,6 @@ ErrorMsg.INVALID_AS.getMsg())); } - ASTNode expr; String tabAlias; String colAlias; @@ -13907,6 +13902,9 @@ pos = genColListRegex(unescapeIdentifier(expr.getChild(1).getText()), unescapeIdentifier(expr.getChild(0).getChild(0).getText().toLowerCase()), expr, col_list, inputRR, pos, out_rwsch, tabAliasesForAllProjs, subQuery); + } else if (expr.toStringTree().contains("TOK_FUNCTIONDI") && !(srcRel instanceof HiveAggregateRel)) { + // Likely a malformed query eg, select hash(distinct c1) from t1; + throw new OptiqSemanticException("Distinct without an aggreggation."); } else { // Case when this is an expression TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR); @@ -14119,6 +14117,10 @@ ASTNode havingClause = qbp.getHavingForClause(qbp.getClauseNames().iterator().next()); if (havingClause != null) { + if (!(srcRel instanceof HiveAggregateRel)) { + // ill-formed query like select * from t1 having c1 > 0; + throw new OptiqSemanticException("Having clause without any group-by."); + } validateNoHavingReferenceToAlias(qb, (ASTNode) havingClause.getChild(0)); gbFilter = genFilterRelNode(qb, (ASTNode) havingClause.getChild(0), srcRel, aliasToRel, true);