alex-balikov commented on code in PR #38405:
URL: https://github.com/apache/spark/pull/38405#discussion_r1007348475


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.functions._
+
+// Tests for the multiple stateful operators support.
+class MultiStatefulOperatorsSuite
+  extends StreamTest with StateStoreMetricsTest with BeforeAndAfter {
+
+  import testImplicits._
+
+  before {
+    SparkSession.setActiveSession(spark) // set this before force initializing 
'joinExec'
+    spark.streams.stateStoreCoordinator // initialize the lazy coordinator
+  }
+
+  after {
+    StateStore.stop()
+  }
+
+  test("window agg -> window agg, append mode") {
+    withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") {
+      val inputData = MemoryStream[Int]
+
+      val stream = inputData.toDF()
+        .withColumn("eventTime", timestamp_seconds($"value"))
+        .withWatermark("eventTime", "0 seconds")
+        .groupBy(window($"eventTime", "5 seconds") as 'window)
+        .agg(count("*") as 'count)
+        .groupBy(window($"window", "10 seconds"))
+        .agg(count("*") as 'count, sum("count") as 'sum)
+        .select($"window".getField("start").cast("long").as[Long],
+          $"count".as[Long], $"sum".as[Long])
+
+      testStream(stream)(
+        AddData(inputData, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21),
+        // op1 W (0, 0)
+        // agg: [10, 15) 5, [15, 20) 5, [20, 25) 2
+        // output: None
+        // state: [10, 15) 5, [15, 20) 5, [20, 25) 2
+        // op2 W (0, 0)
+        // agg: None
+        // output: None
+        // state: None
+
+        // no-data batch triggered
+
+        // op1 W (0, 21)
+        // agg: None
+        // output: [10, 15) 5, [15, 20) 5
+        // state: [20, 25) 2
+        // op2 W (0, 21)
+        // agg: [10, 20) (2, 10)
+        // output: [10, 20) (2, 10)
+        // state: None
+        CheckNewAnswer((10, 2, 10)), // W (21, 21) [20, 25) 2 ... W (0, 21) 
None // [10, 20) 2
+        assertNumStateRows(Seq(0, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0)),
+
+        AddData(inputData, 22, 23, 24, 25, 26, 27, 28, 29),
+        // op1 W (21, 21)
+        // agg: [20, 25) 5, [25, 30) 4
+        // output: None
+        // state: [20, 25) 5, [25, 30) 4
+        // op2 W (21, 21)
+        // agg: None
+        // output: None
+        // state: None
+
+        // no-data batch triggered
+
+        // op1 W (21, 29)
+        // agg: None
+        // output: [20, 25) 5
+        // state: [25, 30) 4
+        // op2 W (20, 25)
+        // agg: [20, 30) (1, 5)
+        // output: None
+        // state: [20, 30) (1, 5)
+        CheckNewAnswer(),
+        assertNumStateRows(Seq(1, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0)),
+
+        // Move the watermark.
+        AddData(inputData, 30, 31),
+        // op1 W (29, 29)
+        // agg: [25, 30) 5 [30, 35) 2
+        // output: None
+        // state: [25, 30) 5 [30, 35) 2
+        // op2 W (29, 29)
+        // agg: None
+        // output: None
+        // state: [20, 30) (1, 5)
+
+        // no-data batch triggered
+
+        // op1 W (29, 31)
+        // agg: None
+        // output: [25, 30) 5
+        // state: [30, 35) 2
+        // op2 W (29, 31)
+        // agg: [20, 30) (1, 5)
+        // output: [20, 30) (1, 5)
+        // state: None
+        CheckNewAnswer((20, 2, 10)),
+        assertNumStateRows(Seq(0, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0))
+      )
+    }
+  }
+
+  test("agg -> agg -> agg, append mode") {
+    withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") {
+      val inputData = MemoryStream[Int]
+
+      val stream = inputData.toDF()
+        .withColumn("eventTime", timestamp_seconds($"value"))
+        .withWatermark("eventTime", "0 seconds")
+        .groupBy(window($"eventTime", "5 seconds") as 'window)
+        .agg(count("*") as 'count)
+        .groupBy(window(window_time($"window"), "10 seconds"))
+        .agg(count("*") as 'count, sum("count") as 'sum)
+        .groupBy(window(window_time($"window"), "20 seconds"))
+        .agg(count("*") as 'count, sum("sum") as 'sum)
+        .select(
+          $"window".getField("start").cast("long").as[Long],
+          $"window".getField("end").cast("long").as[Long],
+          $"count".as[Long], $"sum".as[Long])
+
+      testStream(stream)(
+        AddData(inputData, 0 to 37: _*),
+        // op1 W (0, 0)
+        // agg: [0, 5) 5, [5, 10) 5, [10, 15) 5, [15, 20) 5, [20, 25) 5, [25, 
30) 5, [30, 35) 5,
+        //   [35, 40) 3
+        // output: None
+        // state: [0, 5) 5, [5, 10) 5, [10, 15) 5, [15, 20) 5, [20, 25) 5, 
[25, 30) 5, [30, 35) 5,
+        //   [35, 40) 3
+        // op2 W (0, 0)
+        // agg: None
+        // output: None
+        // state: None
+
+        // no-data batch triggered
+
+        // op1 W (37, 37)
+        // agg: None
+        // output: [0, 5) 5, [5, 10) 5, [10, 15) 5, [15, 20) 5, [20, 25) 5, 
[25, 30) 5, [30, 35) 5
+        // state: [35, 40) 3
+        // op2 W (0, 37)
+        // agg: [0, 10) (2, 10), [10, 20) (2, 10), [20, 30) (2, 10), [30, 40) 
(1, 5)
+        // output: [0, 10) (2, 10), [10, 20) (2, 10), [20, 30) (2, 10)
+        // state: [30, 40) (1, 5)
+        // op3 W (0, 37)
+        // agg: [0, 20) (2, 20), [20, 40) (1, 10)
+        // output: [0, 20) (2, 20)
+        // state: [20, 40) (1, 10)
+        CheckNewAnswer((0, 20, 2, 20)),
+        assertNumStateRows(Seq(1, 1, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0, 0)),
+
+        AddData(inputData, 30 to 60: _*),
+        // op1 W (37, 37)
+        // dropped rows: [30, 35), 1 row <= note that 35, 36, 37 are still in 
effect
+        // agg: [35, 40) 8, [40, 45) 5, [45, 50) 5, [50, 55) 5, [55, 60) 5, 
[60, 65) 1
+        // output: None
+        // state: [35, 40) 8, [40, 45) 5, [45, 50) 5, [50, 55) 5, [55, 60) 5, 
[60, 65) 1
+        // op2 W (37, 37)
+        // output: None
+        // state: [30, 40) (1, 8)
+        // op3 W (37, 37)
+        // output: None
+        // state: [20, 40) (1, 10)
+
+        // no-data batch
+        // op1 W (60, 60)
+        // output: [35, 40) 8, [40, 45) 5, [45, 50) 5, [50, 55) 5, [55, 60) 5
+        // state: [60, 65) 1
+        // op2 W (60, 60)
+        // agg: [30, 40) (2, 13), [40, 50) (2, 10), [50, 60), (2, 10)
+        // output: [30, 40) (2, 13), [40, 50) (2, 10), [50, 60), (2, 10)
+        // state: None
+        // op3 W (60, 60)
+        // agg: [20, 40) (2, 23), [40, 60) (2, 20)
+        // output: [20, 40) (2, 23), [40, 60) (2, 20)
+        // state: None
+
+        CheckNewAnswer((20, 40, 2, 23), (40, 60, 2, 20)),
+        assertNumStateRows(Seq(0, 0, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0, 1))
+      )
+    }
+  }
+
+  test("stream deduplication -> aggregation, append mode") {
+    withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") {
+      val inputData = MemoryStream[Int]
+
+      val deduplication = inputData.toDF()
+        .withColumn("eventTime", timestamp_seconds($"value"))
+        .withWatermark("eventTime", "10 seconds")
+        .dropDuplicates("value", "eventTime")
+
+      val windowedAggregation = deduplication
+        .groupBy(window($"eventTime", "5 seconds") as 'window)
+        .agg(count("*") as 'count, sum("value") as 'sum)
+        .select($"window".getField("start").cast("long").as[Long],
+          $"count".as[Long])
+
+      testStream(windowedAggregation)(
+        // FIXME: we should revisit our watermark condition... we don't allow 
watermark to be

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



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala:
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.functions._
+
+// Tests for the multiple stateful operators support.
+class MultiStatefulOperatorsSuite
+  extends StreamTest with StateStoreMetricsTest with BeforeAndAfter {
+
+  import testImplicits._
+
+  before {
+    SparkSession.setActiveSession(spark) // set this before force initializing 
'joinExec'
+    spark.streams.stateStoreCoordinator // initialize the lazy coordinator
+  }
+
+  after {
+    StateStore.stop()
+  }
+
+  test("window agg -> window agg, append mode") {
+    withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") {
+      val inputData = MemoryStream[Int]
+
+      val stream = inputData.toDF()
+        .withColumn("eventTime", timestamp_seconds($"value"))
+        .withWatermark("eventTime", "0 seconds")
+        .groupBy(window($"eventTime", "5 seconds") as 'window)
+        .agg(count("*") as 'count)
+        .groupBy(window($"window", "10 seconds"))
+        .agg(count("*") as 'count, sum("count") as 'sum)
+        .select($"window".getField("start").cast("long").as[Long],
+          $"count".as[Long], $"sum".as[Long])
+
+      testStream(stream)(
+        AddData(inputData, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21),
+        // op1 W (0, 0)
+        // agg: [10, 15) 5, [15, 20) 5, [20, 25) 2
+        // output: None
+        // state: [10, 15) 5, [15, 20) 5, [20, 25) 2
+        // op2 W (0, 0)
+        // agg: None
+        // output: None
+        // state: None
+
+        // no-data batch triggered
+
+        // op1 W (0, 21)
+        // agg: None
+        // output: [10, 15) 5, [15, 20) 5
+        // state: [20, 25) 2
+        // op2 W (0, 21)
+        // agg: [10, 20) (2, 10)
+        // output: [10, 20) (2, 10)
+        // state: None
+        CheckNewAnswer((10, 2, 10)), // W (21, 21) [20, 25) 2 ... W (0, 21) 
None // [10, 20) 2
+        assertNumStateRows(Seq(0, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0)),
+
+        AddData(inputData, 22, 23, 24, 25, 26, 27, 28, 29),
+        // op1 W (21, 21)
+        // agg: [20, 25) 5, [25, 30) 4
+        // output: None
+        // state: [20, 25) 5, [25, 30) 4
+        // op2 W (21, 21)
+        // agg: None
+        // output: None
+        // state: None
+
+        // no-data batch triggered
+
+        // op1 W (21, 29)
+        // agg: None
+        // output: [20, 25) 5
+        // state: [25, 30) 4
+        // op2 W (20, 25)
+        // agg: [20, 30) (1, 5)
+        // output: None
+        // state: [20, 30) (1, 5)
+        CheckNewAnswer(),
+        assertNumStateRows(Seq(1, 1)),
+        assertNumRowsDroppedByWatermark(Seq(0, 0)),
+
+        // Move the watermark.

Review Comment:
   Not sure what you mean - unfortunately there always will be state left as 
the watermark is up to the latest event time and that row will not be flushed. 
There are 3 microbatches in this test + 3 no-data microbatches. Why would 
another one help?



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to