HeartSaVioR commented on code in PR #47133:
URL: https://github.com/apache/spark/pull/47133#discussion_r1713781596


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala:
##########
@@ -161,6 +161,41 @@ case class FlatMapGroupsInPandasWithState(
     newChild: LogicalPlan): FlatMapGroupsInPandasWithState = copy(child = 
newChild)
 }
 
+object TransformWithStateInPandas {
+  def apply(functionExpr: Expression,

Review Comment:
   nit: move the param to the next line



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala:
##########
@@ -161,6 +161,41 @@ case class FlatMapGroupsInPandasWithState(
     newChild: LogicalPlan): FlatMapGroupsInPandasWithState = copy(child = 
newChild)
 }
 
+object TransformWithStateInPandas {

Review Comment:
   Any reason we can't just use the generated constructor of case class? params 
here are exactly the same with constructor param in case class.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala:
##########
@@ -161,6 +161,41 @@ case class FlatMapGroupsInPandasWithState(
     newChild: LogicalPlan): FlatMapGroupsInPandasWithState = copy(child = 
newChild)
 }
 
+object TransformWithStateInPandas {
+  def apply(functionExpr: Expression,
+      groupingAttributes: Seq[Attribute],
+      outputAttrs: Seq[Attribute],
+      outputMode: OutputMode,
+      timeMode: TimeMode,
+      child: LogicalPlan): TransformWithStateInPandas = {
+
+    new TransformWithStateInPandas(
+      functionExpr,
+      groupingAttributes,
+      outputAttrs,
+      outputMode,
+      timeMode,
+      child = child
+    )
+  }
+}
+
+case class TransformWithStateInPandas(

Review Comment:
   nit: shall we add a short description as class doc while we are here?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala:
##########
@@ -161,6 +161,41 @@ case class FlatMapGroupsInPandasWithState(
     newChild: LogicalPlan): FlatMapGroupsInPandasWithState = copy(child = 
newChild)
 }
 
+object TransformWithStateInPandas {
+  def apply(functionExpr: Expression,
+      groupingAttributes: Seq[Attribute],
+      outputAttrs: Seq[Attribute],
+      outputMode: OutputMode,
+      timeMode: TimeMode,
+      child: LogicalPlan): TransformWithStateInPandas = {
+
+    new TransformWithStateInPandas(

Review Comment:
   nit: omit `new` from case class instantiation



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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 scala.concurrent.duration.NANOSECONDS
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.JobArtifactSet
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, 
Expression, PythonUDF, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, 
groupAndProject, resolveArgOffsets}
+import org.apache.spark.sql.execution.streaming.{StatefulOperatorPartitioning, 
StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, 
WatermarkSupport}
+import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
StateSchemaValidationResult, StateStore, StateStoreOps}
+import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing
+ * [[org.apache.spark.sql.catalyst.plans.logical.TransformWithStateInPandas]]
+ * @param functionExpr function called on each group
+ * @param groupingAttributes used to group the data
+ * @param output used to define the output rows
+ * @param outputMode defines the output mode for the statefulProcessor
+ * @param timeMode The time mode semantics of the stateful processor for 
timers and TTL.
+ * @param stateInfo Used to identify the state store for a given operator.
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering 
late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateInPandasExec(
+    functionExpr: Expression,
+    groupingAttributes: Seq[Attribute],
+    output: Seq[Attribute],
+    outputMode: OutputMode,
+    timeMode: TimeMode,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan) extends UnaryExecNode with StateStoreWriter with 
WatermarkSupport {
+
+  private val pythonUDF = functionExpr.asInstanceOf[PythonUDF]
+  private val pythonFunction = pythonUDF.func
+  private val chainedFunc =
+    Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id))
+
+  private val sessionLocalTimeZone = conf.sessionLocalTimeZone
+  private val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf)
+  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+
+  private val groupingKeyStructFields = groupingAttributes
+    .map(a => StructField(a.name, a.dataType, a.nullable))
+  private val groupingKeySchema = StructType(groupingKeyStructFields)
+  private val groupingKeyExprEncoder = ExpressionEncoder(groupingKeySchema)
+    .resolveAndBind().asInstanceOf[ExpressionEncoder[Any]]
+
+  /** The keys that may have a watermark attribute. */
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", 
BinaryType)

Review Comment:
   Maybe better to leave a code comment to remind that each state variable has 
its own schema and this is just dummy one.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.DataOutputStream
+import java.net.ServerSocket
+
+import scala.concurrent.ExecutionContext
+
+import org.apache.arrow.vector.VectorSchemaRoot
+import org.apache.arrow.vector.ipc.ArrowStreamWriter
+
+import org.apache.spark.TaskContext
+import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, 
PythonRDD}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.arrow
+import org.apache.spark.sql.execution.arrow.ArrowWriter
+import org.apache.spark.sql.execution.metric.SQLMetric
+import 
org.apache.spark.sql.execution.python.TransformWithStateInPandasPythonRunner.{InType,
 OutType}
+import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleImpl
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Python runner implementation for TransformWithStateInPandas.
+ */
+class TransformWithStateInPandasPythonRunner(
+    funcs: Seq[(ChainedPythonFunctions, Long)],
+    evalType: Int,
+    argOffsets: Array[Array[Int]],
+    _schema: StructType,
+    processorHandle: StatefulProcessorHandleImpl,
+    _timeZoneId: String,
+    initialWorkerConf: Map[String, String],
+    override val pythonMetrics: Map[String, SQLMetric],
+    jobArtifactUUID: Option[String],
+    groupingKeySchema: StructType)
+  extends BasePythonRunner[InType, OutType](funcs.map(_._1), evalType, 
argOffsets, jobArtifactUUID)
+    with PythonArrowInput[InType]

Review Comment:
   nit: same indentation with extends



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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 scala.concurrent.duration.NANOSECONDS
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.JobArtifactSet
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, 
Expression, PythonUDF, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, 
groupAndProject, resolveArgOffsets}
+import org.apache.spark.sql.execution.streaming.{StatefulOperatorPartitioning, 
StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, 
WatermarkSupport}
+import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
StateSchemaValidationResult, StateStore, StateStoreOps}
+import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing
+ * [[org.apache.spark.sql.catalyst.plans.logical.TransformWithStateInPandas]]
+ * @param functionExpr function called on each group
+ * @param groupingAttributes used to group the data
+ * @param output used to define the output rows
+ * @param outputMode defines the output mode for the statefulProcessor
+ * @param timeMode The time mode semantics of the stateful processor for 
timers and TTL.
+ * @param stateInfo Used to identify the state store for a given operator.
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering 
late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateInPandasExec(
+    functionExpr: Expression,
+    groupingAttributes: Seq[Attribute],
+    output: Seq[Attribute],
+    outputMode: OutputMode,
+    timeMode: TimeMode,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan) extends UnaryExecNode with StateStoreWriter with 
WatermarkSupport {
+
+  private val pythonUDF = functionExpr.asInstanceOf[PythonUDF]
+  private val pythonFunction = pythonUDF.func
+  private val chainedFunc =
+    Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id))
+
+  private val sessionLocalTimeZone = conf.sessionLocalTimeZone
+  private val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf)
+  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+
+  private val groupingKeyStructFields = groupingAttributes
+    .map(a => StructField(a.name, a.dataType, a.nullable))
+  private val groupingKeySchema = StructType(groupingKeyStructFields)
+  private val groupingKeyExprEncoder = ExpressionEncoder(groupingKeySchema)
+    .resolveAndBind().asInstanceOf[ExpressionEncoder[Any]]
+
+  /** The keys that may have a watermark attribute. */
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", 
BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", 
BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  override def validateAndMaybeEvolveStateSchema(
+      hadoopConf: Configuration,
+      batchId: Long,
+      stateSchemaVersion: Int): List[StateSchemaValidationResult] = {
+    List.empty

Review Comment:
   What is the plan with this? Does this need to follow the same path of Scala 
version of transformWithState and it's just pending other PR's change? If then 
let's file a JIRA ticket and add TODO comment here. Otherwise please leave a 
comment about why we don't need to handle anything here unlike Scala version.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasExec.scala:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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 scala.concurrent.duration.NANOSECONDS
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.JobArtifactSet
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, 
Expression, PythonUDF, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, 
groupAndProject, resolveArgOffsets}
+import org.apache.spark.sql.execution.streaming.{StatefulOperatorPartitioning, 
StatefulOperatorStateInfo, StatefulProcessorHandleImpl, StateStoreWriter, 
WatermarkSupport}
+import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
StateSchemaValidationResult, StateStore, StateStoreOps}
+import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing
+ * [[org.apache.spark.sql.catalyst.plans.logical.TransformWithStateInPandas]]
+ * @param functionExpr function called on each group
+ * @param groupingAttributes used to group the data
+ * @param output used to define the output rows
+ * @param outputMode defines the output mode for the statefulProcessor
+ * @param timeMode The time mode semantics of the stateful processor for 
timers and TTL.
+ * @param stateInfo Used to identify the state store for a given operator.
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering 
late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateInPandasExec(
+    functionExpr: Expression,
+    groupingAttributes: Seq[Attribute],
+    output: Seq[Attribute],
+    outputMode: OutputMode,
+    timeMode: TimeMode,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan) extends UnaryExecNode with StateStoreWriter with 
WatermarkSupport {
+
+  private val pythonUDF = functionExpr.asInstanceOf[PythonUDF]
+  private val pythonFunction = pythonUDF.func
+  private val chainedFunc =
+    Seq((ChainedPythonFunctions(Seq(pythonFunction)), pythonUDF.resultId.id))
+
+  private val sessionLocalTimeZone = conf.sessionLocalTimeZone
+  private val pythonRunnerConf = ArrowPythonRunner.getPythonRunnerConfMap(conf)
+  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+
+  private val groupingKeyStructFields = groupingAttributes
+    .map(a => StructField(a.name, a.dataType, a.nullable))
+  private val groupingKeySchema = StructType(groupingKeyStructFields)
+  private val groupingKeyExprEncoder = ExpressionEncoder(groupingKeySchema)
+    .resolveAndBind().asInstanceOf[ExpressionEncoder[Any]]
+
+  /** The keys that may have a watermark attribute. */
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", 
BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", 
BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  override def validateAndMaybeEvolveStateSchema(
+      hadoopConf: Configuration,
+      batchId: Long,
+      stateSchemaVersion: Int): List[StateSchemaValidationResult] = {
+    List.empty
+  }
+
+  /**
+   * Produces the result of the query as an `RDD[InternalRow]`
+   */
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics
+
+    val (dedupAttributes, argOffsets) = resolveArgOffsets(child.output, 
groupingAttributes)
+
+    child.execute().mapPartitionsWithStateStore[InternalRow](
+      getStateInfo,
+      schemaForKeyRow,
+      schemaForValueRow,
+      NoPrefixKeyStateEncoderSpec(schemaForKeyRow),
+      session.sqlContext.sessionState,
+      Some(session.sqlContext.streams.stateStoreCoordinator),
+      useColumnFamilies = true,
+      useMultipleValuesPerKey = true
+    ) {
+      case (store: StateStore, dataIterator: Iterator[InternalRow]) =>
+        val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+        val commitTimeMs = longMetric("commitTimeMs")
+        val currentTimeNs = System.nanoTime
+        val updatesStartTimeNs = currentTimeNs
+
+        val data = groupAndProject(dataIterator, groupingAttributes, 
child.output, dedupAttributes)
+
+        val processorHandle = new StatefulProcessorHandleImpl(store, 
getStateInfo.queryRunId,
+          groupingKeyExprEncoder, timeMode)
+        val runner = new TransformWithStateInPandasPythonRunner(
+          chainedFunc,
+          PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF,
+          Array(argOffsets),
+          DataTypeUtils.fromAttributes(dedupAttributes),
+          processorHandle,
+          sessionLocalTimeZone,
+          pythonRunnerConf,
+          pythonMetrics,
+          jobArtifactUUID,
+          groupingKeySchema
+        )
+
+        val outputIterator = executePython(data, output, runner)
+
+        CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, 
{

Review Comment:
   Where we count `numOutputRows` in this node?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to