You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "beliefer (via GitHub)" <gi...@apache.org> on 2023/07/11 11:24:13 UTC

[GitHub] [spark] beliefer opened a new pull request, #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

beliefer opened a new pull request, #41939:
URL: https://github.com/apache/spark/pull/41939

   ### What changes were proposed in this pull request?
   `WindowExec` and `WindowInPandasExec` are updated to use the PartitionEvaluator API to do execution.
   
   
   ### Why are the changes needed?
   To define the computing logic and requires the caller side to explicitly list what needs to be serialized and sent to executors
   
   
   ### Does this PR introduce _any_ user-facing change?
   'No'.
   Just update the inner implementation.
   
   
   ### How was this patch tested?
   Add new test cases.
   


-- 
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


[GitHub] [spark] viirya commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1263289294


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala:
##########
@@ -0,0 +1,418 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, JoinedRow, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecificInternalRow, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.util.collection.Utils
+
+trait WindowEvaluatorFactoryBase {
+  def windowExpression: Seq[NamedExpression]
+  def partitionSpec: Seq[Expression]
+  def orderSpec: Seq[SortOrder]
+  def conf: SQLConf

Review Comment:
   Seems `conf` is always `SQLConf.get` in both implementations?



-- 
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


[GitHub] [spark] cloud-fan commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1264891227


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.python
+
+import java.io.File
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{JobArtifactSet, PartitionEvaluator, PartitionEvaluatorFactory, SparkEnv, TaskContext}
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, EmptyRow, Expression, JoinedRow, NamedExpression, PythonFuncExpression, PythonUDAF, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.window.{SlidingWindowFunctionFrame, UnboundedFollowingWindowFunctionFrame, UnboundedPrecedingWindowFunctionFrame, UnboundedWindowFunctionFrame, WindowEvaluatorFactoryBase, WindowFunctionFrame}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType}
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+class WindowInPandasEvaluatorFactory(
+    val windowExpression: Seq[NamedExpression],
+    val partitionSpec: Seq[Expression],
+    val orderSpec: Seq[SortOrder],
+    val childOutput: Seq[Attribute],
+    val spillSize: SQLMetric,
+    pythonMetrics: Map[String, SQLMetric])
+  extends PartitionEvaluatorFactory[InternalRow, InternalRow] with WindowEvaluatorFactoryBase {
+  val conf: SQLConf = SQLConf.get
+
+  /**
+   * Helper functions and data structures for window bounds
+   *
+   * It contains:
+   * (1) Total number of window bound indices in the python input row
+   * (2) Function from frame index to its lower bound column index in the python input row
+   * (3) Function from frame index to its upper bound column index in the python input row
+   * (4) Seq from frame index to its window bound type
+   */
+  private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType])
+
+  /**
+   * Enum for window bound types. Used only inside this class.
+   */
+  private sealed case class WindowBoundType(value: String)
+
+  private object UnboundedWindow extends WindowBoundType("unbounded")
+
+  private object BoundedWindow extends WindowBoundType("bounded")
+
+  private val windowBoundTypeConf = "pandas_window_bound_types"
+
+  private def collectFunctions(
+      udf: PythonFuncExpression): (ChainedPythonFunctions, Seq[Expression]) = {
+    udf.children match {
+      case Seq(u: PythonFuncExpression) =>
+        val (chained, children) = collectFunctions(u)
+        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
+      case children =>
+        // There should not be any other UDFs, or the children can't be evaluated directly.
+        assert(children.forall(!_.exists(_.isInstanceOf[PythonFuncExpression])))
+        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
+    }
+  }
+
+  // Helper functions
+  /**
+   * See [[WindowBoundHelpers]] for details.
+   */
+  private def computeWindowBoundHelpers(
+      factories: Seq[InternalRow => WindowFunctionFrame]): WindowBoundHelpers = {
+    val functionFrames = factories.map(_ (EmptyRow))
+
+    val windowBoundTypes = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => UnboundedWindow
+      case _: UnboundedFollowingWindowFunctionFrame |
+           _: SlidingWindowFunctionFrame |
+           _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow
+      // It should be impossible to get other types of window function frame here
+      case frame => throw QueryExecutionErrors.unexpectedWindowFunctionFrameError(frame.toString)
+    }
+
+    val requiredIndices = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => 0
+      case _ => 2
+    }
+
+    val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail
+
+    val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) =>
+      if (num == 0) {
+        // Sentinel values for unbounded window
+        (-1, -1)
+      } else {
+        (upperBoundIndex - 2, upperBoundIndex - 1)
+      }
+    }
+
+    def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1
+
+    def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2
+
+    (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes)
+  }
+
+  // Unwrap the expressions and factories from the map.
+  private val expressionsWithFrameIndex =

Review Comment:
   cc @Hisoka-X can you also check your PRs (either merged or not)? We should avoid putting variables in the factor class, but define them in the evaluator class instead.



-- 
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


[GitHub] [spark] beliefer commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1631899474

   ping @cloud-fan @viirya  cc @vinodkc
   The CI failure looks unrelated.


-- 
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


[GitHub] [spark] beliefer commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1635277965

   > As tests are skipped now, maybe you can enable it to test change in CI, and revert back current config before merging.
   
   In fact, CI passed in previous commit.


-- 
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


[GitHub] [spark] viirya commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1635277333

   As tests are skipped now, maybe you can enable it to test change in CI, and revert back current config before merging.


-- 
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


[GitHub] [spark] beliefer commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1639308801

   @cloud-fan @viirya @Hisoka-X Thank you.


-- 
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


[GitHub] [spark] vinodkc commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "vinodkc (via GitHub)" <gi...@apache.org>.
vinodkc commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1276605034


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala:
##########
@@ -95,111 +95,23 @@ case class WindowExec(
   )
 
   protected override def doExecute(): RDD[InternalRow] = {
-    // Unwrap the window expressions and window frame factories from the map.
-    val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1)
-    val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
-    val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
-    val spillThreshold = conf.windowExecBufferSpillThreshold
     val spillSize = longMetric("spillSize")
 
-    // Start processing.
-    child.execute().mapPartitions { stream =>
-      new Iterator[InternalRow] {
-
-        // Get all relevant projections.
-        val result = createResultProjection(expressions)
-        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
-        private[this] def fetchNextRow(): Unit = {
-          nextRowAvailable = stream.hasNext
-          if (nextRowAvailable) {
-            nextRow = stream.next().asInstanceOf[UnsafeRow]
-            nextGroup = grouping(nextRow)
-          } else {
-            nextRow = null
-            nextGroup = null
-          }
-        }
-        fetchNextRow()
-
-        // Manage the current partition.
-        val buffer: ExternalAppendOnlyUnsafeRowArray =
-          new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
-
-        var bufferIterator: Iterator[UnsafeRow] = _
-
-        val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType))
-        val frames = factories.map(_(windowFunctionResult))
-        val numFrames = frames.length
-        private[this] def fetchNextPartition(): Unit = {
-          // Collect all the rows in the current partition.
-          // Before we start to fetch new input rows, make a copy of nextGroup.
-          val currentGroup = nextGroup.copy()
-
-          // clear last partition
-          buffer.clear()
-
-          while (nextRowAvailable && nextGroup == currentGroup) {
-            buffer.add(nextRow)
-            fetchNextRow()
-          }
-
-          // Setup the frames.
-          var i = 0
-          while (i < numFrames) {
-            frames(i).prepare(buffer)
-            i += 1
-          }
+    val evaluatorFactory =
+      new WindowEvaluatorFactory(
+        windowExpression,
+        partitionSpec,
+        orderSpec,
+        child.output,
+        spillSize)
 
-          // Setup iteration
-          rowIndex = 0
-          bufferIterator = buffer.generateIterator()
-        }
-
-        // Iteration
-        var rowIndex = 0
-
-        override final def hasNext: Boolean = {
-          val found = (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable
-          if (!found) {
-            // clear final partition
-            buffer.clear()
-            spillSize += buffer.spillSize
-          }
-          found
-        }
-
-        val join = new JoinedRow
-        override final def next(): InternalRow = {
-          // Load the next partition if we need to.
-          if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) {
-            fetchNextPartition()
-          }
-
-          if (bufferIterator.hasNext) {
-            val current = bufferIterator.next()
-
-            // Get the results for the window frames.
-            var i = 0
-            while (i < numFrames) {
-              frames(i).write(rowIndex, current)
-              i += 1
-            }
-
-            // 'Merge' the input row with the window function result
-            join(current, windowFunctionResult)
-            rowIndex += 1
-
-            // Return the projection.
-            result(join)
-          } else {
-            throw new NoSuchElementException
-          }
-        }
+    // Start processing.
+    if (conf.usePartitionEvaluator) {
+      child.execute().mapPartitionsWithEvaluator(evaluatorFactory)
+    } else {
+      child.execute().mapPartitions { iter =>
+        val evaluator = evaluatorFactory.createEvaluator()
+        evaluator.eval(0, iter)

Review Comment:
   @beliefer, Can you please raise a follow-up PR to handle the partition index as this https://github.com/apache/spark/pull/42185



-- 
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


[GitHub] [spark] cloud-fan commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1270194752


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, JoinedRow, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecificInternalRow, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.util.collection.Utils
+
+trait WindowEvaluatorFactoryBase {

Review Comment:
   with hindsight, shall we move it to a new file? Its two sub-classes are not both in this file, so it's a bit weird to put it with one of the sub-class.



-- 
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


[GitHub] [spark] viirya commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1635282816

   > > As tests are skipped now, maybe you can enable it to test change in CI, and revert back current config before merging.
   > 
   > In fact, CI passed in previous commit.
   
   Oh okay. 👍 


-- 
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


[GitHub] [spark] beliefer commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1633544370

   cc @HyukjinKwon @zhengruifeng 


-- 
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


[GitHub] [spark] cloud-fan commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1263219232


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, JoinedRow, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecificInternalRow, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.util.collection.Utils
+
+trait WindowEvaluatorFactoryBase {
+  def windowExpression: Seq[NamedExpression]
+  def partitionSpec: Seq[Expression]
+  def orderSpec: Seq[SortOrder]
+  def conf: SQLConf
+  def childOutput: Seq[Attribute]
+  def spillSize: SQLMetric
+
+  /**
+   * Create the resulting projection.
+   *
+   * This method uses Code Generation. It can only be used on the executor side.
+   *
+   * @param expressions unbound ordered function expressions.
+   * @return the final resulting projection.
+   */
+  protected def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = {
+    val references = expressions.zipWithIndex.map { case (e, i) =>
+      // Results of window expressions will be on the right side of child's output
+      BoundReference(childOutput.size + i, e.dataType, e.nullable)
+    }
+    val unboundToRefMap = Utils.toMap(expressions, references)
+    val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap))
+    UnsafeProjection.create(
+      childOutput ++ patchedWindowExpression,
+      childOutput)
+  }
+
+  /**
+   * Create a bound ordering object for a given frame type and offset. A bound ordering object is
+   * used to determine which input row lies within the frame boundaries of an output row.
+   *
+   * This method uses Code Generation. It can only be used on the executor side.
+   *
+   * @param frame to evaluate. This can either be a Row or Range frame.
+   * @param bound with respect to the row.
+   * @param timeZone the session local timezone for time related calculations.
+   * @return a bound ordering object.
+   */
+  private def createBoundOrdering(
+      frame: FrameType, bound: Expression, timeZone: String): BoundOrdering = {
+    (frame, bound) match {
+      case (RowFrame, CurrentRow) =>
+        RowBoundOrdering(0)
+
+      case (RowFrame, IntegerLiteral(offset)) =>
+        RowBoundOrdering(offset)
+
+      case (RowFrame, _) =>
+        throw new IllegalStateException(s"Unhandled bound in windows expressions: $bound")
+
+      case (RangeFrame, CurrentRow) =>
+        val ordering = RowOrdering.create(orderSpec, childOutput)
+        RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection)
+
+      case (RangeFrame, offset: Expression) if orderSpec.size == 1 =>
+        // Use only the first order expression when the offset is non-null.
+        val sortExpr = orderSpec.head
+        val expr = sortExpr.child
+
+        // Create the projection which returns the current 'value'.
+        val current = MutableProjection.create(expr :: Nil, childOutput)
+
+        // Flip the sign of the offset when processing the order is descending
+        val boundOffset = sortExpr.direction match {
+          case Descending => UnaryMinus(offset)
+          case Ascending => offset
+        }
+
+        // Create the projection which returns the current 'value' modified by adding the offset.
+        val boundExpr = (expr.dataType, boundOffset.dataType) match {
+          case (DateType, IntegerType) => DateAdd(expr, boundOffset)
+          case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(expr, boundOffset)
+          case (TimestampType | TimestampNTZType, CalendarIntervalType) =>
+            TimeAdd(expr, boundOffset, Some(timeZone))
+          case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
+            TimestampAddYMInterval(expr, boundOffset, Some(timeZone))
+          case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) =>
+            TimeAdd(expr, boundOffset, Some(timeZone))
+          case (d: DecimalType, _: DecimalType) => DecimalAddNoOverflowCheck(expr, boundOffset, d)
+          case (a, b) if a == b => Add(expr, boundOffset)
+        }
+        val bound = MutableProjection.create(boundExpr :: Nil, childOutput)
+
+        // Construct the ordering. This is used to compare the result of current value projection
+        // to the result of bound value projection. This is done manually because we want to use
+        // Code Generation (if it is enabled).
+        val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil
+        val ordering = RowOrdering.create(boundSortExprs, Nil)
+        RangeBoundOrdering(ordering, current, bound)
+
+      case (RangeFrame, _) =>
+        throw new IllegalStateException("Non-Zero range offsets are not supported for windows " +
+          "with multiple order expressions.")
+    }
+  }
+
+  /**
+   * Collection containing an entry for each window frame to process. Each entry contains a frame's
+   * [[WindowExpression]]s and factory function for the [[WindowFunctionFrame]].
+   */
+  protected lazy val windowFrameExpressionFactoryPairs = {
+    type FrameKey = (String, FrameType, Expression, Expression, Seq[Expression])
+    type ExpressionBuffer = mutable.Buffer[Expression]
+    val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)]
+
+    // Add a function and its function to the map for a given frame.
+    def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = {
+      val key = fn match {
+        // This branch is used for Lead/Lag to support ignoring null and optimize the performance
+        // for NthValue ignoring null.
+        // All window frames move in rows. If there are multiple Leads, Lags or NthValues acting on
+        // a row and operating on different input expressions, they should not be moved uniformly
+        // by row. Therefore, we put these functions in different window frames.
+        case f: OffsetWindowFunction if f.ignoreNulls =>
+          (tpe, fr.frameType, fr.lower, fr.upper, f.children.map(_.canonicalized))
+        case _ => (tpe, fr.frameType, fr.lower, fr.upper, Nil)
+      }
+      val (es, fns) = framedFunctions.getOrElseUpdate(
+        key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression]))
+      es += e
+      fns += fn
+    }
+
+    // Collect all valid window functions and group them by their frame.
+    windowExpression.foreach { x =>
+      x.foreach {
+        case e@WindowExpression(function, spec) =>
+          val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame]
+          function match {
+            case AggregateExpression(f, _, _, _, _) => collect("AGGREGATE", frame, e, f)
+            case f: FrameLessOffsetWindowFunction =>
+              collect("FRAME_LESS_OFFSET", f.fakeFrame, e, f)
+            case f: OffsetWindowFunction if frame.frameType == RowFrame &&
+              frame.lower == UnboundedPreceding =>
+              frame.upper match {
+                case UnboundedFollowing => collect("UNBOUNDED_OFFSET", f.fakeFrame, e, f)
+                case CurrentRow => collect("UNBOUNDED_PRECEDING_OFFSET", f.fakeFrame, e, f)
+                case _ => collect("AGGREGATE", frame, e, f)
+              }
+            case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f)
+            case f => throw new IllegalStateException(s"Unsupported window function: $f")
+          }
+        case _ =>
+      }
+    }
+
+    // Map the groups to a (unbound) expression and frame factory pair.
+    var numExpressions = 0
+    val timeZone = conf.sessionLocalTimeZone
+    framedFunctions.toSeq.map {
+      case (key, (expressions, functionSeq)) =>
+        val ordinal = numExpressions
+        val functions = functionSeq.toArray
+
+        // Construct an aggregate processor if we need one.
+        // Currently we don't allow mixing of Pandas UDF and SQL aggregation functions
+        // in a single Window physical node. Therefore, we can assume no SQL aggregation
+        // functions if Pandas UDF exists. In the future, we might mix Pandas UDF and SQL
+        // aggregation function in a single physical node.
+        def processor = if (functions.exists(_.isInstanceOf[PythonFuncExpression])) {
+          null
+        } else {
+          AggregateProcessor(
+            functions,
+            ordinal,
+            childOutput,
+            (expressions, schema) =>
+              MutableProjection.create(expressions, schema))
+        }
+
+        // Create the factory to produce WindowFunctionFrame.
+        val factory = key match {
+          // Frameless offset Frame
+          case ("FRAME_LESS_OFFSET", _, IntegerLiteral(offset), _, expr) =>
+            target: InternalRow =>
+              new FrameLessOffsetWindowFunctionFrame(
+                target,
+                ordinal,
+                // OFFSET frame functions are guaranteed be OffsetWindowFunction.
+                functions.map(_.asInstanceOf[OffsetWindowFunction]),
+                childOutput,
+                (expressions, schema) =>
+                  MutableProjection.create(expressions, schema),
+                offset,
+                expr.nonEmpty)
+          case ("UNBOUNDED_OFFSET", _, IntegerLiteral(offset), _, expr) =>
+            target: InternalRow => {
+              new UnboundedOffsetWindowFunctionFrame(
+                target,
+                ordinal,
+                // OFFSET frame functions are guaranteed be OffsetWindowFunction.
+                functions.map(_.asInstanceOf[OffsetWindowFunction]),
+                childOutput,
+                (expressions, schema) =>
+                  MutableProjection.create(expressions, schema),
+                offset,
+                expr.nonEmpty)
+            }
+          case ("UNBOUNDED_PRECEDING_OFFSET", _, IntegerLiteral(offset), _, expr) =>
+            target: InternalRow => {
+              new UnboundedPrecedingOffsetWindowFunctionFrame(
+                target,
+                ordinal,
+                // OFFSET frame functions are guaranteed be OffsetWindowFunction.
+                functions.map(_.asInstanceOf[OffsetWindowFunction]),
+                childOutput,
+                (expressions, schema) =>
+                  MutableProjection.create(expressions, schema),
+                offset,
+                expr.nonEmpty)
+            }
+
+          // Entire Partition Frame.
+          case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing, _) =>
+            target: InternalRow => {
+              new UnboundedWindowFunctionFrame(target, processor)
+            }
+
+          // Growing Frame.
+          case ("AGGREGATE", frameType, UnboundedPreceding, upper, _) =>
+            target: InternalRow => {
+              new UnboundedPrecedingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, upper, timeZone))
+            }
+
+          // Shrinking Frame.
+          case ("AGGREGATE", frameType, lower, UnboundedFollowing, _) =>
+            target: InternalRow => {
+              new UnboundedFollowingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, lower, timeZone))
+            }
+
+          // Moving Frame.
+          case ("AGGREGATE", frameType, lower, upper, _) =>
+            target: InternalRow => {
+              new SlidingWindowFunctionFrame(
+                target,
+                processor,
+                createBoundOrdering(frameType, lower, timeZone),
+                createBoundOrdering(frameType, upper, timeZone))
+            }
+
+          case _ =>
+            throw new IllegalStateException(s"Unsupported factory: $key")
+        }
+
+        // Keep track of the number of expressions. This is a side-effect in a map...
+        numExpressions += expressions.size
+
+        // Create the Window Expression - Frame Factory pair.
+        (expressions, factory)
+    }
+  }
+
+}
+
+class WindowEvaluatorFactory(
+    val windowExpression: Seq[NamedExpression],
+    val partitionSpec: Seq[Expression],
+    val orderSpec: Seq[SortOrder],
+    val conf: SQLConf,

Review Comment:
   we usually don't need to pass the SQLConf, as worker side can get it via `SQLConf.get`



-- 
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


[GitHub] [spark] beliefer commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1263293187


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala:
##########
@@ -0,0 +1,418 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, JoinedRow, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecificInternalRow, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.util.collection.Utils
+
+trait WindowEvaluatorFactoryBase {
+  def windowExpression: Seq[NamedExpression]
+  def partitionSpec: Seq[Expression]
+  def orderSpec: Seq[SortOrder]
+  def conf: SQLConf

Review Comment:
   Yes. I have used `SQLConf.get` in both implementations.



-- 
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


[GitHub] [spark] vinodkc commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "vinodkc (via GitHub)" <gi...@apache.org>.
vinodkc commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1276605034


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala:
##########
@@ -95,111 +95,23 @@ case class WindowExec(
   )
 
   protected override def doExecute(): RDD[InternalRow] = {
-    // Unwrap the window expressions and window frame factories from the map.
-    val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1)
-    val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
-    val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
-    val spillThreshold = conf.windowExecBufferSpillThreshold
     val spillSize = longMetric("spillSize")
 
-    // Start processing.
-    child.execute().mapPartitions { stream =>
-      new Iterator[InternalRow] {
-
-        // Get all relevant projections.
-        val result = createResultProjection(expressions)
-        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
-        private[this] def fetchNextRow(): Unit = {
-          nextRowAvailable = stream.hasNext
-          if (nextRowAvailable) {
-            nextRow = stream.next().asInstanceOf[UnsafeRow]
-            nextGroup = grouping(nextRow)
-          } else {
-            nextRow = null
-            nextGroup = null
-          }
-        }
-        fetchNextRow()
-
-        // Manage the current partition.
-        val buffer: ExternalAppendOnlyUnsafeRowArray =
-          new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
-
-        var bufferIterator: Iterator[UnsafeRow] = _
-
-        val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType))
-        val frames = factories.map(_(windowFunctionResult))
-        val numFrames = frames.length
-        private[this] def fetchNextPartition(): Unit = {
-          // Collect all the rows in the current partition.
-          // Before we start to fetch new input rows, make a copy of nextGroup.
-          val currentGroup = nextGroup.copy()
-
-          // clear last partition
-          buffer.clear()
-
-          while (nextRowAvailable && nextGroup == currentGroup) {
-            buffer.add(nextRow)
-            fetchNextRow()
-          }
-
-          // Setup the frames.
-          var i = 0
-          while (i < numFrames) {
-            frames(i).prepare(buffer)
-            i += 1
-          }
+    val evaluatorFactory =
+      new WindowEvaluatorFactory(
+        windowExpression,
+        partitionSpec,
+        orderSpec,
+        child.output,
+        spillSize)
 
-          // Setup iteration
-          rowIndex = 0
-          bufferIterator = buffer.generateIterator()
-        }
-
-        // Iteration
-        var rowIndex = 0
-
-        override final def hasNext: Boolean = {
-          val found = (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable
-          if (!found) {
-            // clear final partition
-            buffer.clear()
-            spillSize += buffer.spillSize
-          }
-          found
-        }
-
-        val join = new JoinedRow
-        override final def next(): InternalRow = {
-          // Load the next partition if we need to.
-          if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) {
-            fetchNextPartition()
-          }
-
-          if (bufferIterator.hasNext) {
-            val current = bufferIterator.next()
-
-            // Get the results for the window frames.
-            var i = 0
-            while (i < numFrames) {
-              frames(i).write(rowIndex, current)
-              i += 1
-            }
-
-            // 'Merge' the input row with the window function result
-            join(current, windowFunctionResult)
-            rowIndex += 1
-
-            // Return the projection.
-            result(join)
-          } else {
-            throw new NoSuchElementException
-          }
-        }
+    // Start processing.
+    if (conf.usePartitionEvaluator) {
+      child.execute().mapPartitionsWithEvaluator(evaluatorFactory)
+    } else {
+      child.execute().mapPartitions { iter =>
+        val evaluator = evaluatorFactory.createEvaluator()
+        evaluator.eval(0, iter)

Review Comment:
   @ beliefer, Can you please raise a follow-up PR to handle the partition index as this https://github.com/apache/spark/pull/42185



-- 
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


[GitHub] [spark] beliefer commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1277422584


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala:
##########
@@ -95,111 +95,23 @@ case class WindowExec(
   )
 
   protected override def doExecute(): RDD[InternalRow] = {
-    // Unwrap the window expressions and window frame factories from the map.
-    val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1)
-    val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
-    val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
-    val spillThreshold = conf.windowExecBufferSpillThreshold
     val spillSize = longMetric("spillSize")
 
-    // Start processing.
-    child.execute().mapPartitions { stream =>
-      new Iterator[InternalRow] {
-
-        // Get all relevant projections.
-        val result = createResultProjection(expressions)
-        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
-        private[this] def fetchNextRow(): Unit = {
-          nextRowAvailable = stream.hasNext
-          if (nextRowAvailable) {
-            nextRow = stream.next().asInstanceOf[UnsafeRow]
-            nextGroup = grouping(nextRow)
-          } else {
-            nextRow = null
-            nextGroup = null
-          }
-        }
-        fetchNextRow()
-
-        // Manage the current partition.
-        val buffer: ExternalAppendOnlyUnsafeRowArray =
-          new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
-
-        var bufferIterator: Iterator[UnsafeRow] = _
-
-        val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType))
-        val frames = factories.map(_(windowFunctionResult))
-        val numFrames = frames.length
-        private[this] def fetchNextPartition(): Unit = {
-          // Collect all the rows in the current partition.
-          // Before we start to fetch new input rows, make a copy of nextGroup.
-          val currentGroup = nextGroup.copy()
-
-          // clear last partition
-          buffer.clear()
-
-          while (nextRowAvailable && nextGroup == currentGroup) {
-            buffer.add(nextRow)
-            fetchNextRow()
-          }
-
-          // Setup the frames.
-          var i = 0
-          while (i < numFrames) {
-            frames(i).prepare(buffer)
-            i += 1
-          }
+    val evaluatorFactory =
+      new WindowEvaluatorFactory(
+        windowExpression,
+        partitionSpec,
+        orderSpec,
+        child.output,
+        spillSize)
 
-          // Setup iteration
-          rowIndex = 0
-          bufferIterator = buffer.generateIterator()
-        }
-
-        // Iteration
-        var rowIndex = 0
-
-        override final def hasNext: Boolean = {
-          val found = (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable
-          if (!found) {
-            // clear final partition
-            buffer.clear()
-            spillSize += buffer.spillSize
-          }
-          found
-        }
-
-        val join = new JoinedRow
-        override final def next(): InternalRow = {
-          // Load the next partition if we need to.
-          if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) {
-            fetchNextPartition()
-          }
-
-          if (bufferIterator.hasNext) {
-            val current = bufferIterator.next()
-
-            // Get the results for the window frames.
-            var i = 0
-            while (i < numFrames) {
-              frames(i).write(rowIndex, current)
-              i += 1
-            }
-
-            // 'Merge' the input row with the window function result
-            join(current, windowFunctionResult)
-            rowIndex += 1
-
-            // Return the projection.
-            result(join)
-          } else {
-            throw new NoSuchElementException
-          }
-        }
+    // Start processing.
+    if (conf.usePartitionEvaluator) {
+      child.execute().mapPartitionsWithEvaluator(evaluatorFactory)
+    } else {
+      child.execute().mapPartitions { iter =>
+        val evaluator = evaluatorFactory.createEvaluator()
+        evaluator.eval(0, iter)

Review Comment:
   Thank you for the reminder. I will take a look!



-- 
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


[GitHub] [spark] cloud-fan commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1639298460

   thanks, merging to master!


-- 
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


[GitHub] [spark] cloud-fan commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1264890410


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.python
+
+import java.io.File
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{JobArtifactSet, PartitionEvaluator, PartitionEvaluatorFactory, SparkEnv, TaskContext}
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, EmptyRow, Expression, JoinedRow, NamedExpression, PythonFuncExpression, PythonUDAF, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.window.{SlidingWindowFunctionFrame, UnboundedFollowingWindowFunctionFrame, UnboundedPrecedingWindowFunctionFrame, UnboundedWindowFunctionFrame, WindowEvaluatorFactoryBase, WindowFunctionFrame}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType}
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+class WindowInPandasEvaluatorFactory(
+    val windowExpression: Seq[NamedExpression],
+    val partitionSpec: Seq[Expression],
+    val orderSpec: Seq[SortOrder],
+    val childOutput: Seq[Attribute],
+    val spillSize: SQLMetric,
+    pythonMetrics: Map[String, SQLMetric])
+  extends PartitionEvaluatorFactory[InternalRow, InternalRow] with WindowEvaluatorFactoryBase {
+  val conf: SQLConf = SQLConf.get
+
+  /**
+   * Helper functions and data structures for window bounds
+   *
+   * It contains:
+   * (1) Total number of window bound indices in the python input row
+   * (2) Function from frame index to its lower bound column index in the python input row
+   * (3) Function from frame index to its upper bound column index in the python input row
+   * (4) Seq from frame index to its window bound type
+   */
+  private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType])
+
+  /**
+   * Enum for window bound types. Used only inside this class.
+   */
+  private sealed case class WindowBoundType(value: String)
+
+  private object UnboundedWindow extends WindowBoundType("unbounded")
+
+  private object BoundedWindow extends WindowBoundType("bounded")
+
+  private val windowBoundTypeConf = "pandas_window_bound_types"
+
+  private def collectFunctions(
+      udf: PythonFuncExpression): (ChainedPythonFunctions, Seq[Expression]) = {
+    udf.children match {
+      case Seq(u: PythonFuncExpression) =>
+        val (chained, children) = collectFunctions(u)
+        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
+      case children =>
+        // There should not be any other UDFs, or the children can't be evaluated directly.
+        assert(children.forall(!_.exists(_.isInstanceOf[PythonFuncExpression])))
+        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
+    }
+  }
+
+  // Helper functions
+  /**
+   * See [[WindowBoundHelpers]] for details.
+   */
+  private def computeWindowBoundHelpers(
+      factories: Seq[InternalRow => WindowFunctionFrame]): WindowBoundHelpers = {
+    val functionFrames = factories.map(_ (EmptyRow))
+
+    val windowBoundTypes = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => UnboundedWindow
+      case _: UnboundedFollowingWindowFunctionFrame |
+           _: SlidingWindowFunctionFrame |
+           _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow
+      // It should be impossible to get other types of window function frame here
+      case frame => throw QueryExecutionErrors.unexpectedWindowFunctionFrameError(frame.toString)
+    }
+
+    val requiredIndices = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => 0
+      case _ => 2
+    }
+
+    val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail
+
+    val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) =>
+      if (num == 0) {
+        // Sentinel values for unbounded window
+        (-1, -1)
+      } else {
+        (upperBoundIndex - 2, upperBoundIndex - 1)
+      }
+    }
+
+    def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1
+
+    def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2
+
+    (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes)
+  }
+
+  // Unwrap the expressions and factories from the map.
+  private val expressionsWithFrameIndex =

Review Comment:
   so all these `private val`s will be serialized to the executor side, right? I think it's better to define them in the evaluator class, not the factor, to reduce the data to be serialized.



-- 
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


[GitHub] [spark] beliefer commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1270215136


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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 scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, JoinedRow, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecificInternalRow, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.util.collection.Utils
+
+trait WindowEvaluatorFactoryBase {

Review Comment:
   OK. Let's move it.



-- 
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


[GitHub] [spark] beliefer commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1635520681

   The CI failure is unrelated.


-- 
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


[GitHub] [spark] cloud-fan commented on pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #41939:
URL: https://github.com/apache/spark/pull/41939#issuecomment-1635170579

   We can probably skip testing it. Overall it's just a refactor and it's probably too much to run many tests twice. We can enable it by default later so that all tests cover this code path.


-- 
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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #41939:
URL: https://github.com/apache/spark/pull/41939#discussion_r1264906463


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.python
+
+import java.io.File
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{JobArtifactSet, PartitionEvaluator, PartitionEvaluatorFactory, SparkEnv, TaskContext}
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, EmptyRow, Expression, JoinedRow, NamedExpression, PythonFuncExpression, PythonUDAF, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.window.{SlidingWindowFunctionFrame, UnboundedFollowingWindowFunctionFrame, UnboundedPrecedingWindowFunctionFrame, UnboundedWindowFunctionFrame, WindowEvaluatorFactoryBase, WindowFunctionFrame}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType}
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+class WindowInPandasEvaluatorFactory(
+    val windowExpression: Seq[NamedExpression],
+    val partitionSpec: Seq[Expression],
+    val orderSpec: Seq[SortOrder],
+    val childOutput: Seq[Attribute],
+    val spillSize: SQLMetric,
+    pythonMetrics: Map[String, SQLMetric])
+  extends PartitionEvaluatorFactory[InternalRow, InternalRow] with WindowEvaluatorFactoryBase {
+  val conf: SQLConf = SQLConf.get
+
+  /**
+   * Helper functions and data structures for window bounds
+   *
+   * It contains:
+   * (1) Total number of window bound indices in the python input row
+   * (2) Function from frame index to its lower bound column index in the python input row
+   * (3) Function from frame index to its upper bound column index in the python input row
+   * (4) Seq from frame index to its window bound type
+   */
+  private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType])
+
+  /**
+   * Enum for window bound types. Used only inside this class.
+   */
+  private sealed case class WindowBoundType(value: String)
+
+  private object UnboundedWindow extends WindowBoundType("unbounded")
+
+  private object BoundedWindow extends WindowBoundType("bounded")
+
+  private val windowBoundTypeConf = "pandas_window_bound_types"
+
+  private def collectFunctions(
+      udf: PythonFuncExpression): (ChainedPythonFunctions, Seq[Expression]) = {
+    udf.children match {
+      case Seq(u: PythonFuncExpression) =>
+        val (chained, children) = collectFunctions(u)
+        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
+      case children =>
+        // There should not be any other UDFs, or the children can't be evaluated directly.
+        assert(children.forall(!_.exists(_.isInstanceOf[PythonFuncExpression])))
+        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
+    }
+  }
+
+  // Helper functions
+  /**
+   * See [[WindowBoundHelpers]] for details.
+   */
+  private def computeWindowBoundHelpers(
+      factories: Seq[InternalRow => WindowFunctionFrame]): WindowBoundHelpers = {
+    val functionFrames = factories.map(_ (EmptyRow))
+
+    val windowBoundTypes = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => UnboundedWindow
+      case _: UnboundedFollowingWindowFunctionFrame |
+           _: SlidingWindowFunctionFrame |
+           _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow
+      // It should be impossible to get other types of window function frame here
+      case frame => throw QueryExecutionErrors.unexpectedWindowFunctionFrameError(frame.toString)
+    }
+
+    val requiredIndices = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => 0
+      case _ => 2
+    }
+
+    val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail
+
+    val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) =>
+      if (num == 0) {
+        // Sentinel values for unbounded window
+        (-1, -1)
+      } else {
+        (upperBoundIndex - 2, upperBoundIndex - 1)
+      }
+    }
+
+    def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1
+
+    def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2
+
+    (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes)
+  }
+
+  // Unwrap the expressions and factories from the map.
+  private val expressionsWithFrameIndex =

Review Comment:
   I haven't come across a factory that needs to define member variables (except for parameters)😂



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.python
+
+import java.io.File
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{JobArtifactSet, PartitionEvaluator, PartitionEvaluatorFactory, SparkEnv, TaskContext}
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, EmptyRow, Expression, JoinedRow, NamedExpression, PythonFuncExpression, PythonUDAF, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.window.{SlidingWindowFunctionFrame, UnboundedFollowingWindowFunctionFrame, UnboundedPrecedingWindowFunctionFrame, UnboundedWindowFunctionFrame, WindowEvaluatorFactoryBase, WindowFunctionFrame}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType}
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+class WindowInPandasEvaluatorFactory(
+    val windowExpression: Seq[NamedExpression],
+    val partitionSpec: Seq[Expression],
+    val orderSpec: Seq[SortOrder],
+    val childOutput: Seq[Attribute],
+    val spillSize: SQLMetric,
+    pythonMetrics: Map[String, SQLMetric])
+  extends PartitionEvaluatorFactory[InternalRow, InternalRow] with WindowEvaluatorFactoryBase {
+  val conf: SQLConf = SQLConf.get
+
+  /**
+   * Helper functions and data structures for window bounds
+   *
+   * It contains:
+   * (1) Total number of window bound indices in the python input row
+   * (2) Function from frame index to its lower bound column index in the python input row
+   * (3) Function from frame index to its upper bound column index in the python input row
+   * (4) Seq from frame index to its window bound type
+   */
+  private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType])
+
+  /**
+   * Enum for window bound types. Used only inside this class.
+   */
+  private sealed case class WindowBoundType(value: String)
+
+  private object UnboundedWindow extends WindowBoundType("unbounded")
+
+  private object BoundedWindow extends WindowBoundType("bounded")
+
+  private val windowBoundTypeConf = "pandas_window_bound_types"
+
+  private def collectFunctions(
+      udf: PythonFuncExpression): (ChainedPythonFunctions, Seq[Expression]) = {
+    udf.children match {
+      case Seq(u: PythonFuncExpression) =>
+        val (chained, children) = collectFunctions(u)
+        (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children)
+      case children =>
+        // There should not be any other UDFs, or the children can't be evaluated directly.
+        assert(children.forall(!_.exists(_.isInstanceOf[PythonFuncExpression])))
+        (ChainedPythonFunctions(Seq(udf.func)), udf.children)
+    }
+  }
+
+  // Helper functions
+  /**
+   * See [[WindowBoundHelpers]] for details.
+   */
+  private def computeWindowBoundHelpers(
+      factories: Seq[InternalRow => WindowFunctionFrame]): WindowBoundHelpers = {
+    val functionFrames = factories.map(_ (EmptyRow))
+
+    val windowBoundTypes = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => UnboundedWindow
+      case _: UnboundedFollowingWindowFunctionFrame |
+           _: SlidingWindowFunctionFrame |
+           _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow
+      // It should be impossible to get other types of window function frame here
+      case frame => throw QueryExecutionErrors.unexpectedWindowFunctionFrameError(frame.toString)
+    }
+
+    val requiredIndices = functionFrames.map {
+      case _: UnboundedWindowFunctionFrame => 0
+      case _ => 2
+    }
+
+    val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail
+
+    val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) =>
+      if (num == 0) {
+        // Sentinel values for unbounded window
+        (-1, -1)
+      } else {
+        (upperBoundIndex - 2, upperBoundIndex - 1)
+      }
+    }
+
+    def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1
+
+    def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2
+
+    (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes)
+  }
+
+  // Unwrap the expressions and factories from the map.
+  private val expressionsWithFrameIndex =

Review Comment:
   I haven't come across a factory that needs to define member variables (except for parameters)😂



-- 
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


[GitHub] [spark] cloud-fan closed pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #41939: [SPARK-44341][SQL][PYTHON] Define the computing logic through PartitionEvaluator API and use it in WindowExec and WindowInPandasExec
URL: https://github.com/apache/spark/pull/41939


-- 
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