Github user kunalkhamar commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17216#discussion_r106033678
  
    --- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala ---
    @@ -389,6 +390,61 @@ class StreamSuite extends StreamTest {
         query.stop()
         assert(query.exception.isEmpty)
       }
    +
    +  test("SPARK-19873: streaming aggregation with change in number of 
partitions") {
    +    val inputData = MemoryStream[(Int, Int)]
    +    val agg = inputData.toDS().groupBy("_1").count()
    +
    +    testStream(agg, OutputMode.Complete())(
    +      AddData(inputData, (1, 0), (2, 0)),
    +      StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> 
"2")),
    +      CheckAnswer((1, 1), (2, 1)),
    +      StopStream,
    +      AddData(inputData, (3, 0), (2, 0)),
    +      StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> 
"5")),
    +      CheckAnswer((1, 1), (2, 2), (3, 1)),
    +      StopStream,
    +      AddData(inputData, (3, 0), (1, 0)),
    +      StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> 
"1")),
    +      CheckAnswer((1, 2), (2, 2), (3, 2)))
    +  }
    +
    +  test("SPARK-19873: backward compat with checkpoints that do not record 
shuffle partitions") {
    +    val inputData = MemoryStream[Int]
    +    inputData.addData(1, 2, 3, 4)
    +    inputData.addData(3, 4, 5, 6)
    +    inputData.addData(5, 6, 7, 8)
    +
    +    val resourceUri =
    +      
this.getClass.getResource("/structured-streaming/checkpoint-version-2.1.0").toURI
    +    val checkpointDir = new File(resourceUri).getCanonicalPath
    +    val query = inputData
    +      .toDF()
    +      .groupBy($"value")
    +      .agg(count("*"))
    +      .writeStream
    +      .queryName("counts")
    +      .outputMode("complete")
    +      .option("checkpointLocation", checkpointDir)
    +      .format("memory")
    +
    +    // Checkpoint data was generated by a query with 10 shuffle partitions.
    +    // Test if recovery from checkpoint is successful.
    +    withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") {
    +      query.start().processAllAvailable()
    +
    +      QueryTest.checkAnswer(spark.table("counts").toDF(),
    +        Row("1", 1) :: Row("2", 1) :: Row("3", 2) :: Row("4", 2) ::
    +        Row("5", 2) :: Row("6", 2) :: Row("7", 1) :: Row("8", 1) :: Nil)
    +    }
    +
    +    // If the number of partitions is greater, should throw exception.
    +    withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "15") {
    --- End diff --
    
    Seems okay to me. Underlying cause is `FileNotFoundException`. Error 
message indicates _Error reading delta file 
/Users/path/to/checkpoint/state/[operator]/[partition]/[batch].delta_
    > [info] - SPARK-19873: backward compatibility - recover with wrong num 
shuffle partitions *** FAILED *** (12 seconds, 98 milliseconds)
    [info]   org.apache.spark.sql.streaming.StreamingQueryException: Query 
badQuery [id = dddc5e7f-1e71-454c-8362-de184444fb5a, runId = 
b2960c74-257a-4eb1-b242-61d13e20655f] terminated with exception: Job aborted 
due to stage failure: Task 10 in stage 1.0 failed 1 times, most recent failure: 
Lost task 10.0 in stage 1.0 (TID 11, localhost, executor driver): 
java.lang.IllegalStateException: Error reading delta file 
/Users/kunalkhamar/spark/target/tmp/spark-2816c3be-610f-450c-a821-6d0c68a12d91/state/0/10/1.delta
 of HDFSStateStoreProvider[id = (op=0, part=10), dir = 
/Users/kunalkhamar/spark/target/tmp/spark-2816c3be-610f-450c-a821-6d0c68a12d91/state/0/10]:
 
/Users/kunalkhamar/spark/target/tmp/spark-2816c3be-610f-450c-a821-6d0c68a12d91/state/0/10/1.delta
 does not exist
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$updateFromDeltaFile(HDFSBackedStateStoreProvider.scala:384)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$$anonfun$org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$loadMap$1$$anonfun$6.apply(HDFSBackedStateStoreProvider.scala:336)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$$anonfun$org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$loadMap$1$$anonfun$6.apply(HDFSBackedStateStoreProvider.scala:333)
    [info]      at scala.Option.getOrElse(Option.scala:121)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$$anonfun$org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$loadMap$1.apply(HDFSBackedStateStoreProvider.scala:333)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$$anonfun$org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$loadMap$1.apply(HDFSBackedStateStoreProvider.scala:332)
    [info]      at scala.Option.getOrElse(Option.scala:121)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$loadMap(HDFSBackedStateStoreProvider.scala:332)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.getStore(HDFSBackedStateStoreProvider.scala:239)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.StateStore$.get(StateStore.scala:191)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:61)
    [info]      at 
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
    [info]      at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
    [info]      at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
    [info]      at 
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
    [info]      at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
    [info]      at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
    [info]      at 
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
    [info]      at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
    [info]      at 
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
    [info]      at org.apache.spark.scheduler.Task.run(Task.scala:108)
    [info]      at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:317)
    [info]      at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
    [info]      at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
    [info]      at java.lang.Thread.run(Thread.java:745)
    [info] Caused by: java.io.FileNotFoundException: File 
/Users/kunalkhamar/spark/target/tmp/spark-2816c3be-610f-450c-a821-6d0c68a12d91/state/0/10/1.delta
 does not exist
    [info]      at 
org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:539)
    [info]      at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:752)
    [info]      at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:529)
    [info]      at 
org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:409)
    [info]      at 
org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>(ChecksumFileSystem.java:142)
    [info]      at 
org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:346)
    [info]      at 
org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:61)
    [info]      at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:766)
    [info]      at 
org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$updateFromDeltaFile(HDFSBackedStateStoreProvider.scala:381)
    [info]      ... 24 more



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to