[ https://issues.apache.org/jira/browse/SPARK-2984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15602160#comment-15602160 ]
Steve Loughran commented on SPARK-2984: --------------------------------------- Alexy, can you describe your layout a bit more # are you using Amazon EMR? # why s3:// URLs? # what hadoop-* JARs are you using (it's not that easy to tell in spark-1.6...what did you download?) If you can, use Hadoop 2.7.x+ and then switch to s3a. What I actually suspect here, looking at the code listing, is that the following sequence is happening * previous work completes, deletes/renames _temporary * next code gets a list of all files (globStatus(pat)) * Then iterates through, getting info about each one. * During that listing, the _temp dir goes away, which breaks the code. Seems to me the listing logic in {{FileInputFormat.singleThreadedListStatus}} could be more forgiving about FNFEs: if the file is no longer there, well, skip it. However, S3A in Hadoop 2.8 (and in HDP2.5, I'll note) replaces this very inefficient directory listing with one optimised for S3A (HADOOP-13208), which is much less likely to exhibit this problem. Even so, some more resilience would be good; I'll make a note in the relevant bits of the Hadoop code > FileNotFoundException on _temporary directory > --------------------------------------------- > > Key: SPARK-2984 > URL: https://issues.apache.org/jira/browse/SPARK-2984 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.1.0 > Reporter: Andrew Ash > Assignee: Josh Rosen > Priority: Critical > Fix For: 1.3.0 > > > We've seen several stacktraces and threads on the user mailing list where > people are having issues with a {{FileNotFoundException}} stemming from an > HDFS path containing {{_temporary}}. > I ([~aash]) think this may be related to {{spark.speculation}}. I think the > error condition might manifest in this circumstance: > 1) task T starts on a executor E1 > 2) it takes a long time, so task T' is started on another executor E2 > 3) T finishes in E1 so moves its data from {{_temporary}} to the final > destination and deletes the {{_temporary}} directory during cleanup > 4) T' finishes in E2 and attempts to move its data from {{_temporary}}, but > those files no longer exist! exception > Some samples: > {noformat} > 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) > {noformat} > -- Chen Song at > http://apache-spark-user-list.1001560.n3.nabble.com/saveAsTextFiles-file-not-found-exception-td10686.html > {noformat} > 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. > {noformat} > and > {noformat} > 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 > {noformat} > -- Bill Jay at > http://apache-spark-user-list.1001560.n3.nabble.com/saveAsTextFiles-file-not-found-exception-td10686.html > {noformat} > scala> d3.sample(false,0.01,1).map( pair => pair._2 > ).saveAsTextFile("10000.txt") > 14/06/09 22:06:40 ERROR TaskSetManager: Task 0.0:0 failed 4 times; aborting > job > org.apache.spark.SparkException: Job aborted: Task 0.0:0 failed 4 times (most > recent failure: Exception failure: java.io.IOException: The temporary > job-output directory file:/data/spark-0.9.1-bin-hadoop1/10000.txt/_temporary > doesn't exist!) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1020) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1018) > at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) > at > org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$abortStage(DAGScheduler.scala:1018) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:604) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:604) > at scala.Option.foreach(Option.scala:236) > at > org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:604) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:190) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498) > at akka.actor.ActorCell.invoke(ActorCell.scala:456) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237) > at akka.dispatch.Mailbox.run(Mailbox.scala:219) > at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386) > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > {noformat} > -- Oleg Proudnikov at > http://mail-archives.apache.org/mod_mbox/incubator-spark-user/201406.mbox/%3ccafeagwkysxz2gyviqu44-dvp-v1jzbvfapd1x5dlhcqbogt...@mail.gmail.com%3E > {noformat} > [INFO] 11 Dec 2013 12:00:33 - org.apache.spark.Logging$class - Loss was due > to org.apache.hadoop.util.Shell$ExitCodeException > org.apache.hadoop.util.Shell$ExitCodeException: chmod: getting attributes of > `/cygdrive/c/somepath/_temporary/_attempt_201312111200_0000_m_000000_0/part-00000': > No such file or directory > at org.apache.hadoop.util.Shell.runCommand(Shell.java:261) > at org.apache.hadoop.util.Shell.run(Shell.java:188) > at > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:381) > at org.apache.hadoop.util.Shell.execCommand(Shell.java:467) > at org.apache.hadoop.util.Shell.execCommand(Shell.java:450) > at > org.apache.hadoop.fs.RawLocalFileSystem.execCommand(RawLocalFileSystem.java:593) > at > org.apache.hadoop.fs.RawLocalFileSystem.setPermission(RawLocalFileSystem.java:584) > at > org.apache.hadoop.fs.FilterFileSystem.setPermission(FilterFileSystem.java:427) > at > org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:465) > at > org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:433) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:886) > at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:781) > at > org.apache.hadoop.mapred.TextOutputFormat.getRecordWriter(TextOutputFormat.java:118) > at > org.apache.hadoop.mapred.SparkHadoopWriter.open(SparkHadoopWriter.scala:86) > at > org.apache.spark.rdd.PairRDDFunctions.writeToFile$1(PairRDDFunctions.scala:667) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:680) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:680) > at org.apache.spark.scheduler.ResultTask.run(ResultTask.scala:99) > at > org.apache.spark.scheduler.local.LocalScheduler.runTask(LocalScheduler.scala:198) > at > org.apache.spark.scheduler.local.LocalActor$$anonfun$launchTask$1$$anon$1.run(LocalScheduler.scala:68) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > 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:744) > [INFO] 11 Dec 2013 12:00:33 - org.apache.spark.Logging$class - Remove TaskSet > 0.0 from pool > [INFO] 11 Dec 2013 12:00:33 - org.apache.spark.Logging$class - Failed to run > saveAsTextFile at Test.scala:19 > Exception in thread "main" org.apache.spark.SparkException: Job failed: Task > 0.0:0 failed more than 4 times; aborting job > org.apache.hadoop.util.Shell$ExitCodeException: chmod: getting attributes of > `/cygdrive/c/somepath/_temporary/_attempt_201312111200_0000_m_000000_0/part-00000': > No such file or directory > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:760) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:758) > at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:60) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) > at > org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:758) > at > org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:379) > at > org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$run(DAGScheduler.scala:441) > at > org.apache.spark.scheduler.DAGScheduler$$anon$1.run(DAGScheduler.scala:149) > {noformat} > On a Windows box! > -- Nathan Kronenfeld at > http://mail-archives.apache.org/mod_mbox/spark-user/201312.mbox/%3CCAEpWh49EvUEWdnsfKJGvU5MB9V5QsR=HQ=whpufumeetu19...@mail.gmail.com%3E > {noformat} > 14/07/29 16:16:57 ERROR executor.Executor: Exception in task ID 6087 > java.io.IOException: The temporary job-output directory > hdfs://mybox:8020/path/to/a/dir/_temporary doesn't exist! > at > org.apache.hadoop.mapred.FileOutputCommitter.getWorkPath(FileOutputCommitter.java:250) > at > org.apache.hadoop.mapred.FileOutputFormat.getTaskOutputPath(FileOutputFormat.java:240) > at > org.apache.avro.mapred.AvroOutputFormat.getRecordWriter(AvroOutputFormat.java:154) > at > org.apache.hadoop.mapred.SparkHadoopWriter.open(SparkHadoopWriter.scala:90) > at > org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$writeToFile$1(PairRDDFunctions.scala:728) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:741) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:741) > at > org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:109) > at org.apache.spark.scheduler.Task.run(Task.scala:53) > at > org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211) > at > org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:42) > at > org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:41) > 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:1408) > at > org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:41) > at > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176) > 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) > {noformat} > and > {noformat} > 14/07/29 16:16:57 ERROR executor.Executor: Exception in task ID 6158 > > org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException): > No lease on /path/to/a/dir/_temporary/_attempt_201407291616_0000_m_0002 > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2445) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2437) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:2503) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:2480) > at > org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.complete(NameNodeRpcServer.java:556) > at > org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.complete(ClientNamenodeProtocolServerSideTranslatorPB.java:337) > at > org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:44958) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:453) > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1002) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1751) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1747) > 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:1408) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1745) > at org.apache.hadoop.ipc.Client.call(Client.java:1225) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202) > at com.sun.proxy.$Proxy13.complete(Unknown Source) > at sun.reflect.GeneratedMethodAccessor45.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:164) > at > org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:83) > at com.sun.proxy.$Proxy13.complete(Unknown Source) > at > org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.complete(ClientNamenodeProtocolTranslatorPB.java:329) > at > org.apache.hadoop.hdfs.DFSOutputStream.completeFile(DFSOutputStream.java:1769) > at > org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:1756) > at > org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:66) > at > org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:99) > at java.io.FilterOutputStream.close(FilterOutputStream.java:160) > at java.io.FilterOutputStream.close(FilterOutputStream.java:160) > at org.apache.avro.file.DataFileWriter.close(DataFileWriter.java:376) > at > org.apache.avro.mapred.AvroOutputFormat$1.close(AvroOutputFormat.java:163) > at > org.apache.hadoop.mapred.SparkHadoopWriter.close(SparkHadoopWriter.scala:102) > at > org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$writeToFile$1(PairRDDFunctions.scala:737) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:741) > at > org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$2.apply(PairRDDFunctions.scala:741) > at > org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:109) > at org.apache.spark.scheduler.Task.run(Task.scala:53) > at > org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211) > at > org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:42) > {noformat} > and > {noformat} > 14/07/29 21:01:33 ERROR executor.Executor: Exception in task ID 150 > > org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException): > No lease on > /path/to/dir/main/_temporary/_attempt_201407292101_0000_m_000125_150/part-0012 > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2445) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:2437) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:2503) > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:2480) > at > org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.complete(NameNodeRpcServer.java:556) > at > org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.complete(ClientNamenodeProtocolServerSideTranslatorPB.java:337) > at > org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:44958) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:453) > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1002) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1751) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1747) > 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:1408) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1745) > {noformat} > -- Andrew Ash -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org