hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Devaraj Das (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-2391) Speculative Execution race condition with output paths
Date Mon, 10 Dec 2007 04:38:43 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-2391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12549895
] 

Devaraj Das commented on HADOOP-2391:
-------------------------------------

Dennis, the way it works is that there is a TaskCommitQueue thread that saves or discards
outputs. And, yes, the job could be declared as "complete" before the killed tasks' outputs
are discarded since the job is declared as successful as soon as all the tasks are successful
(including saving their outputs).

> Speculative Execution race condition with output paths
> ------------------------------------------------------
>
>                 Key: HADOOP-2391
>                 URL: https://issues.apache.org/jira/browse/HADOOP-2391
>             Project: Hadoop
>          Issue Type: Bug
>         Environment: all
>            Reporter: Dennis Kubes
>            Assignee: Dennis Kubes
>
> I am tracking a problem where when speculative execution is enabled, there is a race
condition when trying to read output paths from a previously completed job.  More specifically
when reduce tasks run their output is put into a working directory under the task name until
the task in completed.  The directory name is something like workdir/_taskid.  Upon completion
the output get moved into workdir.  Regular tasks are checked for this move and not considered
completed until this move is made.  I have not verified it but all indications point to speculative
tasks NOT having this same check for completion and more importantly removal when killed.
 So what we end up with when trying to read the output of previous tasks with speculative
execution enabled is the possibility that previous workdir/_taskid will be present when the
output directory is read by a chained job.  Here is an error when supports my theory:
> Generator: org.apache.hadoop.ipc.RemoteException: java.io.IOException: Cannot open filename
/u01/hadoop/mapred/temp/generate-temp-1197104928603/_task_200712080949_0005_r_000014_1
>         at org.apache.hadoop.dfs.NameNode.open(NameNode.java:234)
>         at sun.reflect.GeneratedMethodAccessor64.invoke(Unknown Source)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:389)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:644)
>         at org.apache.hadoop.ipc.Client.call(Client.java:507)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:186)
>         at org.apache.hadoop.dfs.$Proxy0.open(Unknown Source)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
>         at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
>         at org.apache.hadoop.dfs.$Proxy0.open(Unknown Source)
>         at org.apache.hadoop.dfs.DFSClient$DFSInputStream.openInfo(DFSClient.java:839)
>         at org.apache.hadoop.dfs.DFSClient$DFSInputStream.<init>(DFSClient.java:831)
>         at org.apache.hadoop.dfs.DFSClient.open(DFSClient.java:263)
>         at org.apache.hadoop.dfs.DistributedFileSystem.open(DistributedFileSystem.java:114)
>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1356)
>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1349)
>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:1344)
>         at org.apache.hadoop.mapred.SequenceFileOutputFormat.getReaders(SequenceFileOutputFormat.java:87)
>         at org.apache.nutch.crawl.Generator.generate(Generator.java:429)
>         at org.apache.nutch.crawl.Generator.run(Generator.java:563)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
>         at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:54)
>         at org.apache.nutch.crawl.Generator.main(Generator.java:526)
> I will continue to research this and post as I make progress on tracking down this bug.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message