-
Notifications
You must be signed in to change notification settings - Fork 28.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-12718][SPARK-13720][SQL] SQL generation support for window functions #11555
Changes from all commits
3ce072b
e037814
9a66fbb
40bd17a
276a870
656a13a
c82229a
054f50a
dab7a2f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,7 +42,7 @@ case class SubqueryHolder(query: String) extends LeafExpression with Unevaluable | |
} | ||
|
||
/** | ||
* A builder class used to convert a resolved logical plan into a SQL query string. Note that this | ||
* A builder class used to convert a resolved logical plan into a SQL query string. Note that not | ||
* all resolved logical plan are convertible. They either don't have corresponding SQL | ||
* representations (e.g. logical plans that operate on local Scala collections), or are simply not | ||
* supported by this builder (yet). | ||
|
@@ -103,6 +103,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
case p: Aggregate => | ||
aggregateToSQL(p) | ||
|
||
case w: Window => | ||
windowToSQL(w) | ||
|
||
case Limit(limitExpr, child) => | ||
s"${toSQL(child)} LIMIT ${limitExpr.sql}" | ||
|
||
|
@@ -123,12 +126,12 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
build(toSQL(p.child), "TABLESAMPLE(" + fraction + " PERCENT)") | ||
} | ||
|
||
case p: Filter => | ||
val whereOrHaving = p.child match { | ||
case Filter(condition, child) => | ||
val whereOrHaving = child match { | ||
case _: Aggregate => "HAVING" | ||
case _ => "WHERE" | ||
} | ||
build(toSQL(p.child), whereOrHaving, p.condition.sql) | ||
build(toSQL(child), whereOrHaving, condition.sql) | ||
|
||
case p @ Distinct(u: Union) if u.children.length > 1 => | ||
val childrenSql = u.children.map(c => s"(${toSQL(c)})") | ||
|
@@ -179,7 +182,7 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
build( | ||
toSQL(p.child), | ||
if (p.global) "ORDER BY" else "SORT BY", | ||
p.order.map { case SortOrder(e, dir) => s"${e.sql} ${dir.sql}" }.mkString(", ") | ||
p.order.map(_.sql).mkString(", ") | ||
) | ||
|
||
case p: RepartitionByExpression => | ||
|
@@ -268,9 +271,8 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
case attr: Attribute if groupByAttrMap.contains(attr) => groupByAttrMap(attr) | ||
} | ||
} | ||
val groupingSetSQL = | ||
"GROUPING SETS(" + | ||
groupingSet.map(e => s"(${e.map(_.sql).mkString(", ")})").mkString(", ") + ")" | ||
val groupingSetSQL = "GROUPING SETS(" + | ||
groupingSet.map(e => s"(${e.map(_.sql).mkString(", ")})").mkString(", ") + ")" | ||
|
||
val aggExprs = agg.aggregateExpressions.map { case expr => | ||
expr.transformDown { | ||
|
@@ -297,22 +299,50 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
) | ||
} | ||
|
||
private def windowToSQL(w: Window): String = { | ||
build( | ||
"SELECT", | ||
(w.child.output ++ w.windowExpressions).map(_.sql).mkString(", "), | ||
if (w.child == OneRowRelation) "" else "FROM", | ||
toSQL(w.child) | ||
) | ||
} | ||
|
||
object Canonicalizer extends RuleExecutor[LogicalPlan] { | ||
override protected def batches: Seq[Batch] = Seq( | ||
Batch("Canonicalizer", FixedPoint(100), | ||
Batch("Collapse Project", FixedPoint(100), | ||
// The `WidenSetOperationTypes` analysis rule may introduce extra `Project`s over | ||
// `Aggregate`s to perform type casting. This rule merges these `Project`s into | ||
// `Aggregate`s. | ||
CollapseProject, | ||
|
||
CollapseProject), | ||
Batch("Recover Scoping Info", Once, | ||
// Used to handle other auxiliary `Project`s added by analyzer (e.g. | ||
// `ResolveAggregateFunctions` rule) | ||
RecoverScopingInfo | ||
AddSubquery, | ||
// Previous rule will add extra sub-queries, this rule is used to re-propagate and update | ||
// the qualifiers bottom up, e.g.: | ||
// | ||
// Sort | ||
// ordering = t1.a | ||
// Project | ||
// projectList = [t1.a, t1.b] | ||
// Subquery gen_subquery | ||
// child ... | ||
// | ||
// will be transformed to: | ||
// | ||
// Sort | ||
// ordering = gen_subquery.a | ||
// Project | ||
// projectList = [gen_subquery.a, gen_subquery.b] | ||
// Subquery gen_subquery | ||
// child ... | ||
UpdateQualifiers | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Which test is for this new rule? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The above test is for verifying this rule. The JIRA SPARK-13720 describes the reason why we need to add this rule. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks! @cloud-fan Maybe it is good to add an example at here? |
||
) | ||
) | ||
|
||
object RecoverScopingInfo extends Rule[LogicalPlan] { | ||
override def apply(tree: LogicalPlan): LogicalPlan = tree transform { | ||
object AddSubquery extends Rule[LogicalPlan] { | ||
override def apply(tree: LogicalPlan): LogicalPlan = tree transformUp { | ||
// This branch handles aggregate functions within HAVING clauses. For example: | ||
// | ||
// SELECT key FROM src GROUP BY key HAVING max(value) > "val_255" | ||
|
@@ -324,8 +354,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
// +- Filter ... | ||
// +- Aggregate ... | ||
// +- MetastoreRelation default, src, None | ||
case plan @ Project(_, Filter(_, _: Aggregate)) => | ||
wrapChildWithSubquery(plan) | ||
case plan @ Project(_, Filter(_, _: Aggregate)) => wrapChildWithSubquery(plan) | ||
|
||
case w @ Window(_, _, _, _, Filter(_, _: Aggregate)) => wrapChildWithSubquery(w) | ||
|
||
case plan @ Project(_, | ||
_: SubqueryAlias | ||
|
@@ -338,20 +369,39 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi | |
| _: Sample | ||
) => plan | ||
|
||
case plan: Project => | ||
wrapChildWithSubquery(plan) | ||
case plan: Project => wrapChildWithSubquery(plan) | ||
|
||
// We will generate "SELECT ... FROM ..." for Window operator, so its child operator should | ||
// be able to put in the FROM clause, or we wrap it with a subquery. | ||
case w @ Window(_, _, _, _, | ||
_: SubqueryAlias | ||
| _: Filter | ||
| _: Join | ||
| _: MetastoreRelation | ||
| OneRowRelation | ||
| _: LocalLimit | ||
| _: GlobalLimit | ||
| _: Sample | ||
) => w | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a comment to explain why we need this rule? |
||
|
||
case w: Window => wrapChildWithSubquery(w) | ||
} | ||
|
||
def wrapChildWithSubquery(project: Project): Project = project match { | ||
case Project(projectList, child) => | ||
val alias = SQLBuilder.newSubqueryName | ||
val childAttributes = child.outputSet | ||
val aliasedProjectList = projectList.map(_.transform { | ||
case a: Attribute if childAttributes.contains(a) => | ||
a.withQualifiers(alias :: Nil) | ||
}.asInstanceOf[NamedExpression]) | ||
private def wrapChildWithSubquery(plan: UnaryNode): LogicalPlan = { | ||
val newChild = SubqueryAlias(SQLBuilder.newSubqueryName, plan.child) | ||
plan.withNewChildren(Seq(newChild)) | ||
} | ||
} | ||
|
||
Project(aliasedProjectList, SubqueryAlias(alias, child)) | ||
object UpdateQualifiers extends Rule[LogicalPlan] { | ||
override def apply(tree: LogicalPlan): LogicalPlan = tree transformUp { | ||
case plan => | ||
val inputAttributes = plan.children.flatMap(_.output) | ||
plan transformExpressions { | ||
case a: AttributeReference if !plan.producedAttributes.contains(a) => | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @gatorsmile Not related to this PR. What is difference between There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sounds like There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, but we do not need to add qualifiers for the attributes in |
||
val qualifier = inputAttributes.find(_ semanticEquals a).map(_.qualifiers) | ||
a.withQualifiers(qualifier.getOrElse(Nil)) | ||
} | ||
} | ||
} | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe I missed, where is the method of
sql
for NTile?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is defined here:
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala#L200-L203
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I see. Thanks!