sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591313779


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -739,35 +812,145 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  /**
+   * (Scala-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state 
API v2.
+   * Functions as the function above, but with additional eventTimeColumnName 
for output.
+   *
+   * @tparam U The type of the output objects. Must be encodable to Spark SQL 
types.
+   * @tparam S The type of initial state objects. Must be encodable to Spark 
SQL types.
+   *
+   * Downstream operators would use specified eventTimeColumnName to calculate 
watermark.
+   * Note that TimeMode is set to EventTime to ensure correct flow of 
watermark.
+   *
+   * @param statefulProcessor   Instance of statefulProcessor whose functions 
will
+   *                            be invoked by the operator.
+   * @param eventTimeColumnName eventTime column in the output dataset. Any 
operations after
+   *                            transformWithState will use the new 
eventTimeColumn. The user
+   *                            needs to ensure that the eventTime for emitted 
output adheres to
+   *                            the watermark boundary, otherwise streaming 
query will fail.
+   * @param outputMode          The output mode of the stateful processor.
+   * @param initialState        User provided initial state that will be used 
to initiate state for
+   *                            the query in the first batch.
+   *
+   * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+   */
+  private[sql] def transformWithState[U: Encoder, S: Encoder](
+      statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S],
+      eventTimeColumnName: String,
+      outputMode: OutputMode,
+      initialState: KeyValueGroupedDataset[K, S]): Dataset[U] = {
+    val transformWithState = TransformWithState[K, V, U, S](
+      groupingAttributes,
+      dataAttributes,
+      statefulProcessor,
+      TimeMode.EventTime(),
+      outputMode,
+      child = logicalPlan,
+      initialState.groupingAttributes,
+      initialState.dataAttributes,
+      initialState.queryExecution.analyzed
+    )
+
+    updateEventTimeColumnAfterTransformWithState(transformWithState, 
eventTimeColumnName)
+  }
+
   /**
    * (Java-specific)
    * Invokes methods defined in the stateful processor used in arbitrary state 
API v2.
-   * Functions as the function above, but with additional initial state.
+   * Functions as the function above, but with additional initialStateEncoder 
for state encoding.
+   *
+   * @tparam U The type of the output objects. Must be encodable to Spark SQL 
types.
+   * @tparam S The type of initial state objects. Must be encodable to Spark 
SQL types.
+   * @param statefulProcessor   Instance of statefulProcessor whose functions 
will
+   *                            be invoked by the operator.
+   * @param timeMode            The time mode semantics of the stateful 
processor for
+   *                            timers and TTL.
+   * @param outputMode          The output mode of the stateful processor.
+   * @param initialState        User provided initial state that will be used 
to initiate state for
+   *                            the query in the first batch.
+   * @param outputEncoder       Encoder for the output type.
+   * @param initialStateEncoder Encoder for the initial state type.
+   *
+   * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+   */
+  private[sql] def transformWithState[U: Encoder, S: Encoder](
+      statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S],
+      timeMode: TimeMode,
+      outputMode: OutputMode,
+      initialState: KeyValueGroupedDataset[K, S],
+      outputEncoder: Encoder[U],
+      initialStateEncoder: Encoder[S]): Dataset[U] = {
+    transformWithState(statefulProcessor, timeMode,
+      outputMode, initialState)(outputEncoder, initialStateEncoder)
+  }
+
+  /**
+   * (Java-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state 
API v2.
+   * Functions as the function above, but with additional eventTimeColumnName 
for output.
+   *
+   * Downstream operators would use specified eventTimeColumnName to calculate 
watermark.
+   * Note that TimeMode is set to EventTime to ensure correct flow of 
watermark.
    *
    * @tparam U The type of the output objects. Must be encodable to Spark SQL 
types.
    * @tparam S The type of initial state objects. Must be encodable to Spark 
SQL types.
    * @param statefulProcessor Instance of statefulProcessor whose functions 
will
    *                          be invoked by the operator.
-   * @param timeMode          The time mode semantics of the stateful 
processor for timers and TTL.
    * @param outputMode        The output mode of the stateful processor.
    * @param initialState      User provided initial state that will be used to 
initiate state for
    *                          the query in the first batch.
+   * @param eventTimeColumnName event column in the output dataset. Any 
operations after
+   *                            transformWithState will use the new 
eventTimeColumn. The user
+   *                            needs to ensure that the eventTime for emitted 
output adheres to
+   *                            the watermark boundary, otherwise streaming 
query will fail.
    * @param outputEncoder     Encoder for the output type.
    * @param initialStateEncoder Encoder for the initial state type.
    *
    * See [[Encoder]] for more details on what types are encodable to Spark SQL.
    */
   private[sql] def transformWithState[U: Encoder, S: Encoder](
       statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S],
-      timeMode: TimeMode,
       outputMode: OutputMode,
       initialState: KeyValueGroupedDataset[K, S],
+      eventTimeColumnName: String,
       outputEncoder: Encoder[U],
       initialStateEncoder: Encoder[S]): Dataset[U] = {
-    transformWithState(statefulProcessor, timeMode,
+    transformWithState(statefulProcessor, eventTimeColumnName,
       outputMode, initialState)(outputEncoder, initialStateEncoder)
   }
 
+  /**
+   * Creates a new dataset with updated eventTimeColumn after the 
transformWithState
+   * logical node.
+   */
+  private def updateEventTimeColumnAfterTransformWithState[U: Encoder](
+      transformWithState: LogicalPlan,
+      eventTimeColumnName: String): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.collect {

Review Comment:
   Hmm, good point. I added a Analyzer rule 
`ResolveUpdateEventTimeWatermarkColumn` to extract watermark delay at the end 
of resolution.  



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

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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

Reply via email to