[ 
https://issues.apache.org/jira/browse/SPARK-27018?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16786984#comment-16786984
 ] 

Marco Gaido commented on SPARK-27018:
-------------------------------------

The PeriodicCheckpointer is still there in master, you can check it on github: 
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala.
 You can just check file histories in git in order to know what and how was 
changed. I'd recommend you, anyway, to test whether current master is still 
affected by this issue as it may have been fixed since 2.2.

> Checkpointed RDD deleted prematurely when using GBTClassifier
> -------------------------------------------------------------
>
>                 Key: SPARK-27018
>                 URL: https://issues.apache.org/jira/browse/SPARK-27018
>             Project: Spark
>          Issue Type: Bug
>          Components: ML
>    Affects Versions: 2.2.2, 2.2.3, 2.3.3, 2.4.0
>         Environment: OS: Ubuntu Linux 18.10
> Java: java version "1.8.0_201"
> Java(TM) SE Runtime Environment (build 1.8.0_201-b09)
> Java HotSpot(TM) 64-Bit Server VM (build 25.201-b09, mixed mode)
> Reproducible with a single-node Spark in standalone mode.
> Reproducible with Zepellin or Spark shell.
>  
>            Reporter: Piotr Kołaczkowski
>            Priority: Major
>         Attachments: 
> Fix_check_if_the_next_checkpoint_exists_before_deleting_the_old_one.patch
>
>
> Steps to reproduce:
> {noformat}
> import org.apache.spark.ml.linalg.Vectors
> import org.apache.spark.ml.classification.GBTClassifier
> case class Row(features: org.apache.spark.ml.linalg.Vector, label: Int)
> sc.setCheckpointDir("/checkpoints")
> val trainingData = sc.parallelize(1 to 2426874, 256).map(x => 
> Row(Vectors.dense(x, x + 1, x * 2 % 10), if (x % 5 == 0) 1 else 0)).toDF
> val classifier = new GBTClassifier()
>   .setLabelCol("label")
>   .setFeaturesCol("features")
>   .setProbabilityCol("probability")
>   .setMaxIter(100)
>   .setMaxDepth(10)
>   .setCheckpointInterval(2)
> classifier.fit(trainingData){noformat}
>  
> The last line fails with:
> {noformat}
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
> stage 56.0 failed 10 times, most recent failure: Lost task 0.9 in stage 56.0 
> (TID 12058, 127.0.0.1, executor 0): java.io.FileNotFoundException: 
> /checkpoints/191c9209-0955-440f-8c11-f042bdf7f804/rdd-51
> at 
> com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$1.applyOrElse(DseFileSystem.scala:63)
> at 
> com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$1.applyOrElse(DseFileSystem.scala:61)
> at 
> scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:36)
> at 
> com.datastax.bdp.fs.hadoop.DseFileSystem.com$datastax$bdp$fs$hadoop$DseFileSystem$$translateToHadoopExceptions(DseFileSystem.scala:70)
> at 
> com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$6.apply(DseFileSystem.scala:264)
> at 
> com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$6.apply(DseFileSystem.scala:264)
> at 
> com.datastax.bdp.fs.hadoop.DseFsInputStream.input(DseFsInputStream.scala:31)
> at 
> com.datastax.bdp.fs.hadoop.DseFsInputStream.openUnderlyingDataSource(DseFsInputStream.scala:39)
> at com.datastax.bdp.fs.hadoop.DseFileSystem.open(DseFileSystem.scala:269)
> at 
> org.apache.spark.rdd.ReliableCheckpointRDD$.readCheckpointFile(ReliableCheckpointRDD.scala:292)
> at 
> org.apache.spark.rdd.ReliableCheckpointRDD.compute(ReliableCheckpointRDD.scala:100)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:322)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1165)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
> at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
> at 
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
> at 
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
> at 
> org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1165)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
> at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
> at 
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
> at 
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
> at 
> org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337)
> at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1165)
> at 
> org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
> at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
> at 
> org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
> at 
> org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
> at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
> at 
> org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
> at org.apache.spark.scheduler.Task.run(Task.scala:121)
> at 
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:402)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:408)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748){noformat}
> The problem happens as well when checkpointing directory is placed on the 
> local file system, on a single-node setup. 
> Debugging at the FS level showed that the driver requests to recursively 
> delete the checkpointed rdd-51 soon before the exception gets thrown by the 
> task.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to