anishshri-db commented on code in PR #49277: URL: https://github.com/apache/spark/pull/49277#discussion_r1902170182
########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala: ########## @@ -629,6 +752,197 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } + test("transformWithState - upcasting should succeed") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorInt(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { q => + assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) + assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") == 0) + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + }, + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "2"), ("b", "1")), + StopStream, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckNewAnswer(("b", "2")), + StopStream, + Execute { q => + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + assert(q.lastProgress.stateOperators(0).numRowsRemoved === 1) + }, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckNewAnswer(("a", "1"), ("c", "1")) + ) + + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessor(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), + AddData(inputData, "d"), + CheckNewAnswer(("d", "1")), + StopStream + ) + } + } + } + + test("transformWithState - reordering fields should succeed") { + withSQLConf( + SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + + // First run with initial field order + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorInitialOrder(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + StopStream + ) + + // Second run with reordered fields + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorReorderedFields(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), // Should continue counting from previous state + StopStream + ) + } + } + } + + test("transformWithState - adding field should succeed") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessor(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { q => + assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) + assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") == 0) + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + }, + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "2"), ("b", "1")), + StopStream, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckNewAnswer(("b", "2")), + StopStream, + Execute { q => + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + assert(q.lastProgress.stateOperators(0).numRowsRemoved === 1) + }, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckNewAnswer(("a", "1"), ("c", "1")) + ) + + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorNestedLongs(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), + StopStream + ) + } + } + } + + test("transformWithState - removing field should succeed") { Review Comment: We also need to add negative test cases ? basically where new schema is not compatible with old ? - downcast - renaming fields etc - ########## sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala: ########## @@ -629,6 +752,197 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } + test("transformWithState - upcasting should succeed") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorInt(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { q => + assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) + assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") == 0) + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + }, + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "2"), ("b", "1")), + StopStream, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckNewAnswer(("b", "2")), + StopStream, + Execute { q => + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + assert(q.lastProgress.stateOperators(0).numRowsRemoved === 1) + }, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckNewAnswer(("a", "1"), ("c", "1")) + ) + + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessor(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), + AddData(inputData, "d"), + CheckNewAnswer(("d", "1")), + StopStream + ) + } + } + } + + test("transformWithState - reordering fields should succeed") { + withSQLConf( + SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + + // First run with initial field order + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorInitialOrder(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + StopStream + ) + + // Second run with reordered fields + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorReorderedFields(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), // Should continue counting from previous state + StopStream + ) + } + } + } + + test("transformWithState - adding field should succeed") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { chkptDir => + val dirPath = chkptDir.getCanonicalPath + val inputData = MemoryStream[String] + val result1 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessor(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result1, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "1")), + Execute { q => + assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) + assert(q.lastProgress.stateOperators(0).customMetrics.get("numRegisteredTimers") == 0) + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + }, + AddData(inputData, "a", "b"), + CheckNewAnswer(("a", "2"), ("b", "1")), + StopStream, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckNewAnswer(("b", "2")), + StopStream, + Execute { q => + assert(q.lastProgress.stateOperators(0).numRowsUpdated === 1) + assert(q.lastProgress.stateOperators(0).numRowsRemoved === 1) + }, + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckNewAnswer(("a", "1"), ("c", "1")) + ) + + val result2 = inputData.toDS() + .groupByKey(x => x) + .transformWithState(new RunningCountStatefulProcessorNestedLongs(), + TimeMode.None(), + OutputMode.Update()) + + testStream(result2, OutputMode.Update())( + StartStream(checkpointLocation = dirPath), + AddData(inputData, "a"), + CheckNewAnswer(("a", "2")), + StopStream + ) + } + } + } + + test("transformWithState - removing field should succeed") { Review Comment: We also need to add negative test cases ? basically where new schema is not compatible with old ? - downcast - renaming fields etc -- 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