HeartSaVioR commented on a change in pull request #29256:
URL: https://github.com/apache/spark/pull/29256#discussion_r471936773
##########
File path:
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
##########
@@ -1106,6 +1107,54 @@ class StreamingQuerySuite extends StreamTest with
BeforeAndAfter with Logging wi
}
}
+ test("union in streaming query of append mode without watermark") {
+ val inputData1 = MemoryStream[Int]
+ val inputData2 = MemoryStream[Int]
+ withTempView("s1", "s2") {
+ inputData1.toDF().createOrReplaceTempView("s1")
+ inputData2.toDF().createOrReplaceTempView("s2")
+ val unioned = spark.sql(
+ "select s1.value from s1 union select s2.value from s2")
+ checkExceptionMessage(unioned)
+ }
+ }
+
+ test("distinct in streaming query of append mode without watermark") {
+ val inputData = MemoryStream[Int]
+ withTempView("deduptest") {
+ inputData.toDF().toDF("value").createOrReplaceTempView("deduptest")
+ val distinct = spark.sql("select distinct value from deduptest")
+ checkExceptionMessage(distinct)
+ }
+ }
+
+ test("distinct in streaming query of complete mode") {
+ val inputData = MemoryStream[Int]
+ withTempView("deduptest") {
+ inputData.toDF().toDF("value").createOrReplaceTempView("deduptest")
+ val distinct = spark.sql("select distinct value from deduptest")
+
+ testStream(distinct, Complete)(
+ AddData(inputData, 1, 2, 3, 3, 4),
+ CheckAnswer(Row(1), Row(2), Row(3), Row(4))
Review comment:
http://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#unsupported-operations
It has been clear on the structured streaming guide doc - `distinct` is
unsupported. I'm not sure this patch also enables the case from DataFrame API.
If it fixes the DF case, worth to update the doc. If not, this will be another
case of "known inconsistency".
So I think we have been trying to restrict the operations which don't make
sense or error prone in streaming Datasets (unbounded input) - that has been
working with DataFrame API (I think that's the reason we have "specific"
methods like union which doesn't work like SQL-like union and dropDuplicates
which ignores output mode), and is broken with allowing SQL statements in
streaming Datasets.
@tdas @zsxwing @jose-torres @brkyvz Would like to hear your thought about
this as well, as probably I'm missing something here on the purpose.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]