jingz-db commented on code in PR #48686:
URL: https://github.com/apache/spark/pull/48686#discussion_r1830097511


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala:
##########
@@ -497,4 +587,121 @@ class TransformWithStateInitialStateSuite extends 
StateStoreMetricsTest
       )
     }
   }
+
+  testInitialStateWithStateDataSource(true) { (valDf, listDf, mapDf, 
inputData) =>
+    val valueDf = valDf.selectExpr("key.value AS groupingKey", "value.value AS 
value")
+    val flattenListDf = listDf
+      .selectExpr("key.value AS groupingKey", "list_element.value AS 
listValue")
+    val flattenMapDf = mapDf
+      .selectExpr(
+        "key.value AS groupingKey",
+        "user_map_key.value AS userMapKey",
+        "user_map_value.value AS userMapValue")
+    val df_joined =
+      valueDf.unionByName(flattenListDf, true)
+        .unionByName(flattenMapDf, true)
+    val kvDataSet = inputData.toDS().groupByKey(x => x.key)
+    val initDf = df_joined.as[UnionInitialStateRow].groupByKey(x => 
x.groupingKey)
+    kvDataSet.transformWithState(
+      new InitialStatefulProcessorWithStateDataSource(),
+      TimeMode.None(), OutputMode.Append(), initDf).toDF()
+  }
+
+  testInitialStateWithStateDataSource(false) { (valDf, listDf, mapDf, 
inputData) =>
+    val valueDf = valDf.selectExpr("key.value AS groupingKey", "value.value AS 
value")
+    val unflattenListDf = listDf
+      .selectExpr("key.value AS groupingKey",
+        "list_value.value as listValue")
+    val unflattenMapDf = mapDf
+      .selectExpr(
+        "key.value AS groupingKey",
+        "map_from_entries(transform(map_entries(map_value), x -> " +
+          "struct(x.key.value, x.value.value))) as mapValue")
+    val df_joined =
+      valueDf.unionByName(unflattenListDf, true)
+        .unionByName(unflattenMapDf, true)
+    val kvDataSet = inputData.toDS().groupByKey(x => x.key)
+    val initDf = df_joined.as[UnionUnflattenInitialStateRow].groupByKey(x => 
x.groupingKey)
+    kvDataSet.transformWithState(
+      new InitialStatefulProcessorWithUnflattenStateDataSource(),
+      TimeMode.None(), OutputMode.Append(), initDf).toDF()
+  }
+
+  private def testInitialStateWithStateDataSource(
+      flattenOption: Boolean)
+      (startQuery: (DataFrame, DataFrame, DataFrame,
+        MemoryStream[InitInputRow]) => DataFrame): Unit = {
+    Seq(("5", "2"), ("5", "8"), ("5", "5")).foreach { partitions =>
+      test("transformWithStateWithInitialState - state data source reader 
dataframe " +
+        s"as initial state with flatten option set to $flattenOption, the 
first stream and " +
+        s"the second stream is running on shuffle partition number of 
${partitions._1} and " +
+        s"${partitions._2} respectively.") {
+        withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+          classOf[RocksDBStateStoreProvider].getName) {
+          withTempDir { checkpointDir =>
+            SQLConf.get.setConfString(SQLConf.SHUFFLE_PARTITIONS.key, 
partitions._1)
+            val inputData = MemoryStream[String]
+            val result = inputData.toDS()
+              .groupByKey(x => x)
+              .transformWithState(new StatefulProcessorWithAllStateVars(),
+                TimeMode.None(),
+                OutputMode.Update())
+
+            testStream(result, OutputMode.Update())(
+              StartStream(checkpointLocation = checkpointDir.getCanonicalPath),
+              AddData(inputData, "a", "b"),
+              CheckNewAnswer(("a", "1"), ("b", "1")),
+              AddData(inputData, "a", "b", "a"),
+              CheckNewAnswer(("a", "3"), ("b", "2"))
+            )
+
+            // state data source reader for state vars
+            val valueDf = spark.read
+              .format("statestore")
+              .option(StateSourceOptions.PATH, checkpointDir.getAbsolutePath)
+              .option(StateSourceOptions.STATE_VAR_NAME, "countState")
+              .load()
+
+            val listDf = spark.read
+              .format("statestore")
+              .option(StateSourceOptions.PATH, checkpointDir.getAbsolutePath)
+              .option(StateSourceOptions.STATE_VAR_NAME, "listState")
+              .option(StateSourceOptions.FLATTEN_COLLECTION_TYPES, 
flattenOption)
+              .load()
+
+            val mapDf = spark.read
+              .format("statestore")
+              .option(StateSourceOptions.PATH, checkpointDir.getAbsolutePath)
+              .option(StateSourceOptions.STATE_VAR_NAME, "mapState")
+              .option(StateSourceOptions.FLATTEN_COLLECTION_TYPES, 
flattenOption)
+              .load()
+
+            // create a df where each row contains all value, list, map state 
rows

Review Comment:
   I just modified the second stateful processor to the simplest form of 
updating state variables to count of keys. I guess one reason that the suite is 
complicated is that we need to have coverage for value, list, map state 
variables in a single processor to have the coverage on all composite types. Do 
you think we should only keep it simple by only testing on one of the state 
variable?



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