tdas commented on a change in pull request #33093:
URL: https://github.com/apache/spark/pull/33093#discussion_r662311649



##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
##########
@@ -1268,12 +1269,298 @@ class FlatMapGroupsWithStateSuite extends 
StateStoreMetricsTest {
     assert(e.getMessage === "The output mode of function should be append or 
update")
   }
 
+  import testImplicits._
+
+  /**
+   * FlatMapGroupsWithState function that returns the key, value as passed to 
it
+   * along with the updated state. The state is incremented for every value.
+   */
+  val flatMapGroupsWithStateFunc =
+    (key: String, values: Iterator[String], state: GroupState[RunningCount]) 
=> {
+      val valList = values.toSeq
+      val count = state.getOption.map(_.count).getOrElse(0L) + valList.size
+      state.update(new RunningCount(count))
+      Iterator((key, valList, state.get.count.toString))
+    }
+
+  Seq("1", "2", "6").foreach { shufflePartitions =>
+    testWithAllStateVersions(s"flatMapGroupsWithState - initial " +
+        s"state - all cases - shuffle partitions ${shufflePartitions}") {
+      withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> shufflePartitions) {
+        // We will test them on different shuffle partition configuration to 
make sure the
+        // grouping by key will still work. On higher number of shuffle 
partitions its possible
+        // that all keys end up on different partitions.
+        val initialState: Dataset[(String, RunningCount)] = Seq(
+          ("keyInStateAndData-1", new RunningCount(1)),
+          ("keyInStateAndData-2", new RunningCount(1)),
+          ("keyOnlyInState-1", new RunningCount(2)),
+          ("keyOnlyInState-2", new RunningCount(1))
+        ).toDS()
+
+        val it = initialState.groupByKey(x => x._1).mapValues(_._2)
+        val inputData = MemoryStream[String]
+        val result =
+          inputData.toDS()
+            .groupByKey(x => x)
+            .flatMapGroupsWithState(
+              Update, GroupStateTimeout.NoTimeout, 
it)(flatMapGroupsWithStateFunc)
+
+        testStream(result, Update)(
+          AddData(inputData, "keyOnlyInData", "keyInStateAndData-1"),
+          CheckNewAnswer(
+            ("keyOnlyInState-1", Seq[String](), "2"),
+            ("keyOnlyInState-2", Seq[String](), "1"),
+            ("keyInStateAndData-1", Seq[String]("keyInStateAndData-1"), "2"), 
// inc by 1
+            ("keyInStateAndData-2", Seq[String](), "1"),
+            ("keyOnlyInData", Seq[String]("keyOnlyInData"), "1") // inc by 1
+          ),
+          assertNumStateRows(total = 5, updated = 5),
+          // Stop and Start stream to make sure initial state doesn't get 
applied again.
+          StopStream,
+          StartStream(),
+          AddData(inputData, "keyInStateAndData-2"),
+          CheckNewAnswer(
+            // state incremented by 1
+            ("keyInStateAndData-2", 
ArrayBuffer[String]("keyInStateAndData-2"), "2")
+          ),
+          assertNumStateRows(total = 5, updated = 1),
+          StopStream
+        )
+      }
+    }
+  }
+
+  testQuietly("flatMapGroupsWithState - initial state - duplicate keys") {
+    val initialState = Seq(
+      ("a", new RunningCount(2)),
+      ("a", new RunningCount(1))
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+
+    val inputData = MemoryStream[String]
+    val result =
+      inputData.toDS()
+        .groupByKey(x => x)
+        .flatMapGroupsWithState(Update, NoTimeout(), 
initialState)(flatMapGroupsWithStateFunc)
+    testStream(result, Update)(
+      AddData(inputData, "a"),
+      ExpectFailure[SparkException] { e =>
+        assert(e.getCause.getMessage.contains("The initial state provided 
contained " +
+          "multiple rows(state) with the same key"))
+      }
+    )
+  }
+
+  test("flatMapGroupsWithState - initial state - java api") {
+    val initialState = Seq(
+      ("a", 2)
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+
+    val javaStateFunc = new FlatMapGroupsWithStateFunction[String, String, 
Int, String] {
+      import java.util.{Iterator => JIterator}
+      override def call(
+          key: String,
+          values: JIterator[String],
+          state: GroupState[Int]): JIterator[String] = {
+        state.update(0)
+        new JIterator[String] {
+          override def hasNext: Boolean = false
+          override def next(): String = null
+        }
+      }
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().groupByKey(x => x).flatMapGroupsWithState(
+      javaStateFunc, OutputMode.Update,
+      implicitly[Encoder[Int]], implicitly[Encoder[String]],
+      GroupStateTimeout.NoTimeout, initialState)
+    testStream(result, Update)(
+      AddData(inputData, "b"),
+      CheckNewAnswer(),
+      assertNumStateRows(total = 2, updated = 2)
+    )
+  }
+
+  test("mapGroupsWithState - initial state - java api") {
+    val initialState = Seq(
+      ("a", 4),
+      (null, 2)
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+
+    val javaStateFunc = new MapGroupsWithStateFunction[String, String, Int, 
String] {
+      import java.util.{Iterator => JIterator}
+      override def call(
+          key: String,
+          values: JIterator[String],
+          state: GroupState[Int]): String = {
+        var valSize = 0
+        while (values.hasNext) {
+          valSize += 1
+          values.next()
+        }
+        val count = state.getOption.getOrElse(0)
+        state.update(count + valSize)
+        state.get.toString
+      }
+    }
+
+    val inputData = MemoryStream[String]
+    val result = inputData.toDS().groupByKey(x => x).mapGroupsWithState(
+      javaStateFunc,
+      implicitly[Encoder[Int]], implicitly[Encoder[String]],
+      GroupStateTimeout.NoTimeout, initialState)
+    testStream(result, Update)(
+      AddData(inputData, null),
+      CheckNewAnswer("4", "3"),
+      assertNumStateRows(total = 2, updated = 2)
+    )
+  }
+
+  test("flatMapGroupsWithState - initial state - streaming initial state") {
+    val initialStateData = MemoryStream[(String, RunningCount)]
+    initialStateData.addData(("a", new RunningCount(1)))
+
+    val inputData = MemoryStream[String]
+
+    val result =
+      inputData.toDS()
+        .groupByKey(x => x)
+        .flatMapGroupsWithState(
+          Update, NoTimeout(), 
initialStateData.toDS().groupByKey(_._1).mapValues(_._2)
+        )(flatMapGroupsWithStateFunc)
+
+    val e = intercept[AnalysisException] {
+      result.writeStream
+        .format("console")
+        .start()
+    }
+    assert(e.message.contains("Initial state cannot be a streaming 
DataFrame/Dataset."))
+  }
+
+  testWithAllStateVersions("mapGroupsWithState - initial state - test api") {
+    val mapGroupsWithStateFunc =
+        (key: String, values: Iterator[String], state: 
GroupState[RunningCount]) => {
+      val valList = values.toList
+      val count = state.getOption.map(_.count).getOrElse(0L) + valList.size
+      state.update(new RunningCount(count))
+      (key, valList, state.get.count.toString)
+    }
+    val initialState = Seq(
+      ("key", new RunningCount(5))
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+
+    val inputData = MemoryStream[String]
+    val result =
+      inputData.toDS()
+        .groupByKey(x => x)
+        .mapGroupsWithState(NoTimeout(), initialState)(mapGroupsWithStateFunc)
+    testStream(result, Update)(
+      AddData(inputData, "key", "key"), // 2 values for the same key
+      CheckNewAnswer(("key", ArrayBuffer("key", "key"), "7")), // state is 
incremented by 2
+      assertNumStateRows(total = 1, updated = 1),
+      StopStream
+    )
+  }
+
+  testWithAllStateVersions("flatMapGroupsWithState - initial state - 
processing time timeout") {
+    // function will return -1 on timeout and returns count of the state 
otherwise
+    val stateFunc =
+        (key: String, values: Iterator[(String, Long)], state: 
GroupState[RunningCount]) => {
+      if (state.hasTimedOut) {
+        state.remove()
+        Iterator((key, "-1"))
+      } else {
+        val count = state.getOption.map(_.count).getOrElse(0L) + values.size
+        state.update(RunningCount(count))
+        state.setTimeoutDuration("10 seconds")
+        Iterator((key, count.toString))
+      }
+    }
+
+    val clock = new StreamManualClock
+    val inputData = MemoryStream[(String, Long)]
+    val initialState = Seq(
+      ("c", new RunningCount(2))
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+    val result =
+      inputData.toDF().toDF("key", "time")
+        .selectExpr("key", "timestamp_seconds(time) as timestamp")
+        .withWatermark("timestamp", "10 second")
+        .as[(String, Long)]
+        .groupByKey(x => x._1)
+        .flatMapGroupsWithState(Update, ProcessingTimeTimeout(), 
initialState)(stateFunc)
+
+    testStream(result, Update)(
+      StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+      AddData(inputData, ("a", 1L)),
+      AdvanceManualClock(1 * 1000), // a and c are processed here for the 
first time.
+      CheckNewAnswer(("a", "1"), ("c", "2")),
+      AdvanceManualClock(10 * 1000),
+      AddData(inputData, ("b", 1L)), // this will trigger c and a to get timed 
out
+      AdvanceManualClock(1 * 1000),
+      CheckNewAnswer(("a", "-1"), ("b", "1"), ("c", "-1"))
+    )
+  }
+
+  test("flatMapGroupsWithState - initial state - batch") {
+    val initialState = Seq(
+      ("a", new RunningCount(1))
+    ).toDS().groupByKey(_._1).mapValues(_._2)
+
+    val e = intercept[AnalysisException] {
+      Seq("a", "a", "b").toDS
+        .groupByKey(x => x)
+        .flatMapGroupsWithState(
+          Update, GroupStateTimeout.NoTimeout, 
initialState)(flatMapGroupsWithStateFunc).toDF.show()
+    }
+    assert(e.getMessage.contains("Batch [flatMap|map]GroupsWithState queries 
should not" +
+      " pass an initial state."))
+  }
+
+  testWithAllStateVersions("flatMapGroupsWithState - initial state - case 
class key") {

Review comment:
       I would put this test right after the main test. before all the 
unsupported tests
   Also modify this test to test with primitive values as well .. so that for 
both keys and values, we test primitive and complex types




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