[ https://issues.apache.org/jira/browse/SPARK-28699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16905250#comment-16905250 ]
Yuanjian Li commented on SPARK-28699: ------------------------------------- The current [approach|https://github.com/apache/spark/pull/25420] just a bandage fix for returning the wrong answer. After we finish the work of indeterminate stage rerunning(SPARK-25341), we can fix this by unpersisting the original RDD and rerunning the cached indeterminate stage. Gives a preview codebase [here|https://github.com/xuanyuanking/spark/tree/SPARK-28699-RERUN]. > Cache an indeterminate RDD could lead to incorrect result while stage rerun > --------------------------------------------------------------------------- > > Key: SPARK-28699 > URL: https://issues.apache.org/jira/browse/SPARK-28699 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 3.0.0 > Reporter: Yuanjian Li > Priority: Major > > Related with SPARK-23207 SPARK-23243 > It's another case for the indeterminate stage/RDD rerun while stage rerun > happened. In the CachedRDDBuilder, we miss tracking the `isOrderSensitive` > characteristic to the newly created MapPartitionsRDD. > We can reproduce this by the following code, thanks to Tyson for reporting > this! > > {code:scala} > import scala.sys.process._ > import org.apache.spark.TaskContext > val res = spark.range(0, 10000 * 10000, 1).map\{ x => (x % 1000, x)} > // kill an executor in the stage that performs repartition(239) > val df = res.repartition(113).cache.repartition(239).map { x => > if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && > TaskContext.get.stageAttemptNumber == 0) { > throw new Exception("pkill -f -n java".!!) > } > x > } > val r2 = df.distinct.count() > {code} -- This message was sent by Atlassian JIRA (v7.6.14#76016) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org