You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "sahnib (via GitHub)" <gi...@apache.org> on 2024/03/04 23:59:46 UTC

[PR] Allow chaining other stateful operators after transformWIthState operator. [spark]

sahnib opened a new pull request, #45376:
URL: https://github.com/apache/spark/pull/45376

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   This PR adds support to define event time column in the output dataset of `TransformWithState` operator. The new event time column will be used to evaluate watermark expressions in downstream operators. 
   
   1. Note that the transformWithState operator does not enforce that values generated by user's computation adhere to the watermark semantics. (no output rows are generated which have event time less than watermark). 
   2. Updated the watermark value passed in TimerInfo as evictionWatermark, rather than lateEventsWatermark. 
   3. Ensure that event time column can only be defined in output if a watermark has been defined previously. 
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     5. If you fix a bug, you can clarify why it is a bug.
   -->
   
   This change is required to support chaining of stateful operators after `transformWithState`. Event time column is required to evaluate watermark expressions in downstream stateful operators. 
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Yes. Adds a new version of transformWithState API which allows redefining the event time column. 
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   Added unit test cases. 
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   No
   


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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576698993


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

Review Comment:
   Are we using `Logging` here ? should we remove this and the import otherwise ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576703836


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -78,15 +78,20 @@ case class TransformWithStateExec(
   override def shortName: String = "transformWithStateExec"
 
   override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
-    timeMode match {
-      case ProcessingTime =>
-        // TODO: check if we can return true only if actual timers are registered, or there is
-        // expired state
-        true
-      case EventTime =>
-        eventTimeWatermarkForEviction.isDefined &&
-          newInputWatermark > eventTimeWatermarkForEviction.get
-      case _ => false
+    if (outputMode == OutputMode.Append || outputMode == OutputMode.Update) {

Review Comment:
   Hmm - won't we have an issue with `Append` mode using proc time based time mode ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591715505


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUpdateEventTimeWatermarkColumn.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, UpdateEventTimeWatermarkColumn}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * Extracts the watermark delay and adds it to the UpdateEventTimeWatermarkColumn
+ * logical node (if such a node is present). [[UpdateEventTimeWatermarkColumn]] node updates
+ * the eventTimeColumn for upstream operators.
+ *
+ * If the logical plan contains a [[UpdateEventTimeWatermarkColumn]] node, but no watermark
+ * has been defined, the query will fail with a compilation error.
+ */
+object ResolveUpdateEventTimeWatermarkColumn extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
+    case u: UpdateEventTimeWatermarkColumn =>

Review Comment:
   If we are not clear about the child (or sub-tree) at this moment, we shouldn't apply this rule. `if u.childrenResolved` to apply the rule selectively.



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591752313


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, 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,

Review Comment:
   eventTimeWatermarkForLateEvents?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -442,6 +442,16 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
       case EventTimeWatermark(columnName, delay, child) =>
         EventTimeWatermarkExec(columnName, delay, planLater(child)) :: Nil
 
+      case UpdateEventTimeWatermarkColumn(columnName, delay, child) =>
+        // we expect watermarkDelay to be resolved before physical planning.
+        if (delay.isEmpty) {
+          // This is a sanity check. We should not reach here as delay is updated during
+          // query plan resolution in [[ResolveUpdateEventTimeWatermarkColumn]] Analyzer rule.
+          throw SparkException.internalError(
+            "You hit a query analyzer bug. Please report your query to Spark user mailing list.")

Review Comment:
   This error message is basically provided for all internal errors. Could we please add one-liner high-level context here if we have? I guess it'd be just OK with stack trace, so 2 cents.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -107,25 +109,70 @@ 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] { 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 row = dataIterator.next()
+          if (predicate.eval(row)) {
+            // child node emitted a row which is older than current watermark
+            // this is not allowed
+            val boundEventTimeExpression = bindReference[Expression](eventTime, child.output)
+            val eventTimeProjection = UnsafeProjection.create(boundEventTimeExpression)
+            val rowEventTime = eventTimeProjection(row)
+            throw QueryExecutionErrors.emittedRowsAreOlderThanWatermark(
+              eventTimeWatermarkForEviction.get, rowEventTime.getLong(0))
+          }
+          row
+        }
+      }
+    }
+  }
+
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering

Review Comment:
   `override def outputPartitioning: Partitioning = child.outputPartitioning` as well



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, None,
+        SerializeFromObjectExec(serializer,
+        t: TransformWithStateExec)) if t.stateInfo.isDefined =>
+
+        val stateInfo = t.stateInfo.get
+        val eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(stateInfo)
+        val eventTimeWatermarkForEviction = inputWatermarkForLateEvents(stateInfo)

Review Comment:
   inputWatermarkForEviction?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,365 @@
+/*
+ * 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, SparkThrowable}
+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)),

Review Comment:
   Hmm... this is coupled with UpdateEventTimeWatermarkColumn. If we change the node to compare against watermark for late event, this wouldn't break as expected.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,365 @@
+/*
+ * 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, SparkThrowable}
+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 {
+  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())
+      }
+
+      checkError(ex, "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"),
+          InputEventRow("k1", timestamp("2024-02-01 00:00:00"), "e1")),
+        CheckNewAnswer(OutputRow("k1", timestamp("2024-02-01 00:00:00"), 2)),
+        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()
+        )
+      }
+
+      checkError(ex, "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+        parameters = Map(
+          "objectName" -> "`missingEventTimeColumn`",
+          "proposal" -> "`outputEventTime`, `count`, `key`"))
+    }
+  }
+
+  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 =>
+          checkError(ex.asInstanceOf[SparkThrowable],
+            "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED",
+            parameters = Map("currentWatermark" -> "1706774340000",
+              "emittedRowEventTime" -> "1706774339999000"))
+        }
+      )
+    }
+  }

Review Comment:
   Could we add a test case where we have source and watermark definition to be in temporal view and read from temporal view and add transformWithState?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576756401


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

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578226523


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,27 @@ class IncrementalExecution(
           eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
         )
 
+      case UpdateEventTimeColumnExec(eventTime, delay, _,
+        SerializeFromObjectExec(serializer,
+        TransformWithStateExec(keyDeserializer, valueDeserializer, groupingAttributes,
+        dataAttributes, statefulProcessor, timeMode, outputMode,
+        keyEncoder, outputAttr, stateInfo, batchTimestampMs, _,
+        _, child, isStreaming, hasInitialState,
+        initialStateGroupingAttrs, initialStateDataAttrs,
+        initialStateDeserializer, initialState))) if stateInfo.isDefined =>
+
+        val eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(stateInfo.get)

Review Comment:
   Could we add a comment for this block in general ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591349451


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -331,6 +331,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
       Seq(
         ResolveWithCTE,
         ExtractDistributedSequenceID) ++
+      Seq(ResolveUpdateEventTimeWatermarkColumn) ++

Review Comment:
   @HeartSaVioR I have added the `ResolveUpdateEventTimeWatermarkColumn` after all resolution rules. At this place, we are guaranteed (IIUC) to have resolved the query plan and should be able to extract watermark delay from child nodes of UpdateEventTimeWatermarkColumn. Let me know if you think otherwise. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1592763066


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUpdateEventTimeWatermarkColumn.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, UpdateEventTimeWatermarkColumn}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * Extracts the watermark delay and adds it to the UpdateEventTimeWatermarkColumn
+ * logical node (if such a node is present). [[UpdateEventTimeWatermarkColumn]] node updates
+ * the eventTimeColumn for upstream operators.
+ *
+ * If the logical plan contains a [[UpdateEventTimeWatermarkColumn]] node, but no watermark
+ * has been defined, the query will fail with a compilation error.
+ */
+object ResolveUpdateEventTimeWatermarkColumn extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
+    case u: UpdateEventTimeWatermarkColumn =>

Review Comment:
   Done. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, None,
+        SerializeFromObjectExec(serializer,
+        t: TransformWithStateExec)) if t.stateInfo.isDefined =>
+
+        val stateInfo = t.stateInfo.get
+        val eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(stateInfo)
+        val eventTimeWatermarkForEviction = inputWatermarkForLateEvents(stateInfo)

Review Comment:
   Renamed. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576435733


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       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,
+      timeoutMode,
+      outputMode,
+      child = logicalPlan
+    )
+
+    val twsDS = Dataset[U](
+      sparkSession,
+      transformWithState
+    )
+
+    val delay = existingWatermarkDelay.head
+
+    Dataset[U](sparkSession, EliminateEventTimeWatermark(
+      UpdateEventTimeWatermarkColumn(
+        UnresolvedAttribute(eventTimeColumnName),

Review Comment:
   The Analyzer performs a check on all expressions to ensure there are no unresolved attributes. I verified this with a simple test which provides a eventTimeColumn that is not present and it fails with the error 
   
   ```
   org.apache.spark.sql.catalyst.ExtendedAnalysisException: [UNRESOLVED_COLUMN.WITH_SUGGESTION] A column, variable, or function parameter with name `missingEventTimeColum
   n` cannot be resolved. Did you mean one of the following? [`outputEventTime`, `count`, `key`]. SQLSTATE: 42703;
   ```
   
   I have added a unit test case for this behavior. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576693458


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -739,6 +741,128 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  /**
+   * (Scala-specific)

Review Comment:
   This will add 4 more interfaces - 2 on Scala and potentially 2 on Java. Could we consolidate the existing interfaces to optionally take a `eventTimeColumnName` instead ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576686293


##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -1057,6 +1063,14 @@
     },
     "sqlState" : "4274K"
   },
+  "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {
+    "message" : [
+      "Previous node emitted rows which had eventTime older than current watermark value <currentWatermark>",

Review Comment:
   nit: `current_watermark_value=<currentWatermark>` ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576751501


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -762,6 +890,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
       timeMode: TimeMode,
       outputMode: OutputMode,
       initialState: KeyValueGroupedDataset[K, S],
+      eventTimeColumnName: String,

Review Comment:
   Actually, we need multiple separate Java APIs (for combinations of eventTimeColumn and state). I have added them, thanks. 



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -762,6 +890,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
       timeMode: TimeMode,
       outputMode: OutputMode,
       initialState: KeyValueGroupedDataset[K, S],
+      eventTimeColumnName: String,
       outputEncoder: Encoder[U],
       initialStateEncoder: Encoder[S]): Dataset[U] = {
     transformWithState(statefulProcessor, timeMode,

Review Comment:
   Fixed. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589364420


##########
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:
   Thanks, the second parameter needs to be `eventTimeColumnName`. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513576570


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -125,6 +125,12 @@
     ],
     "sqlState" : "428FR"
   },
+  "CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK" : {
+    "message" : [
+      "Watermark needs to be defined to reassign event time column. Could not find watermark definition in the streaming query."

Review Comment:
   Nit: `Failed to find watermark definition`



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576456757


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "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]
+
+      intercept[AnalysisException] {

Review Comment:
   Done



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "1 minute"))

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on PR #45376:
URL: https://github.com/apache/spark/pull/45376#issuecomment-2076008549

   @HeartSaVioR PTAL, thanks. 


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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578604240


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),

Review Comment:
   Discussed offline, we will most likely resolve to only 2 timeModes (ProcessingTime and EventTime - None would be discarded - pending some more discussion). If the user specifies a eventTimeColumn for downstream operators, timeMode will be enforced to eventTime. This enforces that timers etc emit rows based on eventTime and a proper time flow is maintained. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589359837


##########
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:
   Fixed. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589450965


##########
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:
   Fixed to calculate total count. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591312685


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

Review Comment:
   Makes sense. Added the eventTime value of the row that failed the check. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513592033


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       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,
+      timeoutMode,
+      outputMode,
+      child = logicalPlan
+    )
+
+    val twsDS = Dataset[U](
+      sparkSession,
+      transformWithState
+    )
+
+    val delay = existingWatermarkDelay.head
+
+    Dataset[U](sparkSession, EliminateEventTimeWatermark(
+      UpdateEventTimeWatermarkColumn(

Review Comment:
   How does this modify the plan exactly ? if we have 2 or more `TransformWithState` operators chained together, how would this work exactly ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578608219


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,27 @@ class IncrementalExecution(
           eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
         )
 
+      case UpdateEventTimeColumnExec(eventTime, delay, _,
+        SerializeFromObjectExec(serializer,
+        TransformWithStateExec(keyDeserializer, valueDeserializer, groupingAttributes,
+        dataAttributes, statefulProcessor, timeMode, outputMode,
+        keyEncoder, outputAttr, stateInfo, batchTimestampMs, _,
+        _, child, isStreaming, hasInitialState,
+        initialStateGroupingAttrs, initialStateDataAttrs,
+        initialStateDeserializer, initialState))) if stateInfo.isDefined =>
+
+        val eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(stateInfo.get)

Review Comment:
   yeah. 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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576698149


##########
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:
   Could we consolidate this code with the one above using a private function ? i guess the only diff here is that we are passing the `initalState` dataset ?



##########
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:
   Could we consolidate this code with the one above using a private function ? i guess the only diff here is that we are passing the `initialState` dataset ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589380529


##########
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 meant to say that TransformWithState operator itself passes all the inputRows to the StatefulProcessor function, and does not buffer any input data. 
   
   I can see that it can be confusing when we implement other functionality (like SessionWindow) on top of TWS, updated the doc as suggested. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1575062777


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -129,3 +129,37 @@ case class EventTimeWatermarkExec(
   override protected def withNewChildInternal(newChild: SparkPlan): EventTimeWatermarkExec =
     copy(child = newChild)
 }
+
+case class UpdateEventTimeColumnExec(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    child: SparkPlan) extends UnaryExecNode {
+
+  val delayMs = EventTimeWatermark.getDelayMs(delay)
+  override protected def doExecute(): RDD[InternalRow] = {
+    child.execute()
+  }
+
+  // Update the metadata on the eventTime column to include the desired delay.
+  override val output: Seq[Attribute] = child.output.map { a =>

Review Comment:
   Done. Consolidated the Exec and logical operator code duplication as well. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591716219


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUpdateEventTimeWatermarkColumn.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, UpdateEventTimeWatermarkColumn}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * Extracts the watermark delay and adds it to the UpdateEventTimeWatermarkColumn
+ * logical node (if such a node is present). [[UpdateEventTimeWatermarkColumn]] node updates
+ * the eventTimeColumn for upstream operators.
+ *
+ * If the logical plan contains a [[UpdateEventTimeWatermarkColumn]] node, but no watermark
+ * has been defined, the query will fail with a compilation error.
+ */
+object ResolveUpdateEventTimeWatermarkColumn extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
+    case u: UpdateEventTimeWatermarkColumn =>

Review Comment:
   Also probably a good practice to make the rule be matched once and no longer be matched if it were matched. e.g. if we set the delay in the rule, check the delay in the condition



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513595185


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }

Review Comment:
   Lets also add a test with some interaction with state variables too ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513586387


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -129,3 +129,37 @@ case class EventTimeWatermarkExec(
   override protected def withNewChildInternal(newChild: SparkPlan): EventTimeWatermarkExec =
     copy(child = newChild)
 }
+
+case class UpdateEventTimeColumnExec(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    child: SparkPlan) extends UnaryExecNode {
+
+  val delayMs = EventTimeWatermark.getDelayMs(delay)
+  override protected def doExecute(): RDD[InternalRow] = {
+    child.execute()
+  }
+
+  // Update the metadata on the eventTime column to include the desired delay.
+  override val output: Seq[Attribute] = child.output.map { a =>

Review Comment:
   Same here. We could probably reuse the functions ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576447215


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       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,
+      timeoutMode,
+      outputMode,
+      child = logicalPlan
+    )
+
+    val twsDS = Dataset[U](
+      sparkSession,
+      transformWithState
+    )
+
+    val delay = existingWatermarkDelay.head
+
+    Dataset[U](sparkSession, EliminateEventTimeWatermark(
+      UpdateEventTimeWatermarkColumn(

Review Comment:
   This will add a new node `UpdateEventTimeColumn` after the `TransformWithState` node. The `UpdateEventTimeColumn` will update the column metadata and tag the new column as eventTime. 
   
   Same thing will happen if we have 2 or more `TransformWithState` operators chained together. After each TWS operator, we will add the `UpdateEventTimeColumn` node to tag the correct event time column. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1577018044


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }

Review Comment:
   I dont think we need specifically state variables, as we only deal with the event time of output generated. Let me know if you think otherwise. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578228888


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+            TimeMode.None(),
+            "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(),
+          TimeMode.None(),

Review Comment:
   Could we also add a test for `TimeMode.None` and `TimeMode.ProcessingTime` with this new API ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1593215731


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -442,6 +442,16 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
       case EventTimeWatermark(columnName, delay, child) =>
         EventTimeWatermarkExec(columnName, delay, planLater(child)) :: Nil
 
+      case UpdateEventTimeWatermarkColumn(columnName, delay, child) =>
+        // we expect watermarkDelay to be resolved before physical planning.
+        if (delay.isEmpty) {
+          // This is a sanity check. We should not reach here as delay is updated during
+          // query plan resolution in [[ResolveUpdateEventTimeWatermarkColumn]] Analyzer rule.
+          throw SparkException.internalError(
+            "You hit a query analyzer bug. Please report your query to Spark user mailing list.")

Review Comment:
   Added more detail to the error. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1592763363


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, 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,

Review Comment:
   Renamed



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, 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,

Review Comment:
   Fixed. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513593565


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "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]
+
+      intercept[AnalysisException] {

Review Comment:
   Can we also assert for exception message ?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "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]
+
+      intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[RenamedInputEventRow](
+          new ColumnRenamedStatefulProcessor(),
+          TimeoutMode.NoTimeouts(),
+          "renamedEventTime",
+          OutputMode.Append())
+      }
+    }
+  }
+
+  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[RenamedInputEventRow](
+          new ColumnRenamedStatefulProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Append())
+        .groupBy(window($"renamedEventTime", "1 minute"))
+        .count()
+
+      intercept[ExtendedAnalysisException] {

Review Comment:
   same here 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513587938


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite extends StreamTest

Review Comment:
   Can we rename this to `TransformWithStateChainingSuite` ?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite extends StreamTest

Review Comment:
   Can we rename this to `TransformWithStateChainingSuite` or something equivalent ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513590003


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       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,
+      timeoutMode,
+      outputMode,
+      child = logicalPlan
+    )
+
+    val twsDS = Dataset[U](
+      sparkSession,
+      transformWithState
+    )
+
+    val delay = existingWatermarkDelay.head
+
+    Dataset[U](sparkSession, EliminateEventTimeWatermark(
+      UpdateEventTimeWatermarkColumn(
+        UnresolvedAttribute(eventTimeColumnName),

Review Comment:
   How do we check that this output attribute name is indeed part of the output generated by `TransformWithState` ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513594533


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "1 minute"))

Review Comment:
   Can we add few more tests ? maybe one with `transformWithState` followed by `dropDuplicates`, `join` and `transformWithState` too ?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "1 minute"))

Review Comment:
   Can we add few more tests ? maybe one each with `transformWithState` followed by `dropDuplicates`, `join` and `transformWithState` too ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513626298


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -125,6 +125,12 @@
     ],
     "sqlState" : "428FR"
   },
+  "CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK" : {
+    "message" : [
+      "Watermark needs to be defined to reassign event time column. Could not find watermark definition in the streaming query."

Review Comment:
   Fixed. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576757237


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -78,15 +78,20 @@ case class TransformWithStateExec(
   override def shortName: String = "transformWithStateExec"
 
   override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
-    timeMode match {
-      case ProcessingTime =>
-        // TODO: check if we can return true only if actual timers are registered, or there is
-        // expired state
-        true
-      case EventTime =>
-        eventTimeWatermarkForEviction.isDefined &&
-          newInputWatermark > eventTimeWatermarkForEviction.get
-      case _ => false
+    if (outputMode == OutputMode.Append || outputMode == OutputMode.Update) {

Review Comment:
   This condition should always be true for timeMode set to ProcessingTime. I have refactored this condition. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1575038278


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       eventTimeColumnName: String,
+       outputMode: OutputMode): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.flatMap {

Review Comment:
   Its equivalent, but I still replaced it. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1592763254


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, None,
+        SerializeFromObjectExec(serializer,
+        t: TransformWithStateExec)) if t.stateInfo.isDefined =>
+
+        val stateInfo = t.stateInfo.get
+        val eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(stateInfo)
+        val eventTimeWatermarkForEviction = inputWatermarkForLateEvents(stateInfo)

Review Comment:
   Fixed. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWithState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on PR #45376:
URL: https://github.com/apache/spark/pull/45376#issuecomment-2100800925

   > Let's be sure to either 1) introduce a method to users which gives a watermark value before advancing (late events) or 2) construct a story for users to set the event time timestamp properly without watermark value. @sahnib Could you please file a JIRA ticket with blocker priority?
   
   @HeartSaVioR  Created the JIRA https://issues.apache.org/jira/browse/SPARK-48199 for follow up item. 


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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1524024974


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -72,3 +74,32 @@ case class EventTimeWatermark(
   override protected def withNewChildInternal(newChild: LogicalPlan): EventTimeWatermark =
     copy(child = newChild)
 }
+
+case class UpdateEventTimeWatermarkColumn(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output.map { a =>

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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1522472538


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -40,7 +41,8 @@ object EventTimeWatermark {
 case class EventTimeWatermark(
     eventTime: Attribute,
     delay: CalendarInterval,
-    child: LogicalPlan) extends UnaryNode {
+    child: LogicalPlan) extends UnaryNode
+    with Logging {

Review Comment:
   I had added it for some logging information. Removed it. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589375804


##########
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:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1589383634


##########
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:
   Thanks. Updated. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578228218


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),

Review Comment:
   Don't we need `TimeMode.EventTime()` in all these places ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591717075


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveUpdateEventTimeWatermarkColumn.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, UpdateEventTimeWatermarkColumn}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * Extracts the watermark delay and adds it to the UpdateEventTimeWatermarkColumn
+ * logical node (if such a node is present). [[UpdateEventTimeWatermarkColumn]] node updates
+ * the eventTimeColumn for upstream operators.
+ *
+ * If the logical plan contains a [[UpdateEventTimeWatermarkColumn]] node, but no watermark
+ * has been defined, the query will fail with a compilation error.
+ */
+object ResolveUpdateEventTimeWatermarkColumn extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {

Review Comment:
   Please use the version of pruning, and add pattern for this operator and check with containsPattern.
   
   For example, here's the code snippet from TimeWindowing.
   
   ```
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning(
       _.containsPattern(TIME_WINDOW), ruleId) {
   ```



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1592764603


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala:
##########
@@ -107,25 +109,70 @@ 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] { 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 row = dataIterator.next()
+          if (predicate.eval(row)) {
+            // child node emitted a row which is older than current watermark
+            // this is not allowed
+            val boundEventTimeExpression = bindReference[Expression](eventTime, child.output)
+            val eventTimeProjection = UnsafeProjection.create(boundEventTimeExpression)
+            val rowEventTime = eventTimeProjection(row)
+            throw QueryExecutionErrors.emittedRowsAreOlderThanWatermark(
+              eventTimeWatermarkForEviction.get, rowEventTime.getLong(0))
+          }
+          row
+        }
+      }
+    }
+  }
+
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513585434


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       eventTimeColumnName: String,
+       outputMode: OutputMode): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.flatMap {
+      case EventTimeWatermark(_, delay, _) => Seq(delay)
+      case _ => Seq()
+    }
+
+    if (existingWatermarkDelay.isEmpty) {
+      throw QueryCompilationErrors.cannotAssignEventTimeColumn()

Review Comment:
   Do we need additional checks in the `UnsupportedOperationChecker` too ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513577295


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala:
##########
@@ -579,7 +579,7 @@ object TransformWithState {
       child: LogicalPlan): LogicalPlan = {
     val keyEncoder = encoderFor[K]
     val mapped = new TransformWithState(
-      UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes),
+      UnresolvedDeserializer(keyEncoder.deserializer, groupingAttributes),

Review Comment:
   nice catch !



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](

Review Comment:
   Can we please add the function comment similar to ones above ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576691015


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -762,6 +890,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
       timeMode: TimeMode,
       outputMode: OutputMode,
       initialState: KeyValueGroupedDataset[K, S],
+      eventTimeColumnName: String,

Review Comment:
   Don't we need a separate Java API for this ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576701553


##########
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:
   What does this do exactly ? Could we add a comment just to clarify for the reader ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576692106


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -762,6 +890,7 @@ class KeyValueGroupedDataset[K, V] private[sql](
       timeMode: TimeMode,
       outputMode: OutputMode,
       initialState: KeyValueGroupedDataset[K, S],
+      eventTimeColumnName: String,
       outputEncoder: Encoder[U],
       initialStateEncoder: Encoder[S]): Dataset[U] = {
     transformWithState(statefulProcessor, timeMode,

Review Comment:
   We don't use `eventTimeColumnName` here ?



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1578597899


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+          TimeMode.None(),
+          "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(),
+            TimeMode.None(),
+            "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(),
+          TimeMode.None(),

Review Comment:
   Not needed anymore as TimeMode.EventTime is enforced now for this API. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591767311


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -347,6 +347,28 @@ 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, 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,

Review Comment:
   Note that this would break a test.



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #45376:
URL: https://github.com/apache/spark/pull/45376#issuecomment-2099511072

   Let's be sure to either 1) introduce a method to users which gives a watermark value before advancing (late events) or 2) construct a story for users to set the event time timestamp properly without watermark value. 
   @sahnib Could you please file a JIRA ticket with blocker priority?


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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWithState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #45376: [SPARK-47960][SS] Allow chaining other stateful operators after transformWithState operator.
URL: https://github.com/apache/spark/pull/45376


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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513581167


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -40,7 +41,8 @@ object EventTimeWatermark {
 case class EventTimeWatermark(
     eventTime: Attribute,
     delay: CalendarInterval,
-    child: LogicalPlan) extends UnaryNode {
+    child: LogicalPlan) extends UnaryNode
+    with Logging {

Review Comment:
   Are we using this anywhere ?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -72,3 +74,32 @@ case class EventTimeWatermark(
   override protected def withNewChildInternal(newChild: LogicalPlan): EventTimeWatermark =
     copy(child = newChild)
 }
+
+case class UpdateEventTimeWatermarkColumn(
+    eventTime: Attribute,
+    delay: CalendarInterval,
+    child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output.map { a =>

Review Comment:
   Function is effectively same as the `EventTimeWatermark` case ? can we just have them share the same private function ?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala:
##########
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical
 
 import java.util.concurrent.TimeUnit
 
+import org.apache.spark.internal.Logging

Review Comment:
   Same here



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1513584948


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       eventTimeColumnName: String,
+       outputMode: OutputMode): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.flatMap {

Review Comment:
   Don't we need to do `logicalPlan.collect` ? is this equivalent basically ?



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1522472199


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1575039582


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -676,6 +678,43 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  private[sql] def transformWithState[U: Encoder](
+       statefulProcessor: StatefulProcessor[K, V, U],
+       timeoutMode: TimeoutMode,
+       eventTimeColumnName: String,
+       outputMode: OutputMode): Dataset[U] = {
+    val existingWatermarkDelay = logicalPlan.flatMap {
+      case EventTimeWatermark(_, delay, _) => Seq(delay)
+      case _ => Seq()
+    }
+
+    if (existingWatermarkDelay.isEmpty) {
+      throw QueryCompilationErrors.cannotAssignEventTimeColumn()

Review Comment:
   I dont think we do. `UnsupportedOperationChecker` will throw an error if there are stateful operators chained after transformWithState, but user does not provide the eventTimeColumnName in tws. 



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

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

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576458950


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "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]
+
+      intercept[AnalysisException] {
+        inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState[RenamedInputEventRow](
+          new ColumnRenamedStatefulProcessor(),
+          TimeoutMode.NoTimeouts(),
+          "renamedEventTime",
+          OutputMode.Append())
+      }
+    }
+  }
+
+  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[RenamedInputEventRow](
+          new ColumnRenamedStatefulProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Append())
+        .groupBy(window($"renamedEventTime", "1 minute"))
+        .count()
+
+      intercept[ExtendedAnalysisException] {

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576456955


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateWatermarkSuite.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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
+
+class ColumnRenamedStatefulProcessor
+  extends StatefulProcessor[String, InputEventRow, RenamedInputEventRow]
+  with Logging {
+
+  override def init(outputMode: OutputMode): Unit = { }
+
+  override def handleInputRows(key: String, inputRows: Iterator[InputEventRow],
+      timerValues: TimerValues): Iterator[RenamedInputEventRow] = {
+
+    new Iterator[RenamedInputEventRow] {
+      override def hasNext: Boolean = inputRows.hasNext
+
+      override def next(): RenamedInputEventRow = {
+        Option(inputRows.next()).map { r =>
+          RenamedInputEventRow(
+            r.key, r.eventTime, r.event
+          )
+        }.orNull
+      }
+    }
+
+  }
+
+  override def close(): Unit = { }
+}
+
+case class InputEventRow(
+    key: String,
+    eventTime: Timestamp,
+    event: String)
+
+case class RenamedInputEventRow(
+    key: String,
+    renamedEventTime: Timestamp,
+    event: String)
+
+case class OutputEventRow(
+    key: String,
+    count: Int)
+
+case class Window(
+    start: Timestamp,
+    end: Timestamp)
+
+case class AggEventRow(
+    window: Window,
+    count: Long)
+
+class TransformWithStateWatermarkSuite 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) {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+        val inputData = MemoryStream[InputEventRow]
+
+        val result = inputData.toDS()
+          .withWatermark("eventTime", "1 minute")
+          .groupByKey(x => x.key)
+          .transformWithState[RenamedInputEventRow](
+            new ColumnRenamedStatefulProcessor(),
+            TimeoutMode.NoTimeouts(),
+            "renamedEventTime",
+            OutputMode.Append())
+          .groupBy(window($"renamedEventTime", "1 minute"))

Review Comment:
   Was waiting on other changes before adding these tests. Added testcases of `transformWithState` followed by `dropDuplicates` and `transformWithState`. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576754116


##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -739,6 +741,128 @@ class KeyValueGroupedDataset[K, V] private[sql](
     )
   }
 
+  /**
+   * (Scala-specific)

Review Comment:
   Default values for API params have an issue with Scala/Java compatibility. I guess thats why none of the other APIs have default params either. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #45376:
URL: https://github.com/apache/spark/pull/45376#issuecomment-2072985699

   @sahnib - seems like there are still conflicts on the base branch ?


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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1576708999


##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -1057,6 +1063,14 @@
     },
     "sqlState" : "4274K"
   },
+  "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {
+    "message" : [
+      "Previous node emitted rows which had eventTime older than current watermark value <currentWatermark>",

Review Comment:
   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: reviews-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1577007339


##########
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:
   added a private method to consolidate. 



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45376:
URL: https://github.com/apache/spark/pull/45376#discussion_r1591714252


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -331,6 +331,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
       Seq(
         ResolveWithCTE,
         ExtractDistributedSequenceID) ++
+      Seq(ResolveUpdateEventTimeWatermarkColumn) ++

Review Comment:
   It's not guaranteed for all resolutions to happen in one loop. fixedPoint means having iterations of application of the set of rules. That said, we shouldn't still assume that child is resolved, and only make the rule to be effective when child is resolved. I'll left a comment.



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

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

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


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


Re: [PR] [SPARK-47960][SS] Allow chaining other stateful operators after transformWIthState operator. [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #45376:
URL: https://github.com/apache/spark/pull/45376#issuecomment-2099511347

   Let's handle that later. Thanks! Merging to master.


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

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

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


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