jose-torres commented on code in PR #46247:
URL: https://github.com/apache/spark/pull/46247#discussion_r1599009475


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala:
##########
@@ -1364,6 +1364,35 @@ class StreamingQuerySuite extends StreamTest with 
BeforeAndAfter with Logging wi
     )
   }
 
+  test("Collation aware streaming") {
+    withTable("parquet_streaming_tbl") {
+      spark.sql(
+        """
+          |CREATE TABLE parquet_streaming_tbl
+          |(
+          |  key STRING COLLATE UTF8_BINARY_LCASE,
+          |  value_stream INTEGER
+          |) USING parquet""".stripMargin)
+
+      val streamDf = spark.readStream.table("parquet_streaming_tbl")
+      val filteredDf = streamDf.filter("key = 'aaa'")
+
+      val clock = new StreamManualClock()
+      testStream(filteredDf)(
+        StartStream(triggerClock = clock, trigger = 
Trigger.ProcessingTime(100)),
+        Execute { _ =>
+          spark.createDataFrame(Seq("aaa" -> 1, "AAA" -> 2, "bbb" -> 3, "aa" 
-> 4))

Review Comment:
   I think it's important to also test a scenario where the incoming stream has 
a non-default collation itself.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala:
##########
@@ -484,6 +486,52 @@ class StreamingDeduplicationSuite extends 
StateStoreMetricsTest {
       CheckLastBatch(("c", 9, "c"))
     )
   }
+
+  test("collation aware deduplication") {
+    val inputData = MemoryStream[(String, Int)]
+    val result = inputData.toDF()
+      .select(col("_1")
+        .try_cast(StringType("UNICODE")).as("str"),
+        col("_2").as("int"))
+      .dropDuplicates("str")
+
+    testStream(result, Append)(
+      AddData(inputData, "a" -> 1),
+      CheckLastBatch("a" -> 1),
+      assertNumStateRows(total = 1, updated = 1, droppedByWatermark = 0),
+      AddData(inputData, "a" -> 2), // Dropped
+      CheckLastBatch(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 0),
+      // scalastyle:off
+      AddData(inputData, "ä" -> 1),
+      CheckLastBatch("ä" -> 1),
+      // scalastyle:on
+      assertNumStateRows(total = 2, updated = 1, droppedByWatermark = 0)
+    )
+  }
+
+  test("non-binary collation aware deduplication not supported") {
+    val inputData = MemoryStream[(String)]
+    val result = inputData.toDF()
+      .select(col("value")
+        .try_cast(StringType("UTF8_BINARY_LCASE")).as("str"))

Review Comment:
   I'm a bit confused by this - is the test name flipped or is 
UTF8_BINARY_LCASE considered a non-binary collation?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to