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: [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]