You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HeartSaVioR (via GitHub)" <gi...@apache.org> on 2023/03/27 08:11:48 UTC

[GitHub] [spark] HeartSaVioR opened a new pull request, #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   ### What changes were proposed in this pull request?
   
   This PR proposes to introduce a new API of dropDuplicates which has following different characteristics compared to existing dropDuplicates:
   
   * Weaker constraints on the subset (key)
     * Does not require an event time column on the subset.
   * Looser semantics on deduplication
     * Only guarantee to deduplicate events within the watermark.
   
   Since the new API leverages event time, the new API has following new requirements:
   
   * The input must be streaming DataFrame.
   * The watermark must be defined.
   * The event time column must be defined in the input DataFrame.
   
   More specifically on the semantic, once the operator processes the first arrived event, events arriving within the watermark for the first event will be deduplicated.
   (Technically, the expiration time should be the “event time of the first arrived event + watermark delay threshold”, to match up with future events.)
   
   Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events. (If they are unsure, they can alternatively set the delay threshold large enough, e.g. 48 hours.)
   
   ### Why are the changes needed?
   
   Existing dropDuplicates API does not address the valid use case on streaming query.
   
   There are many cases where the event time is not exact the same, although these events are same. One example is duplicated events are produced due to non-idempotent writer where event time is issued from producer/broker side. Another example is that the value of event time is unstable and users want to use alternative timestamp e.g. ingestion time.
   
   For these case, users have to exclude event time column from subset of deduplication, but then the operator is unable to evict state, leading to indefinitely growing state.
   
   To allow eviction of state while event time column is not required to be a part of subset of deduplication, we need to loose the semantic for the API, which warrants a new API.
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, this introduces a new public API, dropDuplicatesWithinWatermark.
   
   ### How was this patch tested?
   
   New test suite.


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   Thanks for replicating the message :) That said, it seems to be still safe to store underlying value as it is, with LongType.
   
   Maybe users may feel odd if they use TimestampType and TimestampNTZType interchangeably in the same column across multiple microbatches, but it's also an existing case if the query ran from US and stopped and somehow reran from Korea. The value doesn't change, but people may be confused with the representation. Treating this nicely seems to be beyond the scope.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = {

Review Comment:
   Should we maybe have `dropDuplicatesWithinWatermark(cols: String*)` one alone? Uses can invoke `dropDuplicatesWithinWatermark(Seq(...): _*)`. That will automatically create `dropDuplicatesWithinWatermark(colNames: Array[String])` signature too for Java IIRC



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = {

Review Comment:
   Just to clarify, having one method would allow the cases below:
   
   ```python
   def dropDuplicatesWithinWatermark(cols: String*)
   ```
   
   In Scala:
   
   ```scala
   df.dropDuplicatesWithinWatermark(Seq(col1, col2): _*)
   df.dropDuplicatesWithinWatermark(Array(col1, col2): _*)
   df.dropDuplicatesWithinWatermark(col1, col2, ...)
   ```
   
   In Java
   
   ```java
   String[] arr = {co1, col2}
   df.dropDuplicatesWithinWatermark(arr)
   df.dropDuplicatesWithinWatermark(col1, col2, ...)
   ```
   
   The current signatures in the PR:
   
   ```scala
   def dropDuplicatesWithinWatermark(colNames: Seq[String])
   def dropDuplicatesWithinWatermark(colNames: Array[String])
   ```
   
   adds two more Scala use cases:
   
   ```scala
   df.dropDuplicatesWithinWatermark(Seq(col1, col2))
   df.dropDuplicatesWithinWatermark(Array(col1, col2))
   ```
   
   I guess using `:_*` is pretty common given that `Dataset.select` or `Dataset.selectExpr` do not have such variants too.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()
+         +-----+---+------+
+         | name|age|height|
+         +-----+---+------+
+         |Alice|  5|    80|
+         |Alice| 10|    80|
+         +-----+---+------+
+
+         Deduplicate values on 'name' and 'height' columns.
+
+         >>> df.dropDuplicatesWithinWatermark(['name', 'height']).show()

Review Comment:
   Except PySpark specific edge case, we rely on Scala test suite to avoid duplicate every test cases.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/tests/connect/test_parity_dataframe.py:
##########
@@ -41,6 +41,11 @@ def test_observe(self):
     def test_observe_str(self):
         super().test_observe_str()
 
+    # TODO(SPARK-XXXXX): Support Structured Streaming

Review Comment:
   Yeah, I deferred implementing API in Spark connect because we don't support withWatermark now, but we wouldn't need to mention about streaming. We could just say it's not implemented yet.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   batch DataFrame does not require watermark, and we actually remove withWatermark if the query is batch one. If we require batch query to provide watermark, that will be very odd because we ignore the delay threshold in any way.
   
   So I think there are two choices, 1) do not support batch query since it's confusing vs 2) tolerate the difference of UX and just do the same with existing API for batch case. Currently it's 2) but the initial proposal was 1). I'm open for both.
   cc. @rangadi 



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   cc. @zsxwing @viirya @rangadi Please take a look. Thanks in advance!


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   Sigh I didn't indicate we already took a step of Scala API with Spark connect. I thought there's only in PySpark. Thanks for correcting me.


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala:
##########
@@ -1742,6 +1742,8 @@ class DataFrameSuite extends QueryTest
       Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2)))
   }
 
+  // FIXME: add dropDuplicatesWithinWatermark

Review Comment:
   Any reason to post pone it? 



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {

Review Comment:
   without including the event_time column, right? 



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))

Review Comment:
   We might have to use new error framework this error and use `checkError()` here.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)

Review Comment:
   Good to advance the watermark far enough here so that we get 0 entries in the state. 



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {

Review Comment:
   "deduplicate without event time column should result in error"



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),

Review Comment:
   What are these 3? 
   thought only 16 & 17 would be left. 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala:
##########
@@ -464,6 +469,19 @@ object UnsupportedOperationChecker extends Logging {
               throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset")
           }
 
+        case d: DeduplicateWithinWatermark if d.isStreaming =>
+          // Find any attributes that are associated with an eventTime watermark.
+          val watermarkAttributes = d.child.output.collect {
+            case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => a
+          }
+
+          // DeduplicateWithinWatermark requires event time column being set in the input DataFrame
+          if (watermarkAttributes.isEmpty) {
+            throwError(
+              "dropDuplicatesWithinWatermark is not supported on streaming DataFrames/DataSets " +

Review Comment:
   Sure. Your choice.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   Where would it matter for the functionality? 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))

Review Comment:
   Yeah... I'll see whether we converted any of error in UnsupportedOperatorChecker. I understand SS is far behind on the movement of error framework. Probably we should not at least increase the gap.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala:
##########
@@ -1742,6 +1742,8 @@ class DataFrameSuite extends QueryTest
       Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2)))
   }
 
+  // FIXME: add dropDuplicatesWithinWatermark

Review Comment:
   Sorry for confusion. All FIXMEs are remaining tasks I came up yesterday from the decision of supporting batch query. Pretty sure we will address all of them before merging.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   Let's consolidate this to below comment thread. I pinged Gengliang in below comment to understand what is the use case story for TimestampType vs TimestampNTZType, is the type somehow interchangeable, etc. Before that, I'll just roll back the change to match with flatMapGroupsWithState.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   No, probably I have to open a new comment thread as if I roll back the change the below comment thread will also be outdated.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   This is somewhat similar with https://github.com/apache/spark/pull/40561#discussion_r1159347699 too. Having one `*cols` would allow both `df.dropDuplicatesWithinWatermark(col1, col2)` and `dropDuplicatesWithinWatermark(*cols)`



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Can we have one `*cols`? I believe that's more consistent with other API in this file. e.g., groupby, select, etc.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,60 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Thanks, just added.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   I just had a discussion with @zsxwing offline. There was a confusion that we guarantee the same output between batch and streaming for new API which isn't true. To remove any confusion from users, we agreed to remove supporting batch query. I'll reflect the decision.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   batch DataFrame does not require watermark, and we actually remove withWatermark if the query is batch one. If we require batch query to provide watermark, that will be very odd because we ignore the delay threshold in any way.
   
   So I think there are two choices, 1) do not support batch query since it's confusing vs 2) tolerate the difference of UX and just do the same with existing API for batch case.
   cc. @rangadi 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   There is no perfect watermark, and any guarantees that apply only with "perfect watermark" are no guarantees at all. 
   I don't think semantic deferences require disabling batch. 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>
+        val row = r.asInstanceOf[UnsafeRow]
+        val key = getKey(row)
+        val value = store.get(key)
+        if (value == null) {
+          val timestamp = row.getLong(eventTimeColOrdinal)
+          // The unit of timestamp in Spark is microseconds, convert the delay threshold.
+          val expiresAt = timestamp + delayThresholdMillis * 1000
+
+          timeoutRow.setLong(0, expiresAt)
+          store.put(key, timeoutRow)
+
+          numUpdatedStateRows += 1
+          numOutputRows += 1
+          true
+        } else {
+          // Drop duplicated rows
+          numDroppedDuplicateRows += 1
+          false
+        }
+      }
+
+      CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
+        allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+        allRemovalsTimeMs += timeTakenMs {
+          // Convert watermark value to microsecond
+          val watermarkForEviction = eventTimeWatermarkForEviction.get * 1000
+          store.iterator().foreach { rowPair =>

Review Comment:
   It depends on the operator. E.g. for windowed aggregation : if we keep window as the first column and the state store supports efficient range iterator, it could minimize scanning.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),

Review Comment:
   35 should also be in, so 3.



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   @zsxwing one more thing to consider: Shall we extend existing 'DropDuplicate' Node to support this use case rather than introducing another logical node 'DropDuplicatesWithinWatermark'. It does not change the API, just the implementation. I am suggesting adding this option to exiting node. 
   



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {

Review Comment:
   I'll add "in DataFrame" as postfix.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   cc. @gengliangwang since he made efforts for TimestampNTZ. My understanding is that the underlying long value of TimestampType is based on epoch (+00:00) so technically it should be already a form of NTZ, but semantic and expectation from users during dealing with two types may not be very clear. Did we go through this topic in any area (SQL batch query)?



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   I'm not 100% sure whether underlying value for timestamp is based on epoch. If it is, probably OK to simply use LongType and let Spark interpret the value according to the type user desires. Let's see @gengliangwang 's input.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   This commit addressed the rollback of supporting batch query.
   0608889e4d1afc4bb5d1710eef45cf50d3c29a0f
   
   @zsxwing @rangadi Please have a quick look at the change. Thanks!



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   If you don't have any late record then dropDuplicates() guarantees the same result. That enables users to test their code with batch query first, and then migrate to streaming query later.
   
   dropDuplicatesWithinWatermark doesn't guarantee such thing so it's making no sense to test with batch query and migrate to streaming. That was why I had a offline discussion with @zsxwing .



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   Just added a dummy implementation.


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   @HyukjinKwon @amaliujia 
   Would you mind if I ask what happens with the mima check for this PR? 
   https://github.com/HeartSaVioR/spark/actions/runs/4536405777/jobs/7993077860
   
   Is it required to add PySpark API in this PR to pass Spark connect check? At least MiMa check failed in Scala codebase.


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


[GitHub] [spark] amaliujia commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   hmm I am not sure what you already did but I am thinking if you don't add anything into https://github.com/apache/spark/blob/master/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala, then you need update `CheckConnectJvmClientCompatibility`.
   
   


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
docs/structured-streaming-programming-guide.md:
##########
@@ -2132,6 +2132,48 @@ streamingDf <- withWatermark(streamingDf, "eventTime", "10 seconds")
 streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime")
 {% endhighlight %}
 
+</div>

Review Comment:
   Updated the description.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala:
##########
@@ -464,6 +469,19 @@ object UnsupportedOperationChecker extends Logging {
               throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset")
           }
 
+        case d: DeduplicateWithinWatermark if d.isStreaming =>
+          // Find any attributes that are associated with an eventTime watermark.
+          val watermarkAttributes = d.child.output.collect {
+            case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => a
+          }
+
+          // DeduplicateWithinWatermark requires event time column being set in the input DataFrame
+          if (watermarkAttributes.isEmpty) {
+            throwError(
+              "dropDuplicatesWithinWatermark is not supported on streaming DataFrames/DataSets " +

Review Comment:
   Let's leave it as it is, as this is the same pattern we error out for streaming aggregation.
   
   ```
           // watermark a group is never "finished" so we would never output anything.
           if (watermarkAttributes.isEmpty) {
             throwError(
               s"$outputMode output mode not supported when there are streaming aggregations on " +
                   s"streaming DataFrames/DataSets without watermark")(plan)
           }
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.

Review Comment:
   What about just documenting the guarantee in below comment and avoid mentioning the details? I don't think there is an easy way to explain the tricky and indeterministic behavior, and we probably don't need to.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be

Review Comment:
   I'd prefer to just disallow using this operator with batch query, because the guarantee of this operator is limited, and beyond the guarantee we will go with very different output for streaming than batch one. This is far different UX compared with other operators.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   I'd like to treat this operator very differently, because this is probably the first operator we clearly say we do not provide the same output between batch and streaming. Let's hear second voice on this.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>

Review Comment:
   This is physical node. Again I would like to distinguish this with existing API clearly - the semantic we produce is very different. For the code redundancy I'll try to deal with it.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>
+        val row = r.asInstanceOf[UnsafeRow]
+        val key = getKey(row)
+        val value = store.get(key)
+        if (value == null) {
+          val timestamp = row.getLong(eventTimeColOrdinal)
+          // The unit of timestamp in Spark is microseconds, convert the delay threshold.
+          val expiresAt = timestamp + delayThresholdMillis * 1000
+
+          timeoutRow.setLong(0, expiresAt)
+          store.put(key, timeoutRow)
+
+          numUpdatedStateRows += 1
+          numOutputRows += 1
+          true
+        } else {
+          // Drop duplicated rows
+          numDroppedDuplicateRows += 1
+          false
+        }
+      }
+
+      CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
+        allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+        allRemovalsTimeMs += timeTakenMs {
+          // Convert watermark value to microsecond
+          val watermarkForEviction = eventTimeWatermarkForEviction.get * 1000
+          store.iterator().foreach { rowPair =>

Review Comment:
   All stateful operators do this. There could be some approaches to address this, but these approaches tend to incur write amplification hence not very sure it's far better.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withTypedPlan {
+    val resolver = sparkSession.sessionState.analyzer.resolver

Review Comment:
   I'll dedup the code. Using the same logical node (and physical node) is the different issue.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)

Review Comment:
   The intention was to distinguish mills and micros (especially we are dealing with micros here), but not a big deal as we use Ms as millis here and there, and I documented for case of micros.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to

Review Comment:
   "ts + delay threshold" is something we guarantee. Maybe there could be edge case on guarantee for past events since it's more about watermark.
   
   Probably just better to say "This guarantees events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Not sure we want to be more verbose, e.g. explicitly saying that this may deduplicate more events than the time range. I don't think it's required, but I'm open to hear voices.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   I'm OK with that. Let's say batch query has infinite delay threshold of watermark. That will explain the behavior of batch query for this operator. (Pretty sure it's conceptual and we can just route to existing API for batch query.)



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   It actually came from flatMapGroupsWithState but now I get your question in offline. Ideally saying, we should probably use the same type the event time column has, and disallow changing type of the event time column, to prevent the case where the type of timestamp gets changed between TimestampType <-> TimestampNTZType which all of them can be matched with Long value but different semantic.
   
   While we can try to address the issue in this PR, it'd probably require another state format for other stateful operators if they have the same issue. (I'm seeing the same issue from flatMapGroupsWithState but not from others, hopefully.)



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


[GitHub] [spark] gengliangwang commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   @HeartSaVioR both TimestampType and TimestampNTZType are based on epoch. However, for TimestampType, the result will be adjusted based on the SQL conf `spark.sql.session.timeZone`.
   For example, we stored a timestamp of `2023-03-30 20:00:00` as TimestampType in Los Angeles time. If users set the `spark.sql.session.timeZone` as Beijing time(+08:00), the result will be `2023-03-31 11:00:00`.
   If we store it as TimestampNTZ type, the read result will always be `2023-03-30 20:00:00`



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


[GitHub] [spark] gengliangwang commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   @HeartSaVioR both TimestampType and TimestampNTZType are based on epoch. However, for TimestampType, the result will be adjusted based on the SQL conf spark.sql.session.timeZone.
   For example, we stored a timestamp of 2023-03-30 20:00:00 as TimestampType in Los Angeles time. If users set the spark.sql.session.timeZone as Beijing time(+08:00), the result will be 2023-03-31 11:00:00.
   If we store it as TimestampNTZ type, the read result will always be 2023-03-30 20:00:00



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


[GitHub] [spark] dstrodtman-db commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   @HeartSaVioR Confirming that the documentation in the initial PR description is accurate and up to date, as I'll be using it at the example I base my doc updates on.
   
   Also: I don't see these doc changes live in the SS programming guide. What Spark versions should they go live with?


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   I just added the pyspark implementation in this PR. It doesn't seem to be worthwhile to have another round of review specifically for pyspark, given that the review phase is not going fast enough.


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   dropDuplicates does not support exact same output between batch and streaming either. No stateful operation guarantees in the precense of late records. What is the difference here? Better to support batch in the same manner as dropDuplicates(). 
   I don't think it is a good UX for customer to get errors then we fix it by relaxing. 
   But I will leave the decision to you.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Grouping to the above Scala API in question; it is important to decide which one we want this API to be consistent with. I designed this API as a sibling of existing dropDuplicates API, which I wish to provide the same UX.
   
   Say, for the case of batch query, someone can add the postfix of the name from existing query (dropDuplicates -> dropDuplicatesWithinWatermark) and expect the query to be run without any issue. It is respected now regardless of the type of parameter they use. 
   
   For the case of streaming query, they might already have a query dealing with dropDuplicates but had been suffered from the issue I've mentioned in the section `why are the changes needed?`. In most cases, we expect them to just add the postfix of the name from existing query and discard checkpoint, and then enjoy that the problem gets fixed.
   
   I hope this is enough rationale to make the new API be consistent with dropDuplicates.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   I'm unsure we have a safe conversion rule between TimestampType vs TimestampNTZType. The type of event time column can change across multiple query runs, and we want to give the nicer error message even we end up with not supporting this kind of change.
   
   If we just use LongType in state, there is no schema checker being involved, and users may face incorrect output due to the possibly different semantic between two types. It may not be a big deal if both can be safely converted, but I'm not an expert (and nor is @zsxwing as we discussed offline) so just wanted to go conservatively.



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>
+        val row = r.asInstanceOf[UnsafeRow]
+        val key = getKey(row)
+        val value = store.get(key)
+        if (value == null) {
+          val timestamp = row.getLong(eventTimeColOrdinal)
+          // The unit of timestamp in Spark is microseconds, convert the delay threshold.
+          val expiresAt = timestamp + delayThresholdMillis * 1000
+
+          timeoutRow.setLong(0, expiresAt)
+          store.put(key, timeoutRow)
+
+          numUpdatedStateRows += 1
+          numOutputRows += 1
+          true
+        } else {
+          // Drop duplicated rows
+          numDroppedDuplicateRows += 1
+          false
+        }
+      }
+
+      CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
+        allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+        allRemovalsTimeMs += timeTakenMs {
+          // Convert watermark value to microsecond
+          val watermarkForEviction = eventTimeWatermarkForEviction.get * 1000
+          store.iterator().foreach { rowPair =>

Review Comment:
   [Unrelated to this PR] Do we scan the whole state in every microbatch? Probably same in all the stateful operators? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)

Review Comment:
   `Ms` is the common suffix for milliseconds in spark. 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   Now I don't know how Github comment works :) It was outdated and no longer be outdated as I revert the change.
   
   @rangadi @gengliangwang Let's use this comment thread.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)

Review Comment:
   1 entries should be always left as we have no mechanism to advance watermark without input.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)

Review Comment:
   1 entry should be always left as we have no mechanism to advance watermark without input.



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   Just filed another JIRA ticket https://issues.apache.org/jira/browse/SPARK-43027 to support PySpark. Once we merge this in I'll work on PySpark side.


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   >  The type of event time column can change across multiple query runs, 
   
   In that case, how about just storing a long for 'expireAt' rather than matching timestamp type for value? 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))

Review Comment:
   Optional.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))

Review Comment:
   https://issues.apache.org/jira/browse/SPARK-42990



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   That's also just for consistency of method doc for existing dropDuplicates API.



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

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

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/tests/connect/test_parity_dataframe.py:
##########
@@ -41,6 +41,11 @@ def test_observe(self):
     def test_observe_str(self):
         super().test_observe_str()
 
+    # TODO(SPARK-XXXXX): Support Structured Streaming

Review Comment:
   This test does not use streaming.



##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming

Review Comment:
   Remove 'just'



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   > In addition, too late data older than watermark will be dropped.
   
   Why is it required here? Simpler to remove? 



##########
docs/structured-streaming-programming-guide.md:
##########
@@ -2132,6 +2132,61 @@ streamingDf <- withWatermark(streamingDf, "eventTime", "10 seconds")
 streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime")
 {% endhighlight %}
 
+</div>
+
+</div>
+
+Specifically for streaming, you can deduplicate records in data streams using a unique identifier in the events, within the time range of watermark.
+For example, if you set the delay threshold of watermark as "1 hour", duplicated events which occurred within 1 hour can be correctly deduplicated.
+(For more details, please refer to the API doc of [dropDuplicatesWithinWatermark](/api/scala/org/apache/spark/sql/Dataset.html#dropDuplicatesWithinWatermark():org.apache.spark.sql.Dataset[T]).)
+
+This can be used to deal with use case where event time column cannot be a part of unique identifier, mostly due to the case
+where event times are somehow different for the same records. (E.g. non-idempotent writer where issuing event time happens at write)
+
+Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.
+
+This feature requires watermark with delay threshold to be set in streaming DataFrame/Dataset.
+
+<div class="codetabs">
+
+<div data-lang="python"  markdown="1">
+
+{% highlight python %}
+streamingDf = spark.readStream. ...
+
+# deduplicate using guid column with watermark based on eventTime column
+streamingDf \
+  .withWatermark("eventTime", "10 seconds") \
+  .dropDuplicatesWithinWatermark("guid")
+{% endhighlight %}
+
+</div>
+
+<div data-lang="scala"  markdown="1">
+
+{% highlight scala %}
+val streamingDf = spark.readStream. ...  // columns: guid, eventTime, ...
+
+// deduplicate using guid column with watermark based on eventTime column
+streamingDf
+  .withWatermark("eventTime", "10 seconds")

Review Comment:
   How about "1 hour" or "1 day"? 10 seconds seems very low for most use cases of this. 



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Grouping to the above Scala API in question; it is important to decide which one we want this API to be consistent with. I'm seeing this API as a sibling of existing dropDuplicates API, which I wish to provide the same UX.
   
   Say, for the case of batch query, someone can add the postfix of the name from existing query (dropDuplicates -> dropDuplicatesWithinWatermark) and expect the query to be run without any issue. It is respected now regardless of the type of parameter they use. 
   
   For the case of streaming query, they might already have a query dealing with dropDuplicates but had been suffered from the issue I've mentioned in the section `why are the changes needed?`. In most cases, we expect them to just add the postfix of the name from existing query and discard checkpoint, and then enjoy that the problem gets fixed. I hope this is enough rationale to make the new API be consistent with dropDuplicates.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   @zsxwing Does my comment for leaving this as it is make sense to you? Could you please have another round of review?



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:
##########
@@ -562,79 +562,6 @@ private void assertEqualsUnorderly(
     );
   }
 
-  @Test

Review Comment:
   This was actually missing piece for dropDuplicates() as well. Since we remove the functionality for batch query in dropDuplicatesWithinWatermark(), it's probably better to have another PR to deal with this.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   (If we want to implement the same behavior to batch query, we will have to kick the part of "best effort" out as well in streaming. e.g. We deduplicate the event whenever there is an existing state, which does not strictly say they're within delay threshold. We evict the state at the end of processing, hence we are accepting slightly more events to be deduplicated. That might be better behavior for streaming, but if we want to match the behavior between batch and streaming, the behavior must be deterministic.)



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala:
##########
@@ -464,6 +469,19 @@ object UnsupportedOperationChecker extends Logging {
               throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset")
           }
 
+        case d: DeduplicateWithinWatermark if d.isStreaming =>
+          // Find any attributes that are associated with an eventTime watermark.
+          val watermarkAttributes = d.child.output.collect {
+            case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => a
+          }
+
+          // DeduplicateWithinWatermark requires event time column being set in the input DataFrame
+          if (watermarkAttributes.isEmpty) {
+            throwError(
+              "dropDuplicatesWithinWatermark is not supported on streaming DataFrames/DataSets " +

Review Comment:
   [optional] 
   "dropDuplicatesWithinWatermark() requires watermark to be set set on Dataframe, but there is no watermark set." 



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.

Review Comment:
   I know it is a tricky thing, but it might be better to rephrase. 



##########
docs/structured-streaming-programming-guide.md:
##########
@@ -2132,6 +2132,48 @@ streamingDf <- withWatermark(streamingDf, "eventTime", "10 seconds")
 streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime")
 {% endhighlight %}
 
+</div>

Review Comment:
   [From PR description]
   > Only guarantee to deduplicate events within the watermark.
   
   'within watermark delay'



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to

Review Comment:
   I don't think we guarantee this condition : `(ts - delay threshold, ts + delay threshold)`. 
   we likely need to rephrase it. We can look at the scaladoc towards the end before merging this. 



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withTypedPlan {
+    val resolver = sparkSession.sessionState.analyzer.resolver

Review Comment:
   can we share the this with dropDuplicate()? or even better we can reuse 'Deduplicate()' node(s). 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>

Review Comment:
   When we reuse Deduplicate() logical node: mainly this part and removal would be different based on 'dropWithinWatermark' flag. That we most of the remaining code remains unchanged. 



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * as long as event times of duplicated rows are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be
+   * set via [[withWatermark]].
+   *
+   * This will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the following, "If event time of the first arrived event is
+   * 'ts', this guarantees all duplicated rows will be dropped where these rows are within the time
+   * range of (ts - delay threshold, ts + delay threshold)". In practice, users are encouraged to
+   * set the delay threshold of watermark longer than max timestamp differences among duplicated
+   * events.
+   *
+   * In addition, too late data older than watermark will be dropped to avoid any possibility
+   * of duplicates.

Review Comment:
   Same here. 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   Rather than making this a separate logical node, we can we make the new behavior an option in Deduplicate node? That way we don't need to distinguish them in the implementation except in couple of places. 



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be

Review Comment:
   Also mentioned above: `withWatermark()` is allowed in batch, same way this could be allowed to. Essentially this is same as normal dropDuplicates(). 



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   cc. @zsxwing @viirya @rangadi Friendly reminder.


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


[GitHub] [spark] zsxwing commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+
+  protected val extraOptionOnStateStore: Map[String, String] = Map.empty
+
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMs = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  protected def initializeReusedDupInfoRow(): Option[UnsafeRow] = {
+    val timeoutToUnsafeRow = UnsafeProjection.create(schemaForValueRow)
+    val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForValueRow))
+    Some(timeoutRow)
+  }
+
+  protected def putDupInfoIntoState(
+      store: StateStore,
+      data: UnsafeRow,
+      key: UnsafeRow,
+      reusedDupInfoRow: Option[UnsafeRow]): Unit = {
+    assert(reusedDupInfoRow.isDefined, "This should have reused row.")
+    val timeoutRow = reusedDupInfoRow.get
+
+    val timestamp = data.getLong(eventTimeColOrdinal)
+    // The unit of timestamp in Spark is microseconds, convert the delay threshold to micros.
+    val expiresAt = timestamp + delayThresholdMs * 1000

Review Comment:
   nit: Should we use `org.apache.spark.sql.catalyst.util.DateTimeUtils#millisToMicros` to handle overflow (e.g., the user sets a very large `delayThresholdMs`)? We can fix this later since you are just following the current `watermarkExpression` code.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   QQ: why use LongType rather than `TimestampType`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   > I don't think semantic deferences require disabling batch.
   
   +1. I don't see why this should be different than the existing `dropDuplicates`.



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   If we are saving long, I would suggesting changing the name to `expiresAtMs` or `expiresAtMicros` depending on it unit. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   If we are saving long, I would suggest changing the name to `expiresAtMs` or `expiresAtMicros` depending on it unit. 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   So the type of value in the state will be Long? SGTM.



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   Yeah, went with the two node option. @HeartSaVioR reused a lot of the code. 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)

Review Comment:
   It could be possible with stream-stream join but that sounds to me as overkill.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   I might misunderstand your point. Technically it does not need to be the same with existing API - maybe we can emit the late event as it is without deduplication, but that will give us more complicated scenarios to deal with. For example, what if two duplicated events are ingested in late time but they are closer to the watermark delay threshold by themselves? Should/Can we deduplicate them?



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   cc. @zsxwing @viirya @rangadi Could you please review this again? I feel this is very close to the final shape.
   


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming

Review Comment:
   That's just for consistency of method doc for existing dropDuplicates API.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = {

Review Comment:
   E.g. Your suggestion passes the new test addition in sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java but had to change (`: _*` as you mentioned) Scala test suite to pass. Having the same set of overloaded methods makes all tests pass. (Here I meant able to compile as pass.)



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   Updated. I just removed the example (test) for batch case since it sounds to be obvious. I can split example for batch and streaming and add back batch queries if we want to.



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


[GitHub] [spark] zsxwing commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   It sounds weird to me that a method is called dropDuplicates**WithinWatermark** but I don't need to set the watermark.
   
   > For batch, there are a bunch of tools to perform deduplication, distinct / dropDuplicates / dropDuplicatesWithinWatermark. Most of batch use cases don't need to come up with using dropDuplicatesWithinWatermark.
   
   I think the common use case for `dropDuplicatesWithinWatermark` in batch is: develop the code in batch mode and switch to streaming later. In this case, catching potential issues in batch mode is better.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   If we want to support this API in batch query, I think we have to implement the same behavior, not just forwarding to dropDuplicates(). But that's also very odd because we are telling users that watermark is no-op in batch query and now we have to get delay threshold from withWatermark. I'd say it conflicts with base concept.



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


[GitHub] [spark] amaliujia commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   I think you need update at connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   I can't find the code example to add the code in example section but not evaluating as test. It would be tricky if we have to run streaming query from there and also have to validate something.
   
   @HyukjinKwon Would you mind if I ask for kindly guiding about pyspark API doc? Are these codes in example section still executed as tests? If they are, is there a way to prevent it for several lines or per method? Thanks in advance!



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   For now, it just follows the original Scala method name.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   I just had a discussion with @zsxwing offline. There was a confusion that we guarantee the same output between batch and streaming for new API (like existing dropDuplicates) which isn't true. To remove any confusion from users, we agreed to remove supporting batch query. I'll reflect the decision.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Grouping to the above Scala API in question; it is important to decide which one we want this API to be consistent with. I'm seeing this API as a sibling of existing dropDuplicates API, which I wish to provide the same UX.
   
   Say, for the case of batch query, someone can add the postfix of the name from existing query (dropDuplicates -> dropDuplicatesWithinWatermark) and expect the query to be run without any issue. It is respected now regardless of the type of parameter they use. 
   
   For the case of streaming query, they might already have a query dealing with dropDuplicates but had been suffered from the issue I've mentioned in the section `why are the changes needed?`. In most cases, we expect them to just add the postfix of the name from existing query and discard checkpoint, and then enjoy that the problem gets fixed.
   
   I hope this is enough rationale to make the new API be consistent with dropDuplicates.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   (If we want to implement the same behavior to batch query, we will have to kick the part of "best effort" out as well in streaming. e.g. We deduplicate the event whenever there is an existing state, which does not strictly say they're within delay threshold. We evict the state at the end of processing, hence we are accepting slightly more events to be deduplicated. That might be better behavior for streaming, but if we want to guarantee the same result between batch and streaming, the behavior must be deterministic.)



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Oh, I see. Okay, makes sense to me.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   Yeah that's also required to guarantee the semantic of watermark. Good point. Will change the word.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   I'm unsure we have a safe conversion rule between TimestampType vs TimestampNTZType. The type of event time column can change across multiple query runs, and we want to give the nicer error message even we end up with not supporting this kind of change.
   
   If we just use LongType in state, state schema checker does not know whether the value is from TimestampType or TimestampNTZType, and users may face incorrect output due to the possibly different semantic between two types. It may not be a big deal if both can be safely converted with underlying long value, but I'm not an expert (and nor is @zsxwing as we discussed offline) so just wanted to go conservatively.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   cc. @gengliangwang since he made efforts for TimestampNTZ. My understanding is that the underlying long value of TimestampType is based on epoch (+00:00) so technically it should be already a form of NTZ, but semantic and expectation from users during dealing with two types may not be very clear. Did we go through this topic?



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)

Review Comment:
   > 1 entry should be always left as we have no mechanism to advance watermark without input.
   
   Ahh.... 😌



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {

Review Comment:
   That's actually not the main purpose (just wanted to test with subset of columns) but it's also good to frame like that.



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   The error only occurred from linter - it now does not allow a new PR to introduce a new public API "without adding to spark-connect". This PR intentionally postpones addressing PySpark in separate JIRA ticket, hence addressing spark-connect should go to there as well.


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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3038,118 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, as long as event times of duplicated rows
+   * are within delay threshold of watermark.
+   *
+   * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be

Review Comment:
   I would like to explicitly block users simply switching over batch and streaming with this operator and reason about the similar 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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   dropDuplicates() produces different output between batch and streaming too (because batch does not drop late records). 
   As much as possible it is better to keep the API same between batch and streaming. 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   This commit addressed the rollback of supporting batch query. Changes in test suites are rolled back because these test suites are for batch queries.
   0608889e4d1afc4bb5d1710eef45cf50d3c29a0f
   
   @zsxwing @rangadi Please have a quick look at the change. Thanks!



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

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

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


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


[GitHub] [spark] HeartSaVioR closed pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark
URL: https://github.com/apache/spark/pull/40561


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   I actually intend to only provide streaming examples to avoid the additional explanation on batch case. For batch, there are a bunch of tools to perform deduplication, distinct / dropDuplicates / dropDuplicatesWithinWatermark. Most of batch use cases don't need to come up with using dropDuplicatesWithinWatermark.



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   The last update is to rebase with master branch - just to make sure CI is happy with the change before merging this.


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

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

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   It is statement on how watermark works, not related to dropDuplicatate(). R
   May be better to say 'Note that ...' rather than 'In addition ...' 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   [Updated my comment]: I think batch should not require watermark. But all our examples including batch examples should set watermark to emphasis its importance. 



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   Yeah, went with author's preferred two node option. @HeartSaVioR reused a lot of the code. 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
docs/structured-streaming-programming-guide.md:
##########
@@ -2132,6 +2132,61 @@ streamingDf <- withWatermark(streamingDf, "eventTime", "10 seconds")
 streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime")
 {% endhighlight %}
 
+</div>
+
+</div>
+
+Specifically for streaming, you can deduplicate records in data streams using a unique identifier in the events, within the time range of watermark.
+For example, if you set the delay threshold of watermark as "1 hour", duplicated events which occurred within 1 hour can be correctly deduplicated.
+(For more details, please refer to the API doc of [dropDuplicatesWithinWatermark](/api/scala/org/apache/spark/sql/Dataset.html#dropDuplicatesWithinWatermark():org.apache.spark.sql.Dataset[T]).)
+
+This can be used to deal with use case where event time column cannot be a part of unique identifier, mostly due to the case
+where event times are somehow different for the same records. (E.g. non-idempotent writer where issuing event time happens at write)
+
+Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.
+
+This feature requires watermark with delay threshold to be set in streaming DataFrame/Dataset.
+
+<div class="codetabs">
+
+<div data-lang="python"  markdown="1">
+
+{% highlight python %}
+streamingDf = spark.readStream. ...
+
+# deduplicate using guid column with watermark based on eventTime column
+streamingDf \
+  .withWatermark("eventTime", "10 seconds") \
+  .dropDuplicatesWithinWatermark("guid")
+{% endhighlight %}
+
+</div>
+
+<div data-lang="scala"  markdown="1">
+
+{% highlight scala %}
+val streamingDf = spark.readStream. ...  // columns: guid, eventTime, ...
+
+// deduplicate using guid column with watermark based on eventTime column
+streamingDf
+  .withWatermark("eventTime", "10 seconds")

Review Comment:
   Agreed for this API. Will fix. Thanks for the suggestion!



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/tests/connect/test_parity_dataframe.py:
##########
@@ -41,6 +41,11 @@ def test_observe(self):
     def test_observe_str(self):
         super().test_observe_str()
 
+    # TODO(SPARK-XXXXX): Support Structured Streaming

Review Comment:
   I just updated the TODO comment with actual JIRA ticket number.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   I might misunderstand your point. Technically it does not need to be the same with existing API - maybe we can emit the late event as it is without deduplication - but that will give us more complicated scenarios to deal with. For example, what if two duplicated events are ingested in late time but they are closer compared to the watermark delay threshold by themselves? Should/Can we deduplicate them?



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,60 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   This would have to be listed in `https://github.com/apache/spark/blob/master/python/docs/source/reference/pyspark.sql/dataframe.rst`



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   expiresAtMicros looks OK to me. Thanks for suggestion!



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +980,117 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends UnaryExecNode with StateStoreWriter with WatermarkSupport {
+
+  /** Distribute by grouping attributes */
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(
+      keyExpressions, getStateInfo, conf) :: Nil
+  }
+
+  private val schemaForTimeoutRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMillis = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics // force lazy init at driver
+
+    child.execute().mapPartitionsWithStateStore(
+      getStateInfo,
+      keyExpressions.toStructType,
+      schemaForTimeoutRow,
+      numColsPrefixKey = 0,
+      session.sessionState,
+      Some(session.streams.stateStoreCoordinator)) { (store, iter) =>
+      val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output)
+
+      val timeoutToUnsafeRow = UnsafeProjection.create(schemaForTimeoutRow)
+      val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForTimeoutRow))
+
+      val numOutputRows = longMetric("numOutputRows")
+      val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+      val numRemovedStateRows = longMetric("numRemovedStateRows")
+      val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+      val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
+      val commitTimeMs = longMetric("commitTimeMs")
+      val numDroppedDuplicateRows = longMetric("numDroppedDuplicateRows")
+
+      val baseIterator = watermarkPredicateForDataForLateEvents match {
+        case Some(predicate) => applyRemovingRowsOlderThanWatermark(iter, predicate)
+        case None => iter
+      }
+
+      val updatesStartTimeNs = System.nanoTime
+
+      val result = baseIterator.filter { r =>
+        val row = r.asInstanceOf[UnsafeRow]
+        val key = getKey(row)
+        val value = store.get(key)
+        if (value == null) {
+          val timestamp = row.getLong(eventTimeColOrdinal)
+          // The unit of timestamp in Spark is microseconds, convert the delay threshold.
+          val expiresAt = timestamp + delayThresholdMillis * 1000
+
+          timeoutRow.setLong(0, expiresAt)
+          store.put(key, timeoutRow)
+
+          numUpdatedStateRows += 1
+          numOutputRows += 1
+          true
+        } else {
+          // Drop duplicated rows
+          numDroppedDuplicateRows += 1
+          false
+        }
+      }
+
+      CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
+        allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+        allRemovalsTimeMs += timeTakenMs {
+          // Convert watermark value to microsecond
+          val watermarkForEviction = eventTimeWatermarkForEviction.get * 1000
+          store.iterator().foreach { rowPair =>

Review Comment:
   Yeah we leverage range scan in the native support of session window with RocksDB state store provider. Maybe it would enable us to apply further optimization, although we should still be very careful about changing the state format.



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   @dstrodtman-db We will release this feature in Spark 3.5.0. We don't have the tentative date to release Spark 3.5.0.


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala:
##########
@@ -464,6 +469,19 @@ object UnsupportedOperationChecker extends Logging {
               throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset")
           }
 
+        case d: DeduplicateWithinWatermark if d.isStreaming =>
+          // Find any attributes that are associated with an eventTime watermark.
+          val watermarkAttributes = d.child.output.collect {
+            case a: Attribute if a.metadata.contains(EventTimeWatermark.delayKey) => a
+          }
+
+          // DeduplicateWithinWatermark requires event time column being set in the input DataFrame
+          if (watermarkAttributes.isEmpty) {
+            throwError(
+              "dropDuplicatesWithinWatermark is not supported on streaming DataFrames/DataSets " +

Review Comment:
   I'm also OK to go with new error message pattern if you feel like it's clearer.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   No what I meant is, if we assume the perfect watermark on streaming side, streaming query is designed to produce the same output with batch one. dropDuplicates() is no exception.
   
   Arguably, dropDuplicatesWithinWatermark is an exception because they are not only required to reason about "lateness" of the data, but also required to reason about max time duration on duplicated events. (Technically saying, both are different.) Would it be safe to just change read to readStream / write to writeStream and vice versa? Mostly yes for existing API, but maybe not for this API.



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

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

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {

Review Comment:
   Isn't this testing the case where "eventTime" column is not one of the dedup-columns? 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   I'm OK with that. Let's say batch query has infinite delay threshold of watermark. That will explain the behavior of batch query for this operator.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   I'm unsure we have a safe conversion rule between TimestampType vs TimestampNTZType. The type of event time column can change across multiple query runs, and we want to give the nicer error message even we end up with not supporting this kind of change.
   
   If we just use LongType in state, there is no schema checker being involved, and users may face incorrect output due to the possibly different semantic between two types. It may not be a big deal if both can be safely converted with underlying long value, but I'm not an expert (and nor is @zsxwing as we discussed offline) so just wanted to go conservatively.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {

Review Comment:
   Ah it's saying it's in DataFrame not subset, as this operator does not produce different behavior even though event time column is in the subset. It's just to distinguish the test to the above sanity test where there is no event time column in DataFrame.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))
+
+  protected val extraOptionOnStateStore: Map[String, String] = Map.empty
+
+  private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output,
+    allowMultipleEventTimeColumns = false).get
+  private val delayThresholdMs = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey)
+  private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol)
+
+  protected def initializeReusedDupInfoRow(): Option[UnsafeRow] = {
+    val timeoutToUnsafeRow = UnsafeProjection.create(schemaForValueRow)
+    val timeoutRow = timeoutToUnsafeRow(new SpecificInternalRow(schemaForValueRow))
+    Some(timeoutRow)
+  }
+
+  protected def putDupInfoIntoState(
+      store: StateStore,
+      data: UnsafeRow,
+      key: UnsafeRow,
+      reusedDupInfoRow: Option[UnsafeRow]): Unit = {
+    assert(reusedDupInfoRow.isDefined, "This should have reused row.")
+    val timeoutRow = reusedDupInfoRow.get
+
+    val timestamp = data.getLong(eventTimeColOrdinal)
+    // The unit of timestamp in Spark is microseconds, convert the delay threshold to micros.
+    val expiresAt = timestamp + delayThresholdMs * 1000

Review Comment:
   Thanks, I didn't know about the existence of the util. Will try it out.



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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   > What is the decision about batch support?
   
   I just added support of batch in the latest commit. It needs be more test coverage for batch query support so that's why we have new FIXMEs. All FIXMEs should be resolved before merging.


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   I'm not 100% sure whether underlying value for timestamp is based on epoch. If it is, probably OK to simply use LongType and let Spark interpret the value according to the type user desires. Let's see @gengliangwang 's input - it might be likely we already constructed the semantic for this.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = {

Review Comment:
   I'm not sure Scala users would feel convenient to use API which require `: _*` to be set by themselves.
   
   I wouldn't like users to have different UX than existing dropDuplicates as they are sibling APIs. I just tried to add some cases for Scala and Java, and unfortunately removing any of methods don't go well.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   batch DataFrame does not require watermark, and we actually remove withWatermark if the query is batch one. If we require batch query to provide watermark, that will be very odd because we ignore the delay threshold in any way.
   
   So I think there are two choices, do not support batch query since it's confusing vs just do the same with existing API for batch case.
   cc. @rangadi 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   Thanks!



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

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

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   I think it better to include watermark since this serves as important documentation for the users. Most of the time, the user is looking for an example to use in streaming. Not including watermark here is going to be confusing for them. 
   The fact that it is ignored in batch is fine. 



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


[GitHub] [spark] zsxwing commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   @HeartSaVioR Thanks! We can wait for the user's feedback first before supporting batch queries. Changing from an error to no error is always easier.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   (If we want to implement the same behavior to batch query, we will have to kick the part of "best effort" out as well in streaming. e.g. We deduplicate the event whenever there is an existing state, which does not say they're within delay threshold. We evict the state at the end of processing, hence we are accepting slightly more events to be deduplicated. That might be better behavior for streaming, but if we want to match the behavior between batch and streaming, the behavior must be deterministic.)



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   That's my understanding as well. I see drop_duplicates in PySpark DataFrame as well, but most of methods don't have such alias.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.

Review Comment:
   I might misunderstand your point. Technically it does not need to be the same with existing API - maybe we can emit the late event as it is without deduplication - but that will give us more complicated scenarios to deal with. For example, what if two duplicated events are ingested in late time but they are closer to the watermark delay threshold by themselves? Should/Can we deduplicate them?



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   I'll look into it. I blindly followed dropDuplicates() but we'd probably want to emphasize the case of streaming for this API.



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

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

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


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


[GitHub] [spark] zsxwing commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":

Review Comment:
   @HyukjinKwon what's our current policy to add Python methods? Does it have to be a pythonic name such as `drop_duplicates_within_watermark`, or do we still follow the scala method name?



##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()
+         +-----+---+------+
+         | name|age|height|
+         +-----+---+------+
+         |Alice|  5|    80|
+         |Alice| 10|    80|
+         +-----+---+------+
+
+         Deduplicate values on 'name' and 'height' columns.
+
+         >>> df.dropDuplicatesWithinWatermark(['name', 'height']).show()

Review Comment:
   ditto



##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   Can we create examples using watermark?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   > Shall we extend existing 'DropDuplicate' Node to support this use case rather than introducing another logical node 'DropDuplicatesWithinWatermark'.
   
   I don't have a strong opinion on this. Will let you two to decide 😄 



##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   Does batch DataFrame require watermark? I think it's better to make it fail if there is no watermark to make it consistent with streaming DataFrame.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   If you don't have any late record then dropDuplicates() guarantees the same result. That enables users to test their code with batch query first, and then migrate to streaming query later.
   
   dropDuplicatesWithinWatermark doesn't guarantee such thing so it's making no sense to test with batch query and migrate to streaming. That was why I had an offline discussion with @zsxwing .



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer

Review Comment:
   If we want to support this API in batch query, I think we have to implement the same behavior, not forwarding to dropDuplicates(). But that's also very odd because we are telling users that watermark is no-op in batch query and now we have to get delay threshold from withWatermark. I'd say it conflicts with base concept.



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   ~~I can't find the code example to add the code in example section but not evaluating as test. It would be tricky if we have to run streaming query from there and also have to validate something.~~
   
   ~~@HyukjinKwon Would you mind if I ask for kindly guiding about pyspark API doc? Are these codes in example section still executed as tests? If they are, is there a way to prevent it for several lines or per method? Thanks in advance!~~
   
   Never mind, just found it.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   Confirmed CI passed for last commit. https://github.com/HeartSaVioR/spark/runs/12606973127


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   Thanks all for reviewing! Merging to master.


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

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

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


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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()
+         +-----+---+------+
+         | name|age|height|
+         +-----+---+------+
+         |Alice|  5|    80|
+         |Alice| 10|    80|
+         +-----+---+------+
+
+         Deduplicate values on 'name' and 'height' columns.
+
+         >>> df.dropDuplicatesWithinWatermark(['name', 'height']).show()

Review Comment:
   +1. Lets add watermark() even if this is a batch example. better still is to make it streaming example (use rate source, and we relax the test assert)



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


[GitHub] [spark] rangadi commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3928,6 +3928,71 @@ def dropDuplicates(self, subset: Optional[List[str]] = None) -> "DataFrame":
             jdf = self._jdf.dropDuplicates(self._jseq(subset))
         return DataFrame(jdf, self.sparkSession)
 
+    def dropDuplicatesWithinWatermark(self, subset: Optional[List[str]] = None) -> "DataFrame":
+        """Return a new :class:`DataFrame` with duplicate rows removed,
+         optionally only considering certain columns, within watermark.
+
+        For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
+        :class:`DataFrame`, this will keep all data across triggers as intermediate state to drop
+        duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated
+        as long as the time distance of earliest and latest events are smaller than the delay
+        threshold of watermark." The watermark for the input :class:`DataFrame` must be set via
+        :func:`withWatermark`. Users are encouraged to set the delay threshold of watermark longer
+        than max timestamp differences among duplicated events. In addition, too late data older
+        than watermark will be dropped.
+
+         .. versionadded:: 3.5.0
+
+         Parameters
+         ----------
+         subset : List of column names, optional
+             List of columns to use for duplicate comparison (default All columns).
+
+         Returns
+         -------
+         :class:`DataFrame`
+             DataFrame without duplicates.
+
+         Examples
+         --------
+         >>> from pyspark.sql import Row
+         >>> df = spark.createDataFrame([
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=5, height=80),
+         ...     Row(name='Alice', age=10, height=80)
+         ... ])
+
+         Deduplicate the same rows.
+
+         >>> df.dropDuplicatesWithinWatermark().show()

Review Comment:
   Also, even if it is a batch example, better to set watermark. It will serve as good example for users, since most of them will be looking to use in streaming context. 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala:
##########
@@ -679,6 +679,8 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
       d.withNewChildren(Seq(simplifyUnion(u)))
     case d @ Deduplicate(_, u: Union) =>
       d.withNewChildren(Seq(simplifyUnion(u)))
+    case d @ DeduplicateWithinWatermark(_, u: Union) =>

Review Comment:
   No what I meant is, if we assume the perfect watermark on streaming side, streaming query is designed to produce the same output with batch one. dropDuplicates() is no exception.
   
   Arguably, dropDuplicatesWithinWatermark is an exception because they are not only required to reason about "lateness", but also required to reason about max time duration on duplicated events. Would it be safe to just change read to readStream / write to writeStream and vice versa? Mostly yes for existing API, but maybe not for this API.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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

   I was wondering what is different from dropDuplicates and this one. I don't see dropDuplicates being handled separately. Is it because the PySpark implementation of dropDuplicates is available?
   
   If this method has to be excluded, could you please guide how to do that? Thanks in advance.


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))

Review Comment:
   Let's file a JIRA ticket to employ error framework against UnsupportedOperationChecker instead.



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

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

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


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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   https://github.com/apache/spark/pull/40561#discussion_r1153695496



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   OK so that's more about representation and the underlying value is the same. That said, it seems like TimestampType and TimestampNTZType are interchangeable, as long as they fully understand that comparison of both should happen in sprits of NTZ (TimestampType will be adjusted, but it doesn't mean the underlying value is adjusted. It's still about representation.) 
   
   That said, storing the underlying value using LongType sounds to me as still be valid, till we break the underlying format which would incur a lot more troublesome issues.
   
   cc. @zsxwing 



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 org.apache.spark.sql.{AnalysisException, Dataset, SaveMode}
+import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.timestamp_seconds
+
+class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest {
+
+  import testImplicits._
+
+  test("deduplicate without event time column") {
+    def testAndVerify(df: Dataset[_]): Unit = {
+      val exc = intercept[AnalysisException] {
+        df.writeStream.format("noop").start()
+      }
+
+      assert(exc.getMessage.contains("dropDuplicatesWithinWatermark is not supported"))
+      assert(exc.getMessage.contains("streaming DataFrames/DataSets without watermark"))
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().dropDuplicatesWithinWatermark()
+    testAndVerify(result)
+
+    val result2 = inputData.toDS().withColumn("newcol", $"value")
+      .dropDuplicatesWithinWatermark("newcol")
+    testAndVerify(result2)
+
+    val inputData2 = MemoryStream[(String, Int)]
+    val otherSideForJoin = inputData2.toDF()
+      .select($"_1" as "key", timestamp_seconds($"_2") as "time")
+      .withWatermark("Time", "10 seconds")
+
+    val result3 = inputData.toDS()
+      .select($"value".as("key"))
+      // there are two streams which one stream only defines the watermark. the stream which
+      // contains dropDuplicatesWithinWatermark does not define the watermark, which is not
+      // supported.
+      .dropDuplicatesWithinWatermark()
+      .join(otherSideForJoin, "key")
+    testAndVerify(result3)
+  }
+
+  test("deduplicate with all columns with event time column") {
+    val inputData = MemoryStream[Int]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicatesWithinWatermark()
+      .select($"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advance watermark to 5 secs, no-data-batch does not drop state rows
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      // Advance watermark to 7 secs, no-data-batch does not drop state rows
+      AddData(inputData, (13 to 17): _*),
+      // 13 to 15 are duplicated
+      CheckNewAnswer(16, 17),
+      assertNumStateRows(total = 8, updated = 2),
+
+      AddData(inputData, 5), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 8, updated = 0, droppedByWatermark = 1),
+
+      // Advance watermark to 25 secs, no-data-batch drops state rows having expired time <= 25
+      AddData(inputData, 35),
+      CheckNewAnswer(35),
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advance watermark to 45 seconds, no-data-batch drops state rows having expired time <= 45
+      AddData(inputData, 55),
+      CheckNewAnswer(55),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  test("deduplicate with some columns with event time column") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"_2"))
+      .withWatermark("eventTime", "2 seconds")
+      .dropDuplicatesWithinWatermark("_1")
+      .select($"_1", $"eventTime".cast("long").as[Long])
+
+    testStream(result, Append)(
+      // Advances watermark to 15
+      AddData(inputData, "a" -> 17),
+      CheckNewAnswer("a" -> 17),
+      // expired time is set to 19
+      assertNumStateRows(total = 1, updated = 1),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 16),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0),
+
+      // Watermark does not advance
+      // Should not emit anything as data less than watermark
+      AddData(inputData, "a" -> 13),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      // Advances watermark to 20. no-data batch drops state row ("a" -> 19)
+      AddData(inputData, "b" -> 22, "c" -> 21),
+      CheckNewAnswer("b" -> 22, "c" -> 21),
+      // expired time is set to 24 and 23
+      assertNumStateRows(total = 2, updated = 2),
+
+      // Watermark does not advance
+      AddData(inputData, "a" -> 21),
+      // "a" is identified as new event since previous batch dropped state row ("a" -> 19)
+      CheckNewAnswer("a" -> 21),
+      // expired time is set to 23
+      assertNumStateRows(total = 3, updated = 1),
+
+      // Advances watermark to 23. no-data batch drops state row ("a" -> 23), ("c" -> 23)
+      AddData(inputData, "d" -> 25),
+      CheckNewAnswer("d" -> 25),
+      assertNumStateRows(total = 2, updated = 1)
+    )
+  }
+
+  test("SPARK-39650: duplicate with specific keys should allow input to change schema") {
+    withTempDir { checkpoint =>
+      val dedupeInputData = MemoryStream[(String, Int)]
+      val dedupe = dedupeInputData.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark("_1")
+        .select($"_1", $"eventTime".cast("long").as[Long])
+
+      testStream(dedupe, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData, "a" -> 1),
+        CheckNewAnswer("a" -> 1),
+
+        AddData(dedupeInputData, "a" -> 2, "b" -> 3),
+        CheckNewAnswer("b" -> 3)
+      )
+
+      val dedupeInputData2 = MemoryStream[(String, Int, String)]
+      val dedupe2 = dedupeInputData2.toDS()
+        .withColumn("eventTime", timestamp_seconds($"_2"))
+        .withWatermark("eventTime", "10 second")
+        .dropDuplicatesWithinWatermark(Seq("_1"))
+        .select($"_1", $"eventTime".cast("long").as[Long], $"_3")
+
+      // initialize new memory stream with previously executed batches
+      dedupeInputData2.addData(("a", 1, "dummy"))
+      dedupeInputData2.addData(Seq(("a", 2, "dummy"), ("b", 3, "dummy")))
+
+      testStream(dedupe2, Append)(
+        StartStream(checkpointLocation = checkpoint.getCanonicalPath),
+
+        AddData(dedupeInputData2, ("a", 5, "a"), ("b", 2, "b"), ("c", 9, "c")),
+        CheckNewAnswer(("c", 9, "c"))
+      )
+    }
+  }
+
+  // FIXME: test to disallow changing event time column between TimestampType vs TimestampNTZType

Review Comment:
   OK so that's more about representation, and the underlying value is the same. That said, it seems like TimestampType and TimestampNTZType are interchangeable, as long as they fully understand that comparison of both should happen in sprits of NTZ (TimestampType will be adjusted, but it doesn't mean the underlying value is adjusted. It's still about representation.) 
   
   That said, storing the underlying value using LongType sounds to me as still be valid, till we break the underlying format which would incur a lot more troublesome issues.
   
   cc. @zsxwing 



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


[GitHub] [spark] HeartSaVioR commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:
##########
@@ -980,3 +1022,65 @@ object StreamingDeduplicateExec {
   private val EMPTY_ROW =
     UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
 }
+
+case class StreamingDeduplicateWithinWatermarkExec(
+    keyExpressions: Seq[Attribute],
+    child: SparkPlan,
+    stateInfo: Option[StatefulOperatorStateInfo] = None,
+    eventTimeWatermarkForLateEvents: Option[Long] = None,
+    eventTimeWatermarkForEviction: Option[Long] = None)
+  extends BaseStreamingDeduplicateExec {
+
+  protected val schemaForValueRow: StructType = StructType(
+    Array(StructField("expiresAt", LongType, nullable = false)))

Review Comment:
   It actually came from flatMapGroupsWithState but now I get your question in offline. Ideally saying, we should probably use the same type the event time column has, and disallow changing type of the event time column.
   
   While we can try to address the issue in this PR, it'd probably require another state format for other stateful operators if they have the same issue. (I'm seeing the same issue from flatMapGroupsWithState but not from others, hopefully.)



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40561: [SPARK-42931][SS] Introduce dropDuplicatesWithinWatermark

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


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3038,6 +3025,107 @@ class Dataset[T] private[sql](
     dropDuplicates(colNames)
   }
 
+  /**
+   * Returns a new Dataset with duplicates rows removed, within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(): Dataset[T] = {
+    dropDuplicatesWithinWatermark(this.columns)
+  }
+
+  /**
+   * Returns a new Dataset with duplicates rows removed, considering only the subset of columns,
+   * within watermark.
+   *
+   * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]],
+   * this will keep all data across triggers as intermediate state to drop duplicated rows. The
+   * state will be kept to guarantee the semantic, "Events are deduplicated as long as the time
+   * distance of earliest and latest events are smaller than the delay threshold of watermark."
+   * The watermark for the input [[Dataset]] must be set via [[withWatermark]]. Users are
+   * encouraged to set the delay threshold of watermark longer than max timestamp differences
+   * among duplicated events. In addition, too late data older than watermark will be dropped.
+   *
+   * @group typedrel
+   * @since 3.5.0
+   */
+  def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = {

Review Comment:
   Should we maybe have `dropDuplicatesWithinWatermark(cols: String*)` one alone? Uses can invoke `dropDuplicatesWithinWatermark(Seq(...): _*)`. That will automatically create `dropDuplicatesWithinWatermark(colNames: Array[String])` signature too (with `@scala.annotation.varargs`) for Java IIRC



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