viirya commented on a change in pull request #27333:
URL: https://github.com/apache/spark/pull/27333#discussion_r511492575



##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
##########
@@ -1019,6 +1019,56 @@ class FlatMapGroupsWithStateSuite extends 
StateStoreMetricsTest {
       spark.createDataset(Seq(("a", 2), ("b", 1))).toDF)
   }
 
+  testWithAllStateVersions("SPARK-29438: ensure UNION doesn't lead 
(flat)MapGroupsWithState" +
+    " to use shifted partition IDs") {
+    val stateFunc = (key: String, values: Iterator[String], state: 
GroupState[RunningCount]) => {
+      val count = state.getOption.map(_.count).getOrElse(0L) + values.size
+      state.update(RunningCount(count))
+      (key, count.toString)
+    }
+
+    def constructUnionDf(desiredPartitionsForInput1: Int)
+      : (MemoryStream[String], MemoryStream[String], DataFrame) = {
+      val input1 = MemoryStream[String](desiredPartitionsForInput1)
+      val input2 = MemoryStream[String]
+      val df1 = input1.toDF()
+        .select($"value", $"value")
+      val df2 = input2.toDS()
+        .groupByKey(x => x)
+        .mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, 
Str)
+        .toDF()
+
+      // Unioned DF would have columns as (String, String)
+      (input1, input2, df1.union(df2))
+    }
+
+    withTempDir { checkpointDir =>
+      val (input1, input2, unionDf) = constructUnionDf(2)
+      testStream(unionDf, Update)(
+        StartStream(checkpointLocation = checkpointDir.getAbsolutePath),
+        MultiAddData(input1, "input1-a")(input2, "input2-a"),
+        CheckNewAnswer(("input1-a", "input1-a"), ("input2-a", "1")),
+        StopStream
+      )
+
+      // We're restoring the query with different number of partitions in left 
side of UNION,
+      // which may lead right side of union to have mismatched partition IDs 
(e.g. if it relies on
+      // TaskContext.partitionId()). This test will verify 
(flat)MapGroupsWithState doesn't have
+      // such issue.
+
+      val (newInput1, newInput2, newUnionDf) = constructUnionDf(3)
+
+      newInput1.addData("input1-a")
+      newInput2.addData("input2-a")

Review comment:
       Oh I got it. We reuse the same checkpoint directory and so we need to 
fill same batches of previous stream query. Actually we even don't need to put 
the same data.
   
   Then the new query begins from the new added data, and read the same state 
store.




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

Reply via email to