Thanks for putting this together, Andrew.
On Tue, Aug 12, 2014 at 2:11 AM, Andrew Ash <and...@andrewash.com> wrote: > Hi Chen, > > Please see the bug I filed at > https://issues.apache.org/jira/browse/SPARK-2984 with the > FileNotFoundException on _temporary directory issue. > > Andrew > > > On Mon, Aug 11, 2014 at 10:50 PM, Andrew Ash <and...@andrewash.com> wrote: > >> Not sure which stalled HDFS client issue your'e referring to, but there >> was one fixed in Spark 1.0.2 that could help you out -- >> https://github.com/apache/spark/pull/1409. I've still seen one related >> to Configuration objects not being threadsafe though so you'd still need to >> keep speculation on to fix that (SPARK-2546) >> >> As it stands now, I can: >> >> A) have speculation off, in which case I get random hangs for a variety >> of reasons (your HDFS stall, my Configuration safety issue) >> >> or >> >> B) have speculation on, in which case I get random failures related to >> LeaseExpiredExceptions and .../_temporary/... file doesn't exist exceptions. >> >> >> Kind of a catch-22 -- there's no reliable way to run large jobs on Spark >> right now! >> >> I'm going to file a bug for the _temporary and LeaseExpiredExceptions as >> I think these are widespread enough that we need a place to track a >> resolution. >> >> >> On Mon, Aug 11, 2014 at 9:08 AM, Chen Song <chen.song...@gmail.com> >> wrote: >> >>> Andrew that is a good finding. >>> >>> Yes, I have speculative execution turned on, becauseI saw tasks stalled >>> on HDFS client. >>> >>> If I turned off speculative execution, is there a way to circumvent the >>> hanging task issue? >>> >>> >>> >>> On Mon, Aug 11, 2014 at 11:13 AM, Andrew Ash <and...@andrewash.com> >>> wrote: >>> >>>> I've also been seeing similar stacktraces on Spark core (not streaming) >>>> and have a theory it's related to spark.speculation being turned on. Do >>>> you have that enabled by chance? >>>> >>>> >>>> On Mon, Aug 11, 2014 at 8:10 AM, Chen Song <chen.song...@gmail.com> >>>> wrote: >>>> >>>>> Bill >>>>> >>>>> Did you get this resolved somehow? Anyone has any insight into this >>>>> problem? >>>>> >>>>> Chen >>>>> >>>>> >>>>> On Mon, Aug 11, 2014 at 10:30 AM, Chen Song <chen.song...@gmail.com> >>>>> wrote: >>>>> >>>>>> The exception was thrown out in application master(spark streaming >>>>>> driver) and the job shut down after this exception. >>>>>> >>>>>> >>>>>> On Mon, Aug 11, 2014 at 10:29 AM, Chen Song <chen.song...@gmail.com> >>>>>> wrote: >>>>>> >>>>>>> I got the same exception after the streaming job runs for a while, >>>>>>> The ERROR message was complaining about a temp file not being found in >>>>>>> the >>>>>>> output folder. >>>>>>> >>>>>>> 14/08/11 08:05:08 ERROR JobScheduler: Error running job streaming >>>>>>> job 1407744300000 ms.0 >>>>>>> java.io.FileNotFoundException: File >>>>>>> hdfs://hadoopc/user/csong/output/human_bot/-1407744300000.out/_temporary/0/task_201408110805_0000_m_000007 >>>>>>> does not exist. >>>>>>> at >>>>>>> org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:654) >>>>>>> at >>>>>>> org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:102) >>>>>>> at >>>>>>> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:712) >>>>>>> at >>>>>>> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:708) >>>>>>> at >>>>>>> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) >>>>>>> at >>>>>>> org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:708) >>>>>>> at >>>>>>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:360) >>>>>>> at >>>>>>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310) >>>>>>> at >>>>>>> org.apache.hadoop.mapred.FileOutputCommitter.commitJob(FileOutputCommitter.java:136) >>>>>>> at >>>>>>> org.apache.spark.SparkHadoopWriter.commitJob(SparkHadoopWriter.scala:126) >>>>>>> at >>>>>>> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopDataset(PairRDDFunctions.scala:841) >>>>>>> at >>>>>>> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:724) >>>>>>> at >>>>>>> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:643) >>>>>>> at org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1068) >>>>>>> at >>>>>>> org.apache.spark.streaming.dstream.DStream$$anonfun$8.apply(DStream.scala:773) >>>>>>> at >>>>>>> org.apache.spark.streaming.dstream.DStream$$anonfun$8.apply(DStream.scala:771) >>>>>>> at >>>>>>> org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply$mcV$sp(ForEachDStream.scala:41) >>>>>>> at >>>>>>> org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:40) >>>>>>> at >>>>>>> org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:40) >>>>>>> at scala.util.Try$.apply(Try.scala:161) >>>>>>> at org.apache.spark.streaming.scheduler.Job.run(Job.scala:32) >>>>>>> at >>>>>>> org.apache.spark.streaming.scheduler.JobScheduler$JobHandler.run(JobScheduler.scala:172) >>>>>>> at >>>>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) >>>>>>> at >>>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) >>>>>>> at java.lang.Thread.run(Thread.java:745) >>>>>>> >>>>>>> >>>>>>> On Fri, Jul 25, 2014 at 7:04 PM, Bill Jay < >>>>>>> bill.jaypeter...@gmail.com> wrote: >>>>>>> >>>>>>>> I just saw another error after my job was run for 2 hours: >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException): >>>>>>>> No lease on /apps/data/vddil/real-time/checkpoint/temp: File does not >>>>>>>> exist. Holder DFSClient_NONMAPREDUCE_327993456_13 does not have any >>>>>>>> open files. >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2946) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.analyzeFileState(FSNamesystem.java:2766) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2674) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:584) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:440) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java) >>>>>>>> at >>>>>>>> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585) >>>>>>>> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928) >>>>>>>> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2013) >>>>>>>> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2009) >>>>>>>> at java.security.AccessController.doPrivileged(Native Method) >>>>>>>> at javax.security.auth.Subject.doAs(Subject.java:415) >>>>>>>> at >>>>>>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1557) >>>>>>>> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2007) >>>>>>>> >>>>>>>> at org.apache.hadoop.ipc.Client.call(Client.java:1410) >>>>>>>> at org.apache.hadoop.ipc.Client.call(Client.java:1363) >>>>>>>> at >>>>>>>> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206) >>>>>>>> at com.sun.proxy.$Proxy14.addBlock(Unknown Source) >>>>>>>> at sun.reflect.GeneratedMethodAccessor146.invoke(Unknown Source) >>>>>>>> at >>>>>>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) >>>>>>>> at java.lang.reflect.Method.invoke(Method.java:606) >>>>>>>> at >>>>>>>> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:190) >>>>>>>> at >>>>>>>> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:103) >>>>>>>> at com.sun.proxy.$Proxy14.addBlock(Unknown Source) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:361) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.locateFollowingBlock(DFSOutputStream.java:1439) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.nextBlockOutputStream(DFSOutputStream.java:1261) >>>>>>>> at >>>>>>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:525) >>>>>>>> 14/07/25 14:45:12 WARN CheckpointWriter: Error in attempt 1 of writing >>>>>>>> checkpoint to >>>>>>>> hdfs://gnosis-01-01-01.crl.samsung.com/apps/data/vddil/real-time/checkpoint/checkpoint-1406324700000 >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> All my jobs use the same parameter to the function checkpoint. Is >>>>>>>> it the reason for the error? >>>>>>>> >>>>>>>> I will post the stack trace of the other error after it appears >>>>>>>> again. Thanks! >>>>>>>> >>>>>>>> >>>>>>>> Bill >>>>>>>> >>>>>>>> >>>>>>>> On Fri, Jul 25, 2014 at 2:57 PM, Tathagata Das < >>>>>>>> tathagata.das1...@gmail.com> wrote: >>>>>>>> >>>>>>>>> Can you give a stack trace and logs of the exception? Its hard to >>>>>>>>> say anything without any associated stack trace and logs. >>>>>>>>> >>>>>>>>> TD >>>>>>>>> >>>>>>>>> >>>>>>>>> On Fri, Jul 25, 2014 at 1:32 PM, Bill Jay < >>>>>>>>> bill.jaypeter...@gmail.com> wrote: >>>>>>>>> >>>>>>>>>> Hi, >>>>>>>>>> >>>>>>>>>> I am running a Spark Streaming job that uses saveAsTextFiles to >>>>>>>>>> save results into hdfs files. However, it has an exception after 20 >>>>>>>>>> batches >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> result-1406312340000/_temporary/0/task_201407251119_0000_m_000003 >>>>>>>>>> does not exist. >>>>>>>>>> >>>>>>>>>> >>>>>>>>>> When the job is running, I do not change any file in the folder. >>>>>>>>>> Does anyone know why the file cannot be found? >>>>>>>>>> >>>>>>>>>> Thanks! >>>>>>>>>> >>>>>>>>>> Bill >>>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>>> >>>>>>> >>>>>>> -- >>>>>>> Chen Song >>>>>>> >>>>>>> >>>>>> >>>>>> >>>>>> -- >>>>>> Chen Song >>>>>> >>>>>> >>>>> >>>>> >>>>> -- >>>>> Chen Song >>>>> >>>>> >>>> >>> >>> >>> -- >>> Chen Song >>> >>> >> > -- Chen Song