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

    https://github.com/apache/spark/pull/17024#discussion_r111836700
  
    --- Diff: core/src/test/scala/org/apache/spark/CheckpointSuite.scala ---
    @@ -266,13 +309,44 @@ class CheckpointSuite extends SparkFunSuite with 
RDDCheckpointTester with LocalS
       override def sparkContext: SparkContext = sc
     
       runTest("basic checkpointing") { reliableCheckpoint: Boolean =>
    -    val parCollection = sc.makeRDD(1 to 4)
    -    val flatMappedRDD = parCollection.flatMap(x => 1 to x)
    -    checkpoint(flatMappedRDD, reliableCheckpoint)
    -    assert(flatMappedRDD.dependencies.head.rdd === parCollection)
    -    val result = flatMappedRDD.collect()
    -    assert(flatMappedRDD.dependencies.head.rdd != parCollection)
    -    assert(flatMappedRDD.collect() === result)
    +    startSparkContext()
    +    testBasicCheckpoint(sc, reliableCheckpoint)
    +  }
    +
    +  runTest("compression with snappy", skipLocalCheckpoint = true) { _: 
Boolean =>
    --- End diff --
    
    For the new test, I think we just need one simple test. And if we put it 
into a new suite (e.g., the below example), then we don't need to touch the 
existing codes.
    ```
    class CheckpointCompressionSuite extends SparkFunSuite with 
LocalSparkContext {
    
      test("checkpoint compression") {
        val checkpointDir = File.createTempFile("temp", "", 
Utils.createTempDir())
        try {
          val conf = new SparkConf().set("spark.checkpoint.compress", "true")
          sc = new SparkContext("local", "test", conf)
          sc.setCheckpointDir(checkpointDir.toString)
          val rdd = sc.makeRDD(1 to 20, numSlices = 1)
          rdd.checkpoint()
          assert(rdd.collect().toSeq === (1 to 20))
          val checkpointPath = new Path(rdd.getCheckpointFile.get)
          val fs = checkpointPath.getFileSystem(sc.hadoopConfiguration)
          val checkpointFile =
            
fs.listStatus(checkpointPath).map(_.getPath).find(_.getName.startsWith("part-")).get
    
          // Verify the checkpoint file can be decompressed
          val compressedInputStream = CompressionCodec.createCodec(conf)
            .compressedInputStream(fs.open(checkpointFile))
          ByteStreams.toByteArray(compressedInputStream)
    
          // Verify that the compressed content can be read back
          assert(rdd.collect().toSeq === (1 to 20))
        } finally {
          Utils.deleteRecursively(checkpointDir)
        }
      }
    }
    ```


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to