You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "zhengruifeng (JIRA)" <ji...@apache.org> on 2019/06/14 03:32:00 UTC

[jira] [Updated] (SPARK-27018) Checkpointed RDD deleted prematurely when using GBTClassifier

     [ https://issues.apache.org/jira/browse/SPARK-27018?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

zhengruifeng updated SPARK-27018:
---------------------------------
    Component/s: Spark Core

> 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, Spark Core
>    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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org