beliefer commented on code in PR #38799:
URL: https://github.com/apache/spark/pull/38799#discussion_r1036903257
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -627,6 +627,87 @@ abstract class SparkStrategies extends
QueryPlanner[SparkPlan] {
}
}
+ /**
+ * Optimize the filter based on rank-like window function by reduce not
required rows.
+ * This rule optimizes the following cases:
+ * {{{
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE rn = 5
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE 5 = rn
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE rn < 5
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE 5 > rn
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE rn <= 5
+ * SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1
WHERE 5 >= rn
+ * }}}
+ */
+ object WindowGroupLimit extends Strategy with PredicateHelper {
+
+ /**
+ * Extract all the limit values from predicates.
+ */
+ def extractLimits(condition: Expression, attr: Attribute): Option[Int] = {
+ val limits = splitConjunctivePredicates(condition).collect {
+ case EqualTo(IntegerLiteral(limit), e) if e.semanticEquals(attr) =>
limit
+ case EqualTo(e, IntegerLiteral(limit)) if e.semanticEquals(attr) =>
limit
+ case LessThan(e, IntegerLiteral(limit)) if e.semanticEquals(attr) =>
limit - 1
+ case GreaterThan(IntegerLiteral(limit), e) if e.semanticEquals(attr)
=> limit - 1
+ case LessThanOrEqual(e, IntegerLiteral(limit)) if
e.semanticEquals(attr) => limit
+ case GreaterThanOrEqual(IntegerLiteral(limit), e) if
e.semanticEquals(attr) => limit
+ }
+
+ if (limits.nonEmpty) Some(limits.min) else None
+ }
+
+ private def supports(
+ windowExpressions: Seq[NamedExpression]): Boolean =
windowExpressions.exists {
+ case Alias(WindowExpression(_: Rank | _: DenseRank | _: RowNumber,
WindowSpecDefinition(_, _,
+ SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), _) =>
true
+ case _ => false
+ }
+
+ def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+ if (conf.windowGroupLimitThreshold == -1) return Nil
+
+ plan match {
+ case filter @ Filter(condition,
+ window @ logical.Window(windowExpressions, partitionSpec, orderSpec,
child))
+ if !child.isInstanceOf[logical.Window] &&
+ supports(windowExpressions) && orderSpec.nonEmpty =>
Review Comment:
No, if the window exists at least one rank like function, we can apply the
optimization.
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.window
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute,
DenseRank, Expression, Rank, RowNumber, SortOrder, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples,
ClusteredDistribution, Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+sealed trait WindowGroupLimitMode
+
+case object Partial extends WindowGroupLimitMode
+
+case object Final extends WindowGroupLimitMode
+
+/**
+ * This operator is designed to filter out unnecessary rows before WindowExec
+ * for top-k computation.
+ * @param partitionSpec Should be the same as [[WindowExec#partitionSpec]]
+ * @param orderSpec Should be the same as [[WindowExec#orderSpec]]
+ * @param rankLikeFunction The function to compute row rank, should be
RowNumber/Rank/DenseRank.
+ */
+case class WindowGroupLimitExec(
+ partitionSpec: Seq[Expression],
+ orderSpec: Seq[SortOrder],
+ rankLikeFunction: Expression,
+ limit: Int,
+ mode: WindowGroupLimitMode,
+ child: SparkPlan) extends UnaryExecNode {
+
+ override def output: Seq[Attribute] = child.output
+
+ override def requiredChildDistribution: Seq[Distribution] = mode match {
+ case Partial => super.requiredChildDistribution
+ case Final =>
+ if (partitionSpec.isEmpty) {
+ AllTuples :: Nil
+ } else {
+ ClusteredDistribution(partitionSpec) :: Nil
+ }
+ }
+
+ override def requiredChildOrdering: Seq[Seq[SortOrder]] =
+ Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec)
+
+ override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+ override def outputPartitioning: Partitioning = child.outputPartitioning
+
+ protected override def doExecute(): RDD[InternalRow] = {
+
+ abstract class WindowIterator extends Iterator[InternalRow] {
Review Comment:
OK
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]