From issues-return-217268-archive-asf-public=cust-asf.ponee.io@spark.apache.org Fri Mar 1 09:33:05 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id C875B180676 for ; Fri, 1 Mar 2019 10:33:04 +0100 (CET) Received: (qmail 49971 invoked by uid 500); 1 Mar 2019 09:33:03 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 49962 invoked by uid 99); 1 Mar 2019 09:33:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 01 Mar 2019 09:33:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 5D1B3180DF8 for ; Fri, 1 Mar 2019 09:33:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.501 X-Spam-Level: X-Spam-Status: No, score=-109.501 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id QS8p0rQSFT6F for ; Fri, 1 Mar 2019 09:33:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 5563560DF7 for ; Fri, 1 Mar 2019 09:33:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 5833CE0E04 for ; Fri, 1 Mar 2019 09:33:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 0D615256ED for ; Fri, 1 Mar 2019 09:33:00 +0000 (UTC) Date: Fri, 1 Mar 2019 09:33:00 +0000 (UTC) From: =?utf-8?Q?Piotr_Ko=C5=82aczkowski_=28JIRA=29?= To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (SPARK-27018) Checkpointed RDD deleted prematurely when using GBTClassifier MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-27018?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D167= 81501#comment-16781501 ]=20 Piotr Ko=C5=82aczkowski edited comment on SPARK-27018 at 3/1/19 9:32 AM: -------------------------------------------------------------------- Attached a patch for Spark 2.2.2. Should be compatible with all later versi= ons. was (Author: pkolaczk): Atached patch for Spark 2.2.2. Should be compatible with all later versions= . > 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.4.0 > Environment: OS: Ubuntu Linux 18.10 > Java:=C2=A0java 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. > =C2=A0 > Reporter: Piotr Ko=C5=82aczkowski > Priority: Major > Attachments: Fix_check_if_the_next_checkpoint_exists_before_delet= ing_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 =3D sc.parallelize(1 to 2426874, 256).map(x =3D> Row(Vec= tors.dense(x, x + 1, x * 2 % 10), if (x % 5 =3D=3D 0) 1 else 0)).toDF > val classifier =3D new GBTClassifier() > .setLabelCol("label") > .setFeaturesCol("features") > .setProbabilityCol("probability") > .setMaxIter(100) > .setMaxDepth(10) > .setCheckpointInterval(2) > classifier.fit(trainingData){noformat} > =C2=A0 > 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: /c= heckpoints/191c9209-0955-440f-8c11-f042bdf7f804/rdd-51 > at com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$1.applyOrElse(DseFil= eSystem.scala:63) > at com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$1.applyOrElse(DseFil= eSystem.scala:61) > at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.sc= ala:36) > at com.datastax.bdp.fs.hadoop.DseFileSystem.com$datastax$bdp$fs$hadoop$Ds= eFileSystem$$translateToHadoopExceptions(DseFileSystem.scala:70) > at com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$6.apply(DseFileSyste= m.scala:264) > at com.datastax.bdp.fs.hadoop.DseFileSystem$$anonfun$6.apply(DseFileSyste= m.scala:264) > at com.datastax.bdp.fs.hadoop.DseFsInputStream.input(DseFsInputStream.sca= la:31) > at com.datastax.bdp.fs.hadoop.DseFsInputStream.openUnderlyingDataSource(D= seFsInputStream.scala:39) > at com.datastax.bdp.fs.hadoop.DseFileSystem.open(DseFileSystem.scala:269) > at org.apache.spark.rdd.ReliableCheckpointRDD$.readCheckpointFile(Reliabl= eCheckpointRDD.scala:292) > at org.apache.spark.rdd.ReliableCheckpointRDD.compute(ReliableCheckpointR= DD.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(B= lockManager.scala:1165) > at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(B= lockManager.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.sca= la: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:5= 2) > 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(B= lockManager.scala:1165) > at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(B= lockManager.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.sca= la: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:5= 2) > 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(B= lockManager.scala:1165) > at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(B= lockManager.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.sca= la: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:5= 2) > 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:5= 2) > 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:5= 2) > 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(Execut= or.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.j= ava: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.=C2=A0 > 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. > =C2=A0 > =C2=A0 -- 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