You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/11/30 04:58:15 UTC

[GitHub] [spark] LuciferYang commented on a diff in pull request #38799: [SPARK-37099][SQL] Introduce the group limit of Window for rank-based filter to optimize top-k computation

LuciferYang commented on code in PR #38799:
URL: https://github.com/apache/spark/pull/38799#discussion_r1035453396


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -627,6 +627,22 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
     }
   }
 
+  object WindowGroupLimit extends Strategy {
+    def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+      case logical.WindowGroupLimit(partitionSpec, orderSpec, rankLikeFunction, limit, child) =>
+        // TODO: add a physical rule to remove the partialWindowGroupLimit node, if there is no

Review Comment:
   should we file a new jira to tracking this TODO



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimit.scala:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentRow, DenseRank, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, IntegerLiteral, LessThan, LessThanOrEqual, NamedExpression, PredicateHelper, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan, Window, WindowGroupLimit}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{FILTER, WINDOW}
+
+/**
+ * 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 InsertWindowGroupLimit extends Rule[LogicalPlan] 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): LogicalPlan = {
+    if (conf.windowGroupLimitThreshold == -1) return plan
+
+    plan.transformWithPruning(
+      _.containsAllPatterns(FILTER, WINDOW), ruleId) {
+      case filter @ Filter(condition,
+        window @ Window(windowExpressions, partitionSpec, orderSpec, child))
+          if !child.isInstanceOf[WindowGroupLimit] && !child.isInstanceOf[Window] &&
+            supports(windowExpressions) && orderSpec.nonEmpty =>
+        val limits = windowExpressions.collect {
+          case alias @ Alias(WindowExpression(rankLikeFunction, _), _) =>
+            extractLimits(condition, alias.toAttribute).map((_, rankLikeFunction))
+        }
+
+        // multiple different rank-like functions unsupported.
+        if (limits.filter(_.isDefined).groupBy(_.get._2).size > 1) {
+          return filter
+        }
+        val minLimit = limits.sortBy(_.get._1).head

Review Comment:
   could we change to use `minBy`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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] {
+
+      def stream: Iterator[InternalRow]
+
+      val grouping = UnsafeProjection.create(partitionSpec, child.output)
+
+      // Manage the stream and the grouping.
+      var nextRow: UnsafeRow = null
+      var nextGroup: UnsafeRow = null
+      var nextRowAvailable: Boolean = false
+      protected[this] def fetchNextRow(): Unit = {
+        nextRowAvailable = stream.hasNext
+        if (nextRowAvailable) {
+          nextRow = stream.next().asInstanceOf[UnsafeRow]
+          nextGroup = grouping(nextRow)
+        } else {
+          nextRow = null
+          nextGroup = null
+        }
+      }
+      fetchNextRow()
+
+      // Whether or not the rank exceeding the window group limit value.
+      def exceedingLimit(): Boolean
+
+      // Increase the rank value.
+      def increaseRank(): Unit
+
+      // Clear the rank value.
+      def clearRank(): Unit
+
+      var bufferIterator: Iterator[InternalRow] = _
+
+      private[this] def fetchNextPartition(): Unit = {
+        clearRank()
+        bufferIterator = createGroupIterator()
+      }
+
+      override final def hasNext: Boolean = {
+        val found = (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable

Review Comment:
   need the local `val found`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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] {
+
+      def stream: Iterator[InternalRow]
+
+      val grouping = UnsafeProjection.create(partitionSpec, child.output)
+
+      // Manage the stream and the grouping.
+      var nextRow: UnsafeRow = null
+      var nextGroup: UnsafeRow = null
+      var nextRowAvailable: Boolean = false
+      protected[this] def fetchNextRow(): Unit = {
+        nextRowAvailable = stream.hasNext
+        if (nextRowAvailable) {
+          nextRow = stream.next().asInstanceOf[UnsafeRow]
+          nextGroup = grouping(nextRow)
+        } else {
+          nextRow = null
+          nextGroup = null
+        }
+      }
+      fetchNextRow()
+
+      // Whether or not the rank exceeding the window group limit value.
+      def exceedingLimit(): Boolean
+
+      // Increase the rank value.
+      def increaseRank(): Unit
+
+      // Clear the rank value.
+      def clearRank(): Unit
+
+      var bufferIterator: Iterator[InternalRow] = _
+
+      private[this] def fetchNextPartition(): Unit = {
+        clearRank()
+        bufferIterator = createGroupIterator()
+      }
+
+      override final def hasNext: Boolean = {
+        val found = (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable
+        found
+      }
+
+      override final def next(): InternalRow = {
+        // Load the next partition if we need to.
+        if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) {

Review Comment:
   Is this condition equal to `!hasNext`? not sure
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org