hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Christian Kunz (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-308) Task Tracker does not handle the case of read only local dir case correctly
Date Wed, 12 Dec 2007 02:35:43 GMT

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

Christian Kunz commented on HADOOP-308:
---------------------------------------

This still happens. I identified a taskTracker which has 4 disks, of which only one is read-only,
and seemingly submits any task to the read-only disk (no job got successfully submitted since
Nov 30), although mapred.local.dir in hadoop-site.xml specifies local directories on all 4
disks. This node has 3 good disks, still accepts tasks, but cannot execute any of them, de-facto
an unusable node without being detected as such.

The reason seems to be that the localizeJob method in TaskTracker defines a path localJarFile
always on the same disk, because the hash is just based on 'taskTracker/jobcache/', independent
on the job id,
and when the disk has become read-only after that directory got created, then the checking
in getLocalPath in Configuration.java does not help to identify the disk as read-only.


Exception(s) look like:
Error initializing task_200712090222_0017_m_000870_0:
java.io.IOException: Mkdirs failed to create<localDir on disk 2>taskTracker/jobcache/job_200712090222_0017
	at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:345)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:353)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:260)
	at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:139)
	at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:116)
	at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:853)
	at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:834)
	at org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:585)
	at org.apache.hadoop.mapred.TaskTracker.startNewTask(TaskTracker.java:1143)
	at org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:807)
	at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:1179)
	at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:1880)

> Task Tracker does not handle the case of read only local dir  case correctly
> ----------------------------------------------------------------------------
>
>                 Key: HADOOP-308
>                 URL: https://issues.apache.org/jira/browse/HADOOP-308
>             Project: Hadoop
>          Issue Type: Bug
>          Components: mapred
>    Affects Versions: 0.3.2
>         Environment: all
>            Reporter: Runping Qi
>            Assignee: Owen O'Malley
>
> In case that the local dir is not writable on a node, the tasks on the  node will fail
as expected, with an exception like:
> (Read-only file system) at java.io.FileOutputStream.open(Native Method) 
> at java.io.FileOutputStream.(FileOutputStream.java:179) 
> at java.io.FileOutputStream.(FileOutputStream.java:131) 
> at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.(DFSClient.java:723) 
> at org.apache.hadoop.dfs.DFSClient.create(DFSClient.java:241) 
> at org.apache.hadoop.dfs.DistributedFileSystem.createRaw(DistributedFileSystem.java:96)

> at org.apache.hadoop.fs.FSDataOutputStream$Summer.(FSDataOutputStream.java:44) 
> at org.apache.hadoop.fs.FSDataOutputStream.(FSDataOutputStream.java:134) 
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:224) 
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:176) 
> ....
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:265) 
> at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:847)
> However, the task tracker will continue accept new tasks and continue to fail.
> The runloop of tasktracker should detect such a problem and exits.

-- 
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