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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -107,25 +109,67 @@ case class EventTimeWatermarkExec(
   }
 
   // Update the metadata on the eventTime column to include the desired delay.
-  override val output: Seq[Attribute] = child.output.map { a =>
-    if (a semanticEquals eventTime) {
-      val updatedMetadata = new MetadataBuilder()
-        .withMetadata(a.metadata)
-        .putLong(EventTimeWatermark.delayKey, delayMs)
-        .build()
-      a.withMetadata(updatedMetadata)
-    } else if (a.metadata.contains(EventTimeWatermark.delayKey)) {
-      // Remove existing watermark
-      val updatedMetadata = new MetadataBuilder()
-        .withMetadata(a.metadata)
-        .remove(EventTimeWatermark.delayKey)
-        .build()
-      a.withMetadata(updatedMetadata)
-    } else {
-      a
-    }
+  override val output: Seq[Attribute] = {
+    val delayMs = EventTimeWatermark.getDelayMs(delay)
+    updateEventTimeColumn(child.output, delayMs, eventTime)
   }
 
   override protected def withNewChildInternal(newChild: SparkPlan): 
EventTimeWatermarkExec =
     copy(child = newChild)
 }
+
+/**
+ * Updates the event time column to [[eventTime]] in the child output.
+ * Any watermark calculations performed after this node will use the
+ * updated eventTimeColumn.
+ *
+ * This node also ensures that output emitted by the child node adheres
+ * to watermark. If the child node emits rows which are older than global
+ * watermark, the node will throw an query execution error and fail the user
+ * query.
+ */
+case class UpdateEventTimeColumnExec(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan) extends UnaryExecNode with Logging {
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions[InternalRow](
+      dataIterator => {
+        val watermarkExpression = WatermarkSupport.watermarkExpression(
+          Some(eventTime), eventTimeWatermarkForEviction)
+
+        if (watermarkExpression.isEmpty) {
+          // watermark should always be defined in this node.
+          throw QueryExecutionErrors.cannotGetEventTimeWatermarkError()
+        }
+
+        val predicate = Predicate.create(watermarkExpression.get, child.output)
+        new Iterator[InternalRow] {
+          override def hasNext: Boolean = dataIterator.hasNext
+          override def next(): InternalRow = {
+            val nextRow = dataIterator.next()
+            if (predicate.eval(nextRow)) {
+              // child node emitted a row which is older than current watermark
+              // which is not allowed
+              throw QueryExecutionErrors.emittedRowsAreOlderThanWatermark(
+                eventTimeWatermarkForEviction.get)
+            }
+            nextRow
+          }
+        }
+      },
+      preservesPartitioning = true

Review Comment:
   As per my understanding, this preserves the partitioning from the child to 
upstream operators - and we want to keep that partitioning - because 
UpdateEventTimeColumnExec node does not modify the order of input rows. 
   
   However, I am confused by Javadoc of `mapPartitions` suggesting this 
parameter should only used for PairRDDs. @HeartSaVioR do you have more context 
here? 



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -739,6 +741,128 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  /**
+   * (Scala-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state 
API v2.
+   * We allow the user to act on per-group set of input rows along with keyed 
state and the
+   * user can choose to output/return 0 or more rows.
+   * For a streaming dataframe, we will repeatedly invoke the interface 
methods for new rows
+   * in each trigger and the user's state/state variables will be stored 
persistently across
+   * invocations.
+   *
+   * @tparam U The type of the output 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 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.
+   *
+   * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+   */
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeMode: TimeMode,
+       eventTimeColumnName: String,
+       outputMode: OutputMode): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.flatMap {
+      case EventTimeWatermark(_, delay, _) => Seq(delay)
+      case _ => Seq()
+    }
+
+    if (existingWatermarkDelay.isEmpty) {
+      throw QueryCompilationErrors.cannotAssignEventTimeColumn()
+    }
+
+    val transformWithState = TransformWithState[K, V, U](
+      groupingAttributes,
+      dataAttributes,
+      statefulProcessor,
+      timeMode,
+      outputMode,
+      child = logicalPlan
+    )
+
+    val twsDS = Dataset[U](
+      sparkSession,
+      transformWithState
+    )
+
+    val delay = existingWatermarkDelay.head
+
+    Dataset[U](sparkSession, EliminateEventTimeWatermark(
+      UpdateEventTimeWatermarkColumn(
+        UnresolvedAttribute(eventTimeColumnName),
+        delay,
+        twsDS.logicalPlan)))
+  }
+
+  /**
+   * (Scala-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state 
API v2.
+   * Functions as the function above, but with additional initial state.
+   *
+   * @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 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],
+      timeMode: TimeMode,
+      eventTimeColumnName: String,
+      outputMode: OutputMode,
+      initialState: KeyValueGroupedDataset[K, S]): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.collect {
+      case EventTimeWatermark(_, delay, _) => delay
+    }
+
+    if (existingWatermarkDelay.isEmpty) {

Review Comment:
   yep, done



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