You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "Hisoka-X (via GitHub)" <gi...@apache.org> on 2023/07/17 06:03:31 UTC

[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

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