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

    https://github.com/apache/spark/pull/10934#discussion_r51330972
  
    --- Diff: 
streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala ---
    @@ -821,6 +821,70 @@ class CheckpointSuite extends TestSuiteBase with 
DStreamCheckpointTester
         checkpointWriter.stop()
       }
     
    +  test("SPARK-6847: stack overflow when updateStateByKey is followed by a 
checkpointed dstream") {
    +    // In this test, there are two updateStateByKey operators. The RDD DAG 
is as follows:
    +    //
    +    //     batch 1            batch 2            batch 3     ...
    +    //
    +    // 1) input rdd          input rdd          input rdd
    +    //       |                  |                  |
    +    // 2) cogroup rdd   ---> cogroup rdd   ---> cogroup rdd  ...
    +    //       |         /        |         /        |
    +    // 3)  map rdd ---        map rdd ---        map rdd     ...
    +    //       |
    +    // 4) cogroup rdd   ---> cogroup rdd   ---> cogroup rdd  ...
    +    //       |         /        |         /        |
    +    // 5)  map rdd ---        map rdd ---        map rdd     ...
    +    //
    +    // Every batch depends on its previous batch, so "updateStateByKey" 
needs to do checkpoint to
    +    // break the RDD chain. However, before SPARK-6847, when the state RDD 
(layer 5) of the second
    +    // "updateStateByKey" does checkpoint, it won't checkpoint the state 
RDD (layer 3) of the first
    +    // "updateStateByKey" (Note: "updateStateByKey" has already marked 
that its state RDD (layer 3)
    +    // should be checkpointed). Hence, the connections between layer 2 and 
layer 3 won't be broken
    +    // and the RDD chain will grow infinitely and cause StackOverflow.
    +    //
    +    // Therefore SPARK-6847 introduces 
"spark.checkpoint.checkpointAllMarked" to force checkpointing
    +    // all marked RDDs in the DAG to resolve this issue. (For the previous 
example, it will break
    +    // connections between layer 2 and layer 3)
    +    ssc = new StreamingContext(master, framework, batchDuration)
    +    val batchCounter = new BatchCounter(ssc)
    +    ssc.checkpoint(checkpointDir)
    +    val inputDStream = new CheckpointInputDStream(ssc)
    +    val updateFunc = (values: Seq[Int], state: Option[Int]) => {
    +      Some(values.sum + state.getOrElse(0))
    +    }
    +    @volatile var checkpointAllMarkedRDDsEnable = false
    +    @volatile var rddsCheckpointed = false
    +    inputDStream.map(i => (i, i))
    +      .updateStateByKey(updateFunc).checkpoint(batchDuration)
    +      .updateStateByKey(updateFunc).checkpoint(batchDuration)
    +      .foreachRDD { rdd =>
    +        checkpointAllMarkedRDDsEnable =
    +          
Option(rdd.sparkContext.getLocalProperty(RDD.CHECKPOINT_ALL_MARKED)).
    +            map(_.toBoolean).getOrElse(false)
    +
    +        val stateRDDs = {
    +          def findAllMarkedRDDs(_rdd: RDD[_], buffer: 
ArrayBuffer[RDD[_]]): Unit = {
    +            if (_rdd.checkpointData.isDefined) {
    +              buffer += _rdd
    +            }
    +            _rdd.dependencies.foreach(dep => findAllMarkedRDDs(dep.rdd, 
buffer))
    +          }
    +
    +          val buffer = new ArrayBuffer[RDD[_]]
    +          findAllMarkedRDDs(rdd, buffer)
    +          buffer.toSeq
    +        }
    --- End diff --
    
    can you extract this to a helper method?


---
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