cloud-fan commented on a change in pull request #34504:
URL: https://github.com/apache/spark/pull/34504#discussion_r755213627



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -1548,6 +1548,31 @@ object PushPredicateThroughNonJoin extends 
Rule[LogicalPlan] with PredicateHelpe
         filter
       }
 
+    case filter @ Filter(condition, w: Window) if w.partitionSpec.isEmpty =>
+      w.windowExpressions match {
+        case Seq(alias @ Alias(WindowExpression(_: RowNumber, 
WindowSpecDefinition(Nil, orderSpec,
+            SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), 
_)) =>
+          val aliasAttr = alias.toAttribute
+          val limitValue = splitConjunctivePredicates(condition).collectFirst {
+            case LessThanOrEqual(e, IntegerLiteral(v)) if 
e.semanticEquals(aliasAttr) => v
+            case Equality(e, IntegerLiteral(v)) if e.semanticEquals(aliasAttr) 
=> v
+            case LessThan(e, IntegerLiteral(v)) if e.semanticEquals(aliasAttr) 
=> v - 1
+          }
+
+          limitValue match {
+            case Some(lv) if lv <= 0 =>
+              LocalRelation(filter.output, data = Seq.empty, isStreaming = 
filter.isStreaming)
+            case Some(lv)
+                if lv < conf.topKSortFallbackThreshold && 
w.child.maxRows.forall(_ > lv) =>
+              filter.copy(child =
+                w.copy(child = Limit(Literal(lv), Sort(orderSpec, true, 
w.child))))

Review comment:
       One worry is what if other optimizer rules put/move operators between 
Limit and Sort? Then we can't use the `TakeOrderedAndProjectExec` and introduce 
a big overhead by this global sort.




-- 
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-unsubscr...@spark.apache.org

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



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

Reply via email to