tdas commented on a change in pull request #33093: URL: https://github.com/apache/spark/pull/33093#discussion_r662309555
########## 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") { Review comment: Java API test must be in a Java file. They are in JavaDatasetSuite ########## 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._ Review comment: You also need Java API tests. -- 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