Skip to content

Commit

Permalink
SPARK-2180: support HAVING clauses in Hive queries
Browse files Browse the repository at this point in the history
This PR extends Spark's HiveQL support to handle HAVING clauses in aggregations.  The HAVING test from the Hive compatibility suite doesn't appear to be runnable from within Spark, so I added a simple comparable test to `HiveQuerySuite`.

Author: William Benton <willb@redhat.com>

Closes #1136 from willb/SPARK-2180 and squashes the following commits:

3bbaf26 [William Benton] Added casts to HAVING expressions
83f1340 [William Benton] scalastyle fixes
18387f1 [William Benton] Add test for HAVING without GROUP BY
b880bef [William Benton] Added semantic error for HAVING without GROUP BY
942428e [William Benton] Added test coverage for SPARK-2180.
56084cc [William Benton] Add support for HAVING clauses in Hive queries.
  • Loading branch information
willb authored and rxin committed Jun 20, 2014
1 parent 6a224c3 commit 171ebb3
Show file tree
Hide file tree
Showing 2 changed files with 53 additions and 6 deletions.
30 changes: 24 additions & 6 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,9 @@ private[hive] object HiveQl {
class ParseException(sql: String, cause: Throwable)
extends Exception(s"Failed to parse: $sql", cause)

class SemanticException(msg: String)
extends Exception(s"Error in semantic analysis: $msg")

/**
* Returns the AST for the given SQL string.
*/
Expand Down Expand Up @@ -480,6 +483,7 @@ private[hive] object HiveQl {
whereClause ::
groupByClause ::
orderByClause ::
havingClause ::
sortByClause ::
clusterByClause ::
distributeByClause ::
Expand All @@ -494,6 +498,7 @@ private[hive] object HiveQl {
"TOK_WHERE",
"TOK_GROUPBY",
"TOK_ORDERBY",
"TOK_HAVING",
"TOK_SORTBY",
"TOK_CLUSTERBY",
"TOK_DISTRIBUTEBY",
Expand Down Expand Up @@ -576,21 +581,34 @@ private[hive] object HiveQl {
val withDistinct =
if (selectDistinctClause.isDefined) Distinct(withProject) else withProject

val withHaving = havingClause.map { h =>

if (groupByClause == None) {
throw new SemanticException("HAVING specified without GROUP BY")
}

val havingExpr = h.getChildren.toSeq match {
case Seq(hexpr) => nodeToExpr(hexpr)
}

Filter(Cast(havingExpr, BooleanType), withDistinct)
}.getOrElse(withDistinct)

val withSort =
(orderByClause, sortByClause, distributeByClause, clusterByClause) match {
case (Some(totalOrdering), None, None, None) =>
Sort(totalOrdering.getChildren.map(nodeToSortOrder), withDistinct)
Sort(totalOrdering.getChildren.map(nodeToSortOrder), withHaving)
case (None, Some(perPartitionOrdering), None, None) =>
SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withDistinct)
SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withHaving)
case (None, None, Some(partitionExprs), None) =>
Repartition(partitionExprs.getChildren.map(nodeToExpr), withDistinct)
Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving)
case (None, Some(perPartitionOrdering), Some(partitionExprs), None) =>
SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder),
Repartition(partitionExprs.getChildren.map(nodeToExpr), withDistinct))
Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving))
case (None, None, None, Some(clusterExprs)) =>
SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)),
Repartition(clusterExprs.getChildren.map(nodeToExpr), withDistinct))
case (None, None, None, None) => withDistinct
Repartition(clusterExprs.getChildren.map(nodeToExpr), withHaving))
case (None, None, None, None) => withHaving
case _ => sys.error("Unsupported set of ordering / distribution clauses.")
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,32 @@ class HiveQuerySuite extends HiveComparisonTest {
TestHive.reset()
}

test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") {
val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3))
.zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)}

TestHive.sparkContext.parallelize(fixture).registerAsTable("having_test")

val results =
hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3")
.collect()
.map(x => Pair(x.getString(0), x.getInt(1)))

assert(results === Array(Pair("foo", 4)))

TestHive.reset()
}

test("SPARK-2180: HAVING without GROUP BY raises exception") {
intercept[Exception] {
hql("SELECT value, attr FROM having_test HAVING attr > 3")
}
}

test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") {
val results = hql("select key, count(*) c from src group by key having c").collect()
}

test("Query Hive native command execution result") {
val tableName = "test_native_commands"

Expand Down Expand Up @@ -441,3 +467,6 @@ class HiveQuerySuite extends HiveComparisonTest {
// since they modify /clear stuff.

}

// for SPARK-2180 test
case class HavingRow(key: Int, value: String, attr: Int)

0 comments on commit 171ebb3

Please sign in to comment.