rangadi commented on code in PR #39931:
URL: https://github.com/apache/spark/pull/39931#discussion_r1114957687
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala:
##########
@@ -548,17 +549,131 @@ class EventTimeWatermarkSuite extends StreamTest with
BeforeAndAfter with Matche
assert(e.getMessage contains "should not be negative.")
}
- test("the new watermark should override the old one") {
- val df = MemoryStream[(Long, Long)].toDF()
+ private def buildTestQueryForOverridingWatermark(): (MemoryStream[(Long,
Long)], DataFrame) = {
+ val input = MemoryStream[(Long, Long)]
+ val df = input.toDF()
.withColumn("first", timestamp_seconds($"_1"))
.withColumn("second", timestamp_seconds($"_2"))
.withWatermark("first", "1 minute")
+ .select("*")
.withWatermark("second", "2 minutes")
+ .groupBy(window($"second", "1 minute"))
+ .count()
- val eventTimeColumns = df.logicalPlan.output
- .filter(_.metadata.contains(EventTimeWatermark.delayKey))
- assert(eventTimeColumns.size === 1)
- assert(eventTimeColumns(0).name === "second")
+ (input, df)
+ }
+
+ test("overriding watermark should not be allowed by default") {
+ val (input, df) = buildTestQueryForOverridingWatermark()
+ testStream(df)(
+ AddData(input, (100L, 200L)),
+ ExpectFailure[AnalysisException](assertFailure = exc => {
+ assert(exc.getMessage.contains("Redefining watermark is disallowed."))
+
assert(exc.getMessage.contains(SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE.key))
+ })
+ )
+ }
+
+ test("overriding watermark should not fail in compatibility mode") {
+ withSQLConf(SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE.key -> "false") {
+ val (input, df) = buildTestQueryForOverridingWatermark()
+ testStream(df)(
+ AddData(input, (100L, 200L)),
+ CheckAnswer(),
+ Execute { query =>
+ val lastExecution = query.lastExecution
+ val aggSaveOperator = lastExecution.executedPlan.collect {
+ case j: StateStoreSaveExec => j
+ }.head
+
+ // - watermark from first definition = 100 - 60 = 40
+ // - watermark from second definition = 200 - 120 = 80
+ // - global watermark = min(40, 60) = 40
+ //
+ // As we see the result, even though we override the watermark
definition, the old
+ // definition of watermark still plays to calculate global watermark.
+ //
+ // This is conceptually the right behavior. For operators after the
first watermark
+ // definition, the column named "first" is considered as event time
column, and for
+ // operators after the second watermark definition, the column named
"second" is
+ // considered as event time column. The correct "single" value of
watermark satisfying
+ // all operators should be lower bound of both columns "first" and
"second".
+ //
+ // That said, this easily leads to incorrect definition - e.g.
re-define watermark
+ // against the output of streaming aggregation for append mode. The
global watermark
+ // cannot advance. This is the reason we don't allow re-define
watermark in new behavior.
+ val expectedWatermarkMs = 40 * 1000
+
+ assert(aggSaveOperator.eventTimeWatermarkForLateEvents ===
Some(expectedWatermarkMs))
+ assert(aggSaveOperator.eventTimeWatermarkForEviction ===
Some(expectedWatermarkMs))
+
+ val eventTimeCols = aggSaveOperator.keyExpressions.filter(
+ _.metadata.contains(EventTimeWatermark.delayKey))
+ assert(eventTimeCols.size === 1)
+ assert(eventTimeCols.head.name === "window")
+ // 2 minutes delay threshold
+
assert(eventTimeCols.head.metadata.getLong(EventTimeWatermark.delayKey) === 120
* 1000)
+ }
+ )
+ }
+ }
+
+ private def buildTestQueryForMultiEventTimeColumns()
+ : (MemoryStream[(String, Long)], MemoryStream[(String, Long)], DataFrame)
= {
+ val input1 = MemoryStream[(String, Long)]
+ val input2 = MemoryStream[(String, Long)]
+ val df1 = input1.toDF()
+ .selectExpr("_1 AS id1", "timestamp_seconds(_2) AS ts1")
+ .withWatermark("ts1", "1 minute")
+
+ val df2 = input2.toDF()
+ .selectExpr("_1 AS id2", "timestamp_seconds(_2) AS ts2")
+ .withWatermark("ts2", "2 minutes")
+
+ val joined = df1.join(df2, expr("id1 = id2 AND ts1 = ts2 + INTERVAL 10
SECONDS"), "inner")
+ .selectExpr("id1", "ts1", "ts2")
+ // the output of join contains both ts1 and ts2
+ val dedup = joined.dropDuplicates()
+ .selectExpr("id1", "CAST(ts1 AS LONG) AS ts1", "CAST(ts2 AS LONG) AS
ts2")
+
+ (input1, input2, dedup)
+ }
+
+ test("multiple event time columns in an input DataFrame for stateful
operator is " +
Review Comment:
How do we make this query work? Looks like correct query.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -463,6 +437,442 @@ class MultiStatefulOperatorsSuite
)
}
+ test("stream-stream time interval left outer join -> aggregation, append
mode") {
+ val input1 = MemoryStream[(String, Timestamp)]
+ val input2 = MemoryStream[(String, Timestamp)]
+
+ val s1 = input1.toDF()
+ .selectExpr("_1 AS id1", "_2 AS timestamp1")
Review Comment:
Minor: I think `.toDF("id1", "timestamp1")` is equivalent to this.
##########
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:
Yes. Just to help the readers.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -463,6 +437,442 @@ class MultiStatefulOperatorsSuite
)
}
+ test("stream-stream time interval left outer join -> aggregation, append
mode") {
+ val input1 = MemoryStream[(String, Timestamp)]
+ val input2 = MemoryStream[(String, Timestamp)]
+
+ val s1 = input1.toDF()
+ .selectExpr("_1 AS id1", "_2 AS timestamp1")
+ .withWatermark("timestamp1", "0 seconds")
+ .as("s1")
+
+ val s2 = input2.toDF()
+ .selectExpr("_1 AS id2", "_2 AS timestamp2")
+ .withWatermark("timestamp2", "0 seconds")
+ .as("s2")
+
+ val s3 = s1.join(s2, expr("s1.id1 = s2.id2 AND (s1.timestamp1 BETWEEN " +
+ "s2.timestamp2 - INTERVAL 1 hour AND s2.timestamp2 + INTERVAL 1 hour)"),
"leftOuter")
+
+ val agg = s3.groupBy(window($"timestamp1", "10 minutes"))
Review Comment:
How is this different from `join() followed deDup()` test in
EventTimeWatermarkSuite.scala? The latter was not allowed.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -463,6 +437,442 @@ class MultiStatefulOperatorsSuite
)
}
+ test("stream-stream time interval left outer join -> aggregation, append
mode") {
+ val input1 = MemoryStream[(String, Timestamp)]
+ val input2 = MemoryStream[(String, Timestamp)]
+
+ val s1 = input1.toDF()
+ .selectExpr("_1 AS id1", "_2 AS timestamp1")
+ .withWatermark("timestamp1", "0 seconds")
+ .as("s1")
+
+ val s2 = input2.toDF()
+ .selectExpr("_1 AS id2", "_2 AS timestamp2")
+ .withWatermark("timestamp2", "0 seconds")
+ .as("s2")
+
+ val s3 = s1.join(s2, expr("s1.id1 = s2.id2 AND (s1.timestamp1 BETWEEN " +
+ "s2.timestamp2 - INTERVAL 1 hour AND s2.timestamp2 + INTERVAL 1 hour)"),
"leftOuter")
+
+ val agg = s3.groupBy(window($"timestamp1", "10 minutes"))
+ .agg(count("*").as("cnt"))
+ .selectExpr("CAST(window.start AS STRING) AS window_start",
+ "CAST(window.end AS STRING) AS window_end", "cnt")
+
+ // for ease of verification, we change the session timezone to UTC
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
+ testStream(agg)(
+ MultiAddData(
+ (input1, Seq(
+ ("1", Timestamp.valueOf("2023-01-01 01:00:10")),
+ ("2", Timestamp.valueOf("2023-01-01 01:00:30")))
+ ),
+ (input2, Seq(
+ ("1", Timestamp.valueOf("2023-01-01 01:00:20"))))
+ ),
+
+ // < data batch >
+ // global watermark (0, 0)
+ // op1 (join)
+ // -- IW (0, 0)
+ // -- OW 0
+ // -- left state
+ // ("1", "2023-01-01 01:00:10", matched=true)
+ // ("1", "2023-01-01 01:00:30", matched=false)
+ // -- right state
+ // ("1", "2023-01-01 01:00:20")
+ // -- result
+ // ("1", "2023-01-01 01:00:10", "1", "2023-01-01 01:00:20")
+ // op2 (aggregation)
+ // -- IW (0, 0)
+ // -- OW 0
+ // -- state row
+ // ("2023-01-01 01:00:00", "2023-01-01 01:10:00", 1)
+ // -- result
+ // None
+
+ // -- watermark calculation
+ // watermark in left input: 2023-01-01 01:00:30
+ // watermark in right input: 2023-01-01 01:00:20
+ // origin watermark: 2023-01-01 01:00:20
+
+ // < no-data batch >
+ // global watermark (0, 2023-01-01 01:00:20)
+ // op1 (join)
+ // -- IW (0, 2023-01-01 01:00:20)
+ // -- OW 2023-01-01 00:00:19.999999
+ // -- left state
+ // ("1", "2023-01-01 01:00:10", matched=true)
+ // ("1", "2023-01-01 01:00:30", matched=false)
+ // -- right state
+ // ("1", "2023-01-01 01:00:20")
+ // -- result
+ // None
+ // op2 (aggregation)
+ // -- IW (0, 2023-01-01 00:00:19.999999)
+ // -- OW 2023-01-01 00:00:19.999999
+ // -- state row
+ // ("2023-01-01 01:00:00", "2023-01-01 01:10:00", 1)
+ // -- result
+ // None
+ CheckAnswer(),
+
+ Execute { query =>
+ val lastExecution = query.lastExecution
+ val joinOperator = lastExecution.executedPlan.collect {
+ case j: StreamingSymmetricHashJoinExec => j
+ }.head
+ val aggSaveOperator = lastExecution.executedPlan.collect {
+ case j: StateStoreSaveExec => j
+ }.head
+
+ assert(joinOperator.eventTimeWatermarkForLateEvents === Some(0))
+ assert(joinOperator.eventTimeWatermarkForEviction ===
+ Some(Timestamp.valueOf("2023-01-01 01:00:20").getTime))
+
+ assert(aggSaveOperator.eventTimeWatermarkForLateEvents === Some(0))
+ assert(aggSaveOperator.eventTimeWatermarkForEviction ===
+ Some(Timestamp.valueOf("2023-01-01 00:00:20").getTime - 1))
+ },
+
+ MultiAddData(
+ (input1, Seq(("5", Timestamp.valueOf("2023-01-01 01:15:00")))),
+ (input2, Seq(("6", Timestamp.valueOf("2023-01-01 01:15:00"))))
+ ),
+
+ // < data batch >
Review Comment:
Thanks for these detailed comments in the tests.
##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -463,6 +437,442 @@ class MultiStatefulOperatorsSuite
)
}
+ test("stream-stream time interval left outer join -> aggregation, append
mode") {
Review Comment:
This is a detailed test with great comments inline. It will nice to give a
short overview of this test does at the top here.
##########
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:
[For understanding]: Where is called? want to see if we can figure out how
it is only called only once.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]