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

dhruba borthakur commented on HADOOP-1502:
------------------------------------------

I see that there is a semantic problem with ClientProtocol.exists() and 
ClientProtocol.isDir(). The locking is such that they can race with another 
thread that is creating the directory at the same time. It can so happen that 
Thread A is is executing mkdir(foodir), but before this transaction is logged 
into the FsEdits log, another thread B executes exists(foodir) and returns 
success. Right after this, the namenode shuts down and the transaction is never 
logged. However, the namenode had already returned "true" to the exists(foodir) 
call.

This appears to be a semantic bug to me.

> IOException at task startup "No valid local directories in property: 
> dfs.client.buffer.dir"
> -------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-1502
>                 URL: https://issues.apache.org/jira/browse/HADOOP-1502
>             Project: Hadoop
>          Issue Type: Bug
>          Components: dfs
>    Affects Versions: 0.12.3
>            Reporter: Koji Noguchi
>            Priority: Minor
>
> When tasks started up on a brand new mapred cluster, I saw couples of them 
> fail immediately  by 
> java.io.IOException: No valid local directories in property: 
> dfs.client.buffer.dir
>       at org.apache.hadoop.conf.Configuration.getFile(Configuration.java:401)
>       at 
> org.apache.hadoop.dfs.DFSClient$DFSOutputStream.newBackupFile(DFSClient.java:1037)
>       at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.(DFSClient.java:1004)
>       at org.apache.hadoop.dfs.DFSClient.create(DFSClient.java:276)
>       at 
> org.apache.hadoop.dfs.DistributedFileSystem$RawDistributedFileSystem.create(DistributedFileSystem.java:143)
>       at 
> org.apache.hadoop.fs.ChecksumFileSystem$FSOutputSummer.(ChecksumFileSystem.java:367)
>       at 
> org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:442)
>       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:346)
>       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:253)
>       at 
> org.apache.hadoop.util.CopyFiles$FSCopyFilesMapper.copy(CopyFiles.java:215)
>       at 
> org.apache.hadoop.util.CopyFiles$FSCopyFilesMapper.map(CopyFiles.java:410)
>       at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:48)
>       at org.apache.hadoop.mapred.MapTask.run(MapTask.java:175)
>       at 
> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1445)
> (Run on version 0.12.3 + many patches, so the line number could be a little 
> off)

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

Reply via email to