[ http://issues.apache.org/jira/browse/HADOOP-142?page=comments#action_12374831 ]
Runping Qi commented on HADOOP-142: ----------------------------------- Ideally, it will be the best if the tasktracker can diagnose whether the failure was task specific or is a general case. If it is a general case, the tasktracker should generate some alter and stop polling for tasks before the problem is corrected. Here is a case I encountered. For some reason, the tmp dir of the DFS was not writable: 060417 144314 task_r_3e1f4h Error running child 060417 144314 task_r_3e1f4h java.io.FileNotFoundException: /export/crawlspace2/k ryptonite/hadoop/dfs/data/tmp/client-5535743708351505322 (Read-only file system) 060417 144314 task_r_3e1f4h at java.io.FileOutputStream.open(Native Method) 060417 144314 task_r_3e1f4h at java.io.FileOutputStream.<init>(FileOutputStr eam.java:179) 060417 144314 task_r_3e1f4h at java.io.FileOutputStream.<init>(FileOutputStr eam.java:131) 060417 144314 task_r_3e1f4h at org.apache.hadoop.dfs.DFSClient$DFSOutputStre am.<init>(DFSClient.java:576) 060417 144314 task_r_3e1f4h at org.apache.hadoop.dfs.DFSClient.create(DFSCli ent.java:127) 060417 144314 task_r_3e1f4h at org.apache.hadoop.dfs.DistributedFileSystem.c reateRaw(DistributedFileSystem.java:83) 060417 144314 task_r_3e1f4h at org.apache.hadoop.fs.FSDataOutputStream$Summe r.<init>(FSDataOutputStream.java:43) 060417 144314 task_r_3e1f4h at org.apache.hadoop.fs.FSDataOutputStream.<init >(FSDataOutputStream.java:132) 060417 144314 task_r_3e1f4h at org.apache.hadoop.fs.FileSystem.create(FileSy stem.java:201) 060417 144314 task_r_3e1f4h at org.apache.hadoop.fs.FileSystem.create(FileSy stem.java:168) 060417 144314 task_r_3e1f4h at com.yahoo.yst.crawl.aggregation.CrawledDocOut putFormat.getRecordWriter(CrawledDocOutputFormat.java:39) 060417 144314 task_r_3e1f4h at org.apache.hadoop.mapred.ReduceTask.run(Reduc eTask.java:265) 060417 144314 task_r_3e1f4h at org.apache.hadoop.mapred.TaskTracker$Child.ma in(TaskTracker.java:709) In this case, all reduce tasks will fail. Thus, it does not make sense to get any new reduce tasks. In some other case, if the map.local.dir is not writable, then both map tasks and reduce rasks will fail. In this case, it should stop polling for new tasks. > failed tasks should be rescheduled on different hosts after other jobs > ---------------------------------------------------------------------- > > Key: HADOOP-142 > URL: http://issues.apache.org/jira/browse/HADOOP-142 > Project: Hadoop > Type: Improvement > Components: mapred > Versions: 0.1.1 > Reporter: Owen O'Malley > Assignee: Owen O'Malley > Fix For: 0.2 > > Currently when tasks fail, they are usually rerun immediately on the same > host. This causes problems in a couple of ways. > 1.The task is more likely to fail on the same host. > 2.If there is cleanup code (such as clearing pendingCreates) it does not > always run immediately, leading to cascading failures. > For a first pass, I propose that when a task fails, we start the scan for new > tasks to launch at the following task of the same type (within that job). So > if maps[99] fails, when we are looking to assign new map tasks from this job, > we scan like maps[100]...maps[N], maps[0]..,maps[99]. > A more involved change would avoid running tasks on nodes where it has failed > before. This is a little tricky, because you don't want to prevent > re-excution of tasks on 1 node clusters and the job tracker needs to schedule > one task tracker at a time. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira
