You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "rangadi (via GitHub)" <gi...@apache.org> on 2023/02/13 19:29:21 UTC

[GitHub] [spark] rangadi commented on a diff in pull request #39931: [SPARK-42376][SS] Introduce watermark propagation among operators

rangadi commented on code in PR #39931:
URL: https://github.com/apache/spark/pull/39931#discussion_r1104859860


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala:
##########
@@ -699,4 +701,18 @@ case class StreamingSymmetricHashJoinExec(
     } else {
       Nil
     }
+
+  // This operator will evict based on the state watermark on both side of inputs; we would like
+  // to let users leverage both sides of event time column for output of join, so the watermark
+  // must be lower bound of both sides of event time column. The lower bound of event time column
+  // for each side is determined by state watermark, hence we take a minimum of (left state
+  // watermark, right state watermark, input watermark) to decide the output watermark.
+  override def produceWatermark(inputWatermarkMs: Long): Long = {
+    val (leftStateWatermark, rightStateWatermark) =
+      StreamingSymmetricHashJoinHelper.getStateWatermark(
+        left.output, right.output, leftKeys, rightKeys, condition.full, Some(inputWatermarkMs))
+
+    (Seq(leftStateWatermark, rightStateWatermark).filter(_.isDefined).map(_.get) ++

Review Comment:
   Simpler to write: `(leftStateWatermark ++ leftStateWatermark ++ Some(inputWatermarkMs)).min`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -216,52 +219,119 @@ class IncrementalExecution(
           keys,
           child,
           Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None)
 
       case m: FlatMapGroupsWithStateExec =>
         // We set this to true only for the first batch of the streaming query.
         val hasInitialState = (currentBatchId == 0L && m.hasInitialState)
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None,
           hasInitialState = hasInitialState
         )
 
       case m: FlatMapGroupsInPandasWithStateExec =>
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction)
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
         )
 
       case j: StreamingSymmetricHashJoinExec =>
         j.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
-          stateWatermarkPredicates =
-            StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
-              j.left.output, j.right.output, j.leftKeys, j.rightKeys, j.condition.full,
-              Some(eventTimeWatermarkForEviction)))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
+        )
 
       case l: StreamingGlobalLimitExec =>
         l.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           outputMode = Some(outputMode))
+    }
+  }
 
-      case StreamingLocalLimitExec(limit, child) if hasNoStatefulOp(child) =>
-        // Optimize limit execution by replacing StreamingLocalLimitExec (consumes the iterator
-        // completely) to LocalLimitExec (does not consume the iterator) when the child plan has
-        // no stateful operator (i.e., consuming the iterator is not needed).
-        LocalLimitExec(limit, child)
+  val watermarkPropagationRule = new Rule[SparkPlan] {
+    private def simulateWatermarkPropagation(plan: SparkPlan): Unit = {
+      val watermarkForPrevBatch = prevOffsetSeqMetadata.map(_.batchWatermarkMs).getOrElse(0L)
+      val watermarkForCurrBatch = offsetSeqMetadata.batchWatermarkMs
+
+      // This is to simulate watermark propagation for late events.
+      watermarkPropagator.propagate(currentBatchId - 1, plan, watermarkForPrevBatch)
+      // This is to simulate watermark propagation for eviction.
+      watermarkPropagator.propagate(currentBatchId, plan, watermarkForCurrBatch)
+    }
+
+    private def inputWatermarkForLateEvents(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForLateEvents(currentBatchId,
+        stateInfo.operatorId))
+    }
+
+    private def inputWatermarkForEviction(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForEviction(currentBatchId, stateInfo.operatorId))
+    }
+
+    override def apply(plan: SparkPlan): SparkPlan = {
+      simulateWatermarkPropagation(plan)
+      plan transform {
+        case s: StateStoreSaveExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: SessionWindowStateStoreSaveExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: SessionWindowStateStoreRestoreExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: StreamingDeduplicateExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case m: FlatMapGroupsWithStateExec if m.stateInfo.isDefined =>
+          m.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(m.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
+          )
+
+        case m: FlatMapGroupsInPandasWithStateExec if m.stateInfo.isDefined =>
+          m.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(m.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
+          )
+
+        case j: StreamingSymmetricHashJoinExec =>
+          val iwLateEvents = inputWatermarkForLateEvents(j.stateInfo.get)
+          val iwEviction = inputWatermarkForEviction(j.stateInfo.get)
+          j.copy(
+            eventTimeWatermarkForLateEvents = iwLateEvents,
+            eventTimeWatermarkForEviction = iwEviction,
+            stateWatermarkPredicates =
+              StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
+                j.left.output, j.right.output, j.leftKeys, j.rightKeys, j.condition.full,
+                iwEviction)
+          )
+      }
     }
   }
 
-  override def preparations: Seq[Rule[SparkPlan]] = state +: super.preparations
+  override def preparations: Seq[Rule[SparkPlan]] = Seq(
+    shufflePartitionsRule, convertLocalLimitRule, stateOpIdRule, watermarkPropagationRule

Review Comment:
   Minor: Could you move these to one per line? Since the order matters, it is more readable to see the order. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation

Review Comment:
   In fact this will be called for every node in the plan, right (stateful or not)?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -216,52 +219,119 @@ class IncrementalExecution(
           keys,
           child,
           Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None)
 
       case m: FlatMapGroupsWithStateExec =>
         // We set this to true only for the first batch of the streaming query.
         val hasInitialState = (currentBatchId == 0L && m.hasInitialState)
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None,
           hasInitialState = hasInitialState
         )
 
       case m: FlatMapGroupsInPandasWithStateExec =>
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction)
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
         )
 
       case j: StreamingSymmetricHashJoinExec =>
         j.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
-          stateWatermarkPredicates =
-            StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
-              j.left.output, j.right.output, j.leftKeys, j.rightKeys, j.condition.full,
-              Some(eventTimeWatermarkForEviction)))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
+        )
 
       case l: StreamingGlobalLimitExec =>
         l.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           outputMode = Some(outputMode))
+    }
+  }
 
-      case StreamingLocalLimitExec(limit, child) if hasNoStatefulOp(child) =>
-        // Optimize limit execution by replacing StreamingLocalLimitExec (consumes the iterator
-        // completely) to LocalLimitExec (does not consume the iterator) when the child plan has
-        // no stateful operator (i.e., consuming the iterator is not needed).
-        LocalLimitExec(limit, child)
+  val watermarkPropagationRule = new Rule[SparkPlan] {
+    private def simulateWatermarkPropagation(plan: SparkPlan): Unit = {
+      val watermarkForPrevBatch = prevOffsetSeqMetadata.map(_.batchWatermarkMs).getOrElse(0L)
+      val watermarkForCurrBatch = offsetSeqMetadata.batchWatermarkMs
+
+      // This is to simulate watermark propagation for late events.
+      watermarkPropagator.propagate(currentBatchId - 1, plan, watermarkForPrevBatch)
+      // This is to simulate watermark propagation for eviction.
+      watermarkPropagator.propagate(currentBatchId, plan, watermarkForCurrBatch)
+    }
+
+    private def inputWatermarkForLateEvents(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForLateEvents(currentBatchId,
+        stateInfo.operatorId))
+    }
+
+    private def inputWatermarkForEviction(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForEviction(currentBatchId, stateInfo.operatorId))
+    }
+
+    override def apply(plan: SparkPlan): SparkPlan = {
+      simulateWatermarkPropagation(plan)

Review Comment:
   How about moving this into case statements below? Or move it into `inputWatermarkForLateEvents()` and  `inputWatermarkForEviction()`? That way it will only be called for stateful operators. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceWatermark(input watermark)`.
+ *      @see [[StateStoreWriter.produceWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Long]] =
+    mutable.Map[Long, Map[Long, Long]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  private def getInputWatermarks(
+      node: SparkPlan,
+      nodeToOutputWatermark: mutable.Map[Int, Long]): Seq[Long] = {
+    node.children.map { child =>
+      nodeToOutputWatermark.getOrElse(child.id, {
+        throw new IllegalStateException(
+          s"watermark for the node ${child.id} should be registered")
+      })
+    }.filter { case curr =>
+      // This path is to exclude children from watermark calculation
+      // which don't have watermark information
+      curr != DEFAULT_WATERMARK_MS
+    }
+  }
+
+  private def doSimulate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    val statefulOperatorIdToNodeId = mutable.HashMap[Long, Int]()
+    val nodeToOutputWatermark = mutable.HashMap[Int, Long]()
+    val nextStatefulOperatorToWatermark = mutable.HashMap[Long, Long]()
+
+    // This calculation relies on post-order traversal of the query plan.
+    plan.transformUp {
+      case node: EventTimeWatermarkExec =>
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        if (inputWatermarks.nonEmpty) {
+          throw new AnalysisException("Redefining watermark is disallowed. You can set the " +
+            s"config '${SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE.key}' to 'false' to restore " +
+            "the previous behavior. Note that multiple stateful operators will be disallowed.")
+        }
+
+        nodeToOutputWatermark.put(node.id, originWatermark)
+        node
+
+      case node: StateStoreWriter =>
+        val stOpId = node.stateInfo.get.operatorId
+        statefulOperatorIdToNodeId.put(stOpId, node.id)
+
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        val finalInputWatermarkMs = if (inputWatermarks.nonEmpty) {
+          inputWatermarks.min
+        } else {
+          // We can't throw exception here, as we allow stateful operator to process without
+          // watermark. E.g. streaming aggregation with update/complete mode.
+          DEFAULT_WATERMARK_MS
+        }
+
+        val outputWatermarkMs = node.produceWatermark(finalInputWatermarkMs)
+        nodeToOutputWatermark.put(node.id, outputWatermarkMs)
+        nextStatefulOperatorToWatermark.put(stOpId, finalInputWatermarkMs)
+        node
+
+      case node =>
+        // pass-through, but also consider multiple children like the case of union
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        val finalInputWatermarkMs = if (inputWatermarks.nonEmpty) {
+          val minCurrInputWatermarkMs = inputWatermarks.min
+          minCurrInputWatermarkMs
+        } else {
+          DEFAULT_WATERMARK_MS
+        }
+
+        nodeToOutputWatermark.put(node.id, finalInputWatermarkMs)
+        node
+    }
+
+    inputWatermarks.put(batchId, nextStatefulOperatorToWatermark.toMap)
+    batchIdToWatermark.put(batchId, originWatermark)
+
+    logDebug(s"global watermark for batch ID $batchId is set to $originWatermark")
+    logDebug(s"input watermarks for batch ID $batchId is set to $nextStatefulOperatorToWatermark")
+  }
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      logDebug(s"watermark for batch ID $batchId is received as $originWatermark, " +
+        s"call site: ${Utils.getCallSite().longForm}")
+      doSimulate(batchId, plan, originWatermark)

Review Comment:
   Clarification: How does this ensure this is propagated through the entire plan? Does this plan arg here matter? 
    - `populate()` is called for every node
    -  `doSimulate()` is called only once per batch.
    -  Assuming `transformUp()` above only traverses partial tree, how do ensure we traverse the entire tree the first time?
        - Is the sync node the root and and thus we traverse the entire tree in the first iteration?    
    Could you attach the debug log at line 220 and 221 from a unit test? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceWatermark(input watermark)`.
+ *      @see [[StateStoreWriter.produceWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Long]] =

Review Comment:
   What does this contain? Add a comment. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -324,6 +344,33 @@ object WatermarkSupport {
       }
     Some(evictionExpression)
   }
+
+  def findEventTimeColumn(

Review Comment:
   Add method documentation since this is used outside.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala:
##########
@@ -699,4 +701,18 @@ case class StreamingSymmetricHashJoinExec(
     } else {
       Nil
     }
+
+  // This operator will evict based on the state watermark on both side of inputs; we would like
+  // to let users leverage both sides of event time column for output of join, so the watermark
+  // must be lower bound of both sides of event time column. The lower bound of event time column
+  // for each side is determined by state watermark, hence we take a minimum of (left state
+  // watermark, right state watermark, input watermark) to decide the output watermark.
+  override def produceWatermark(inputWatermarkMs: Long): Long = {

Review Comment:
   For better context: What node does this `inputWatermarkMs` correspond to?
    (will check `StreamingSymmetricHashJoinHelper.getStateWatermark()` as well). 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {

Review Comment:
   Better to make it an `object` (it makes it clear). 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceWatermark(input watermark)`.
+ *      @see [[StateStoreWriter.produceWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Long]] =
+    mutable.Map[Long, Map[Long, Long]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  private def getInputWatermarks(

Review Comment:
   Add a comment on what this does. i.e. what does it return?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceWatermark(input watermark)`.
+ *      @see [[StateStoreWriter.produceWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Long]] =
+    mutable.Map[Long, Map[Long, Long]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  private def getInputWatermarks(
+      node: SparkPlan,
+      nodeToOutputWatermark: mutable.Map[Int, Long]): Seq[Long] = {
+    node.children.map { child =>
+      nodeToOutputWatermark.getOrElse(child.id, {
+        throw new IllegalStateException(
+          s"watermark for the node ${child.id} should be registered")
+      })
+    }.filter { case curr =>
+      // This path is to exclude children from watermark calculation
+      // which don't have watermark information
+      curr != DEFAULT_WATERMARK_MS
+    }
+  }
+
+  private def doSimulate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    val statefulOperatorIdToNodeId = mutable.HashMap[Long, Int]()
+    val nodeToOutputWatermark = mutable.HashMap[Int, Long]()
+    val nextStatefulOperatorToWatermark = mutable.HashMap[Long, Long]()
+
+    // This calculation relies on post-order traversal of the query plan.
+    plan.transformUp {

Review Comment:
   Btw, in physical plan, are children executed before the parent? (E.g. in a stream-stream join, are left & right children of join node?)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -96,6 +98,20 @@ trait StateStoreReader extends StatefulOperator {
 /** An operator that writes to a StateStore. */
 trait StateStoreWriter extends StatefulOperator with PythonSQLMetrics { self: SparkPlan =>
 
+  /**
+   * Produce the output watermark for given input watermark (ms).
+   *
+   * In most cases, this is same as the criteria of state eviction, as most stateful operators
+   * produce the output from two different kinds:
+   *
+   * 1. without buffering
+   * 2. with buffering (state)
+   *
+   * The state eviction happens when event time exceeds a "certain threshold of timestamp", which
+   * denotes a lower bound of event time values for output (output watermark).
+   */
+  def produceWatermark(inputWatermarkMs: Long): Long

Review Comment:
   How about renaming it to `produceOutputWatermark()` for clarity? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.WatermarkPropagator.DEFAULT_WATERMARK_MS
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/** Interface for propagating watermark. */
+trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+class NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceWatermark(input watermark)`.
+ *      @see [[StateStoreWriter.produceWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Long]] =
+    mutable.Map[Long, Map[Long, Long]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  private def getInputWatermarks(
+      node: SparkPlan,
+      nodeToOutputWatermark: mutable.Map[Int, Long]): Seq[Long] = {
+    node.children.map { child =>
+      nodeToOutputWatermark.getOrElse(child.id, {
+        throw new IllegalStateException(
+          s"watermark for the node ${child.id} should be registered")
+      })
+    }.filter { case curr =>
+      // This path is to exclude children from watermark calculation
+      // which don't have watermark information
+      curr != DEFAULT_WATERMARK_MS
+    }
+  }
+
+  private def doSimulate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    val statefulOperatorIdToNodeId = mutable.HashMap[Long, Int]()
+    val nodeToOutputWatermark = mutable.HashMap[Int, Long]()
+    val nextStatefulOperatorToWatermark = mutable.HashMap[Long, Long]()
+
+    // This calculation relies on post-order traversal of the query plan.
+    plan.transformUp {
+      case node: EventTimeWatermarkExec =>
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        if (inputWatermarks.nonEmpty) {
+          throw new AnalysisException("Redefining watermark is disallowed. You can set the " +
+            s"config '${SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE.key}' to 'false' to restore " +
+            "the previous behavior. Note that multiple stateful operators will be disallowed.")
+        }
+
+        nodeToOutputWatermark.put(node.id, originWatermark)
+        node
+
+      case node: StateStoreWriter =>
+        val stOpId = node.stateInfo.get.operatorId
+        statefulOperatorIdToNodeId.put(stOpId, node.id)
+
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        val finalInputWatermarkMs = if (inputWatermarks.nonEmpty) {
+          inputWatermarks.min
+        } else {
+          // We can't throw exception here, as we allow stateful operator to process without
+          // watermark. E.g. streaming aggregation with update/complete mode.
+          DEFAULT_WATERMARK_MS
+        }
+
+        val outputWatermarkMs = node.produceWatermark(finalInputWatermarkMs)
+        nodeToOutputWatermark.put(node.id, outputWatermarkMs)
+        nextStatefulOperatorToWatermark.put(stOpId, finalInputWatermarkMs)
+        node
+
+      case node =>
+        // pass-through, but also consider multiple children like the case of union
+        val inputWatermarks = getInputWatermarks(node, nodeToOutputWatermark)
+        val finalInputWatermarkMs = if (inputWatermarks.nonEmpty) {
+          val minCurrInputWatermarkMs = inputWatermarks.min
+          minCurrInputWatermarkMs
+        } else {
+          DEFAULT_WATERMARK_MS
+        }
+
+        nodeToOutputWatermark.put(node.id, finalInputWatermarkMs)
+        node
+    }
+
+    inputWatermarks.put(batchId, nextStatefulOperatorToWatermark.toMap)
+    batchIdToWatermark.put(batchId, originWatermark)
+
+    logDebug(s"global watermark for batch ID $batchId is set to $originWatermark")
+    logDebug(s"input watermarks for batch ID $batchId is set to $nextStatefulOperatorToWatermark")
+  }
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      logDebug(s"watermark for batch ID $batchId is received as $originWatermark, " +
+        s"call site: ${Utils.getCallSite().longForm}")
+      doSimulate(batchId, plan, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      // In current Spark's logic, event time watermark cannot go down to negative. So even there is
+      // no input watermark for operator, the final input watermark for operator should be 0L.
+      val opWatermark = inputWatermarks(batchId).get(stateOpId)

Review Comment:
   Style: use match for better readability (here and other places):
   ```scala
   inputWatermarks(batchId).get(stateOpId) match {
      case Some(wm) => Math.max(0, wm)
      case None => throw IllegalStateException(...)
   }
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -809,6 +860,10 @@ case class SessionWindowStateStoreSaveExec(
       newNumRowsUpdated = stateOpProgress.numRowsUpdated,
       newNumRowsDroppedByWatermark = numRowsDroppedByWatermark)
   }
+
+  // This operator will evict based on min input watermark and ensure it will be minimum of
+  // the event time value for the output so far (including output from eviction).
+  override def produceWatermark(inputWatermarkMs: Long): Long = inputWatermarkMs

Review Comment:
   How about making this the default implementation so that no all stateful operators need to define this default 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