@@ -28,7 +28,7 @@ import org.apache.spark.sql.{Dataset, Encoders, Row}
2828import org .apache .spark .sql .catalyst .util .stringToFile
2929import org .apache .spark .sql .execution .streaming ._
3030import org .apache .spark .sql .execution .streaming .TransformWithStateKeyValueRowSchema .KEY_ROW_SCHEMA
31- import org .apache .spark .sql .execution .streaming .state .{AlsoTestWithChangelogCheckpointingEnabled , ColumnFamilySchema , ColumnFamilySchemaV1 , NoPrefixKeyStateEncoderSpec , RocksDBStateStoreProvider , StatefulProcessorCannotPerformOperationWithInvalidHandleState , StateSchemaV3File , StateStoreMultipleColumnFamiliesNotSupportedException }
31+ import org .apache .spark .sql .execution .streaming .state .{AlsoTestWithChangelogCheckpointingEnabled , ColumnFamilySchema , ColumnFamilySchemaV1 , NoPrefixKeyStateEncoderSpec , OperatorStateMetadataV2 , RocksDBStateStoreProvider , StatefulProcessorCannotPerformOperationWithInvalidHandleState , StateSchemaV3File , StateStoreMultipleColumnFamiliesNotSupportedException }
3232import org .apache .spark .sql .functions .timestamp_seconds
3333import org .apache .spark .sql .internal .SQLConf
3434import org .apache .spark .sql .streaming .util .StreamManualClock
@@ -805,7 +805,7 @@ class TransformWithStateSuite extends StateStoreMetricsTest
805805 getLatest().get._2.
806806 asInstanceOf [OperatorStateMetadataV2 ].
807807 stateStoreInfo.head.stateSchemaFilePath
808- fetchStateSchemaV3File(checkpointDir, operatorId).get (new Path (stateSchemaFilePath))
808+ fetchStateSchemaV3File(checkpointDir, operatorId).getWithPath (new Path (stateSchemaFilePath))
809809 }
810810
811811 private def fetchStateSchemaV3File (
@@ -900,45 +900,6 @@ class TransformWithStateSuite extends StateStoreMetricsTest
900900 }
901901 }
902902
903- test(" transformWithState - verify that StateSchemaV3 files are purged" ) {
904- withSQLConf(SQLConf .STATE_STORE_PROVIDER_CLASS .key ->
905- classOf [RocksDBStateStoreProvider ].getName,
906- SQLConf .SHUFFLE_PARTITIONS .key ->
907- TransformWithStateSuiteUtils .NUM_SHUFFLE_PARTITIONS .toString,
908- SQLConf .MIN_BATCHES_TO_RETAIN .key -> " 1" ) {
909- withTempDir { chkptDir =>
910- val inputData = MemoryStream [String ]
911- val result = inputData.toDS()
912- .groupByKey(x => x)
913- .transformWithState(new RunningCountStatefulProcessor (),
914- TimeMode .None (),
915- OutputMode .Update ())
916-
917- testStream(result, OutputMode .Update ())(
918- StartStream (checkpointLocation = chkptDir.getCanonicalPath),
919- AddData (inputData, " a" ),
920- CheckNewAnswer ((" a" , " 1" )),
921- StopStream ,
922- StartStream (checkpointLocation = chkptDir.getCanonicalPath),
923- AddData (inputData, " a" ),
924- CheckNewAnswer ((" a" , " 2" )),
925- StopStream ,
926- StartStream (checkpointLocation = chkptDir.getCanonicalPath),
927- AddData (inputData, " a" ),
928- CheckNewAnswer (),
929- StopStream
930- )
931- // If the StateSchemaV3 files are not purged, there would be
932- // three files, but we should have only one file.
933- val batchesWithSchemaV3File = fetchStateSchemaV3File(
934- chkptDir.getCanonicalPath, 0 ).listBatchesOnDisk
935- assert(batchesWithSchemaV3File.length == 1 )
936- // Make sure that only the latest batch has the schema file
937- assert(batchesWithSchemaV3File.head == 2 )
938- }
939- }
940- }
941-
942903 test(" transformWithState - verify that OperatorStateMetadataV2" +
943904 " file is being written correctly" ) {
944905 withSQLConf(SQLConf .STATE_STORE_PROVIDER_CLASS .key ->
0 commit comments