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


##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -1057,6 +1063,14 @@
     },
     "sqlState" : "4274K"
   },
+  "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {

Review Comment:
   just a 2 cents: Probably good to provide the actual event time value as 
well? Users have to fix the issue to resume the query, hence they are required 
to debug their logic by themselves. Maybe slightly helpful.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -107,25 +108,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 {
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions[InternalRow](

Review Comment:
   nit: `{ dataIterator =>`
   
   We can save one indentation.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -49,26 +80,31 @@ case class EventTimeWatermark(
   // logic here because we also maintain the compatibility flag. (See
   // SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE for details.)
   // TODO: Disallow updating the metadata once we remove the compatibility 
flag.
-  override val output: Seq[Attribute] = child.output.map { a =>
-    if (a semanticEquals eventTime) {
-      val delayMs = EventTimeWatermark.getDelayMs(delay)
-      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: LogicalPlan): 
EventTimeWatermark =
     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.
+ */
+case class UpdateEventTimeWatermarkColumn(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = {
+    val delayMs = EventTimeWatermark.getDelayMs(delay)
+    updateEventTimeColumn(child.output, delayMs, eventTime)
+}

Review Comment:
   nit: 2 spaces?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -78,15 +78,32 @@ case class TransformWithStateExec(
   override def shortName: String = "transformWithStateExec"
 
   override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
+    if (timeMode == ProcessingTime) {
+      // TODO: check if we can return true only if actual timers are 
registered, or there is
+      // expired state
+      true
+    } else if (outputMode == OutputMode.Append || outputMode == 
OutputMode.Update) {
+      eventTimeWatermarkForEviction.isDefined &&
+      newInputWatermark > eventTimeWatermarkForEviction.get
+    } else {
+      false
+    }
+  }
+
+  /**
+   * Controls watermark propagation to downstream modes. If timeMode is
+   * ProcessingTime, the output rows cannot be interpreted in eventTime, hence
+   * this node will not propagate watermark in this timeMode.
+   *
+   * For timeMode EventTime, output watermark is same as input Watermark 
because
+   * transformWithState node does not buffer any input rows between 
micro-batches.

Review Comment:
   I'm not sure it's correct to say the node does not buffer any input rows 
hence input watermark = output watermark. We can implement session window in 
TWS which effectively buffers input rows.
   
   The reason we can still apply input watermark = output watermark even with 
streaming aggregation is described in the method doc for trait, and we have 
slightly different reasoning here. The right reasoning here is, 
transformWithState does not allow users to set the event time column to be 
earlier than the watermark, hence it's enforced.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1

Review Comment:
   Do we depend on this? Given how this is calculated, it seems to lead 
confusion as it's not a count of input rows for a grouping key but a count of 
input rows which replace the min value. I'd simplify if we are not using this 
to test some tricky case.



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -702,6 +742,39 @@ class KeyValueGroupedDataset[K, V] private[sql](
     transformWithState(statefulProcessor, timeMode, outputMode)(outputEncoder)
   }
 
+  /**
+   * (Java-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.
+   *
+   * 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.
+   * @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 outputEncoder     Encoder for the output type.
+   *
+   * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+   */
+  private[sql] def transformWithState[U: Encoder](
+      statefulProcessor: StatefulProcessor[K, V, U],
+      eventTimeColumnName: String,
+      outputMode: OutputMode,
+      outputEncoder: Encoder[U]): Dataset[U] = {
+    transformWithState(statefulProcessor, TimeMode.EventTime(), 
outputMode)(outputEncoder)

Review Comment:
   eventTimeColumnName isn't used. Looks like a silly bug?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,33 @@ class IncrementalExecution(
           eventTimeWatermarkForEviction = 
inputWatermarkForEviction(m.stateInfo.get)
         )
 
+      // UpdateEventTimeColumnExec is used to tag the eventTime column, and 
validate
+      // emitted rows adhere to watermark in the output of transformWithState.
+      // Hence, this node shares the same watermark value as 
TransformWithStateExec.
+      // However, given that UpdateEventTimeColumnExec does not store any 
state, it
+      // does not have any StateInfo. We simply use the StateInfo of 
transformWithStateExec
+      // to propagate watermark to both UpdateEventTimeColumnExec and 
transformWithStateExec.
+      case UpdateEventTimeColumnExec(eventTime, delay, _,

Review Comment:
   I guess this could be simplified like below:
   
   ```
         case UpdateEventTimeColumnExec(eventTime, delay, None,
           SerializeFromObjectExec(serializer, t: TransformWithStateExec)) if 
t.stateInfo.isDefined =>
   
           val stateInfo = t.stateInfo.get
   
           val eventTimeWatermarkForLateEvents = 
inputWatermarkForLateEvents(stateInfo)
           val eventTimeWatermarkForEviction = 
inputWatermarkForLateEvents(stateInfo)
   
           UpdateEventTimeColumnExec(eventTime, delay, 
eventTimeWatermarkForEviction,
             SerializeFromObjectExec(serializer,
               t.copy(
                 eventTimeWatermarkForLateEvents = 
eventTimeWatermarkForLateEvents,
                 eventTimeWatermarkForEviction = eventTimeWatermarkForEviction)
             )
           )
   ```
   
   Two points: 
   1) match with None explicitly so that the pattern will match only once even 
the rule runs multiple times.
   2) remove out unnecessary de-structuring and use copy.



##########
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:
   This is probably too early to determine - let's say we have (temp) view. We 
need to pass the logical plan through analyzer to resolve node(s) and safely 
check the event time column after children are resolved.
   
   That said, maybe we need to do this with analysis rule, likewise we did with 
window function & TimeWindowing rule.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }

Review Comment:
   nit: same, one-liner



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {

Review Comment:
   nit: one more space



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }

Review Comment:
   nit: up one line (one-liner)



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (timerValues.getCurrentWatermarkInMs() > 0) {
+      Iterator.single(
+        OutputRow(
+          key,
+          
Timestamp.from(Instant.ofEpochMilli(timerValues.getCurrentWatermarkInMs() - 1)),
+          inputRows.size))
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (timerValues.getCurrentWatermarkInMs() > 0) {
+      Iterator.single(
+        OutputRow(
+          key,
+          
Timestamp.from(Instant.ofEpochMilli(timerValues.getCurrentWatermarkInMs() - 1)),
+          inputRows.size))
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateChainingSuite extends StreamTest
+  with Logging {
+  import testImplicits._
+
+  test("watermark is propagated correctly for next stateful operator" +
+    " after transformWithState") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+        .as[AggEventRow]
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing 
is
+        // emitted as all records have timestamp > epoch
+        CheckNewAnswer(),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        // global watermark should now be 1 minute behind  `2024-02-01 
00:00:00`.
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 
00:01:00")), 1)
+        ),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 
00:01:00")), 1)
+        )
+      )
+    }
+  }
+
+  test("passing eventTime column to transformWithState fails if" +
+    " no watermark is defined") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+      }
+
+      
assert(ex.getMessage.contains("CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK"))

Review Comment:
   nit: let's try checkError for exception bound to error class framework



##########
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:
   I also don't know much about it, but it's more about RDD level planning and 
what we have to take care instead is SQL level planning - we need to override 
two methods 1) outputPartitioning and 2) outputOrdering and produce the same 
with child.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (timerValues.getCurrentWatermarkInMs() > 0) {
+      Iterator.single(
+        OutputRow(
+          key,
+          
Timestamp.from(Instant.ofEpochMilli(timerValues.getCurrentWatermarkInMs() - 1)),
+          inputRows.size))
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateChainingSuite extends StreamTest
+  with Logging {
+  import testImplicits._
+
+  test("watermark is propagated correctly for next stateful operator" +
+    " after transformWithState") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+        .as[AggEventRow]
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing 
is
+        // emitted as all records have timestamp > epoch
+        CheckNewAnswer(),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        // global watermark should now be 1 minute behind  `2024-02-01 
00:00:00`.
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 
00:01:00")), 1)
+        ),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 
00:01:00")), 1)
+        )
+      )
+    }
+  }
+
+  test("passing eventTime column to transformWithState fails if" +
+    " no watermark is defined") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+      }
+
+      
assert(ex.getMessage.contains("CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK"))
+    }
+  }
+
+  test("missing eventTime column to transformWithState fails the query if" +
+    " another stateful operator is added") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("there are streaming aggregations on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("chaining multiple transformWithState operators") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new InputCountStatefulProcessor[OutputRow](),
+          TimeMode.None(),
+          OutputMode.Append()
+        )
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(1)
+      )
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator" +
+    " fails if watermark column is not provided") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("dropDuplicatesWithinWatermark is not 
supported on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 1)),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        }
+      )
+    }
+  }
+
+  test("query fails if the output dataset does not contain specified 
eventTimeColumn") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[ExtendedAnalysisException] {
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[OutputRow](
+            new TestStatefulProcessor(),
+            "missingEventTimeColumn",
+            OutputMode.Append())
+
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+
+      assert(ex.getMessage.contains("UNRESOLVED_COLUMN.WITH_SUGGESTION"))
+    }
+  }
+
+  test("query fails if the output dataset contains rows older than current 
watermark") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new StatefulProcessorEmittingRowsOlderThanWatermark(),
+          "outputEventTime",
+          OutputMode.Append())
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 1)),
+        // this batch would fail now
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        ExpectFailure[SparkRuntimeException] { ex =>
+          logWarning(s"${ex.getMessage}")
+          
assert(ex.getMessage.contains("EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED"))

Review Comment:
   ditto, checkError?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (timerValues.getCurrentWatermarkInMs() > 0) {
+      Iterator.single(
+        OutputRow(
+          key,
+          
Timestamp.from(Instant.ofEpochMilli(timerValues.getCurrentWatermarkInMs() - 1)),
+          inputRows.size))
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateChainingSuite extends StreamTest
+  with Logging {
+  import testImplicits._
+
+  test("watermark is propagated correctly for next stateful operator" +
+    " after transformWithState") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+        .as[AggEventRow]
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing 
is
+        // emitted as all records have timestamp > epoch
+        CheckNewAnswer(),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        // global watermark should now be 1 minute behind  `2024-02-01 
00:00:00`.
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 
00:01:00")), 1)
+        ),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 
00:01:00")), 1)
+        )
+      )
+    }
+  }
+
+  test("passing eventTime column to transformWithState fails if" +
+    " no watermark is defined") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+      }
+
+      
assert(ex.getMessage.contains("CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK"))
+    }
+  }
+
+  test("missing eventTime column to transformWithState fails the query if" +
+    " another stateful operator is added") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("there are streaming aggregations on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("chaining multiple transformWithState operators") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new InputCountStatefulProcessor[OutputRow](),
+          TimeMode.None(),
+          OutputMode.Append()
+        )
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(1)
+      )
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator" +
+    " fails if watermark column is not provided") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("dropDuplicatesWithinWatermark is not 
supported on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 1)),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        }
+      )
+    }
+  }
+
+  test("query fails if the output dataset does not contain specified 
eventTimeColumn") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[ExtendedAnalysisException] {
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[OutputRow](
+            new TestStatefulProcessor(),
+            "missingEventTimeColumn",
+            OutputMode.Append())
+
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+
+      assert(ex.getMessage.contains("UNRESOLVED_COLUMN.WITH_SUGGESTION"))
+    }
+  }
+
+  test("query fails if the output dataset contains rows older than current 
watermark") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new StatefulProcessorEmittingRowsOlderThanWatermark(),
+          "outputEventTime",
+          OutputMode.Append())
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 1)),
+        // this batch would fail now
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        ExpectFailure[SparkRuntimeException] { ex =>
+          logWarning(s"${ex.getMessage}")

Review Comment:
   nit: looks like leftover log, let's remove



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.streaming
+
+import java.sql.Timestamp
+import java.time.{Instant, LocalDateTime, ZoneId}
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.ExtendedAnalysisException
+import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.functions.window
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class OutputRow(
+    key: String,
+    outputEventTime: Timestamp,
+    count: Int)
+
+class TestStatefulProcessor
+ extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (inputRows.isEmpty) {
+      Iterator.empty
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+class InputCountStatefulProcessor[T]
+  extends StatefulProcessor[String, T, Int] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[Int] = {
+    Iterator.single(inputRows.size)
+  }
+}
+
+/**
+ * Emits output row with timestamp older than current watermark for batchId > 
0.
+ */
+class StatefulProcessorEmittingRowsOlderThanWatermark
+  extends StatefulProcessor[String, InputEventRow, OutputRow] {
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputRow] = {
+    if (timerValues.getCurrentWatermarkInMs() > 0) {
+      Iterator.single(
+        OutputRow(
+          key,
+          
Timestamp.from(Instant.ofEpochMilli(timerValues.getCurrentWatermarkInMs() - 1)),
+          inputRows.size))
+    } else {
+      var minEventTime = inputRows.next().eventTime
+      var count = 1
+      inputRows.foreach { row =>
+        if (row.eventTime.before(minEventTime)) {
+          minEventTime = row.eventTime
+          count += 1
+        }
+      }
+      Iterator.single(OutputRow(key, minEventTime, count))
+    }
+  }
+}
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateChainingSuite extends StreamTest
+  with Logging {
+  import testImplicits._
+
+  test("watermark is propagated correctly for next stateful operator" +
+    " after transformWithState") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+        .as[AggEventRow]
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        // watermark should be 1 minute behind `2024-01-01 00:00:00`, nothing 
is
+        // emitted as all records have timestamp > epoch
+        CheckNewAnswer(),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        // global watermark should now be 1 minute behind  `2024-02-01 
00:00:00`.
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-01-01 00:00:00"), timestamp("2024-01-01 
00:01:00")), 1)
+        ),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(AggEventRow(
+          Window(timestamp("2024-02-01 00:00:00"), timestamp("2024-02-01 
00:01:00")), 1)
+        )
+      )
+    }
+  }
+
+  test("passing eventTime column to transformWithState fails if" +
+    " no watermark is defined") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+      }
+
+      
assert(ex.getMessage.contains("CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK"))
+    }
+  }
+
+  test("missing eventTime column to transformWithState fails the query if" +
+    " another stateful operator is added") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .groupBy(window($"outputEventTime", "1 minute"))
+        .count()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("there are streaming aggregations on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("chaining multiple transformWithState operators") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new InputCountStatefulProcessor[OutputRow](),
+          TimeMode.None(),
+          OutputMode.Append()
+        )
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-01-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2023-12-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(1),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        },
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-02 
00:00:00"), "e1")),
+        CheckNewAnswer(1)
+      )
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator" +
+    " fails if watermark column is not provided") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          TimeMode.None(),
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      val ex = intercept[ExtendedAnalysisException] {
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+      assert(ex.getMessage.contains("dropDuplicatesWithinWatermark is not 
supported on" +
+        " streaming DataFrames/DataSets without watermark"))
+    }
+  }
+
+  test("dropDuplicateWithWatermark after transformWithState operator") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+      val result = inputData.toDS()
+        .withWatermark("eventTime", "1 minute")
+        .groupByKey(x => x.key)
+        .transformWithState[OutputRow](
+          new TestStatefulProcessor(),
+          "outputEventTime",
+          OutputMode.Append())
+        .dropDuplicatesWithinWatermark()
+
+      testStream(result, OutputMode.Append())(
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        AddData(inputData, InputEventRow("k1", timestamp("2024-02-01 
00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 1)),
+        Execute("assertWatermarkEquals") { q =>
+          assertWatermarkEquals(q, timestamp("2024-01-31 23:59:00"))
+        }
+      )
+    }
+  }
+
+  test("query fails if the output dataset does not contain specified 
eventTimeColumn") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputEventRow]
+
+      val ex = intercept[ExtendedAnalysisException] {
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[OutputRow](
+            new TestStatefulProcessor(),
+            "missingEventTimeColumn",
+            OutputMode.Append())
+
+        testStream(result, OutputMode.Append())(
+          StartStream()
+        )
+      }
+
+      assert(ex.getMessage.contains("UNRESOLVED_COLUMN.WITH_SUGGESTION"))

Review Comment:
   ditto, checkError?



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