You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Koji Noguchi (JIRA)" <ji...@apache.org> on 2007/06/19 09:27:26 UTC

[jira] Created: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

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
            Reporter: Koji Noguchi
            Priority: Minor
             Fix For: 0.12.3


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.


[jira] Commented: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "Michael Bieniosek (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12507545 ] 

Michael Bieniosek commented on HADOOP-1502:
-------------------------------------------

I tried to bring up a new 2-node 0.13.0 cluster with the namenode formatted.  I get this error every time I try to write a file (directories are okay).  I can't find a work-around.

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


[jira] Updated: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "Koji Noguchi (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Koji Noguchi updated HADOOP-1502:
---------------------------------

        Fix Version/s:     (was: 0.12.3)
    Affects Version/s: 0.12.3

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


[jira] Updated: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

dhruba borthakur updated HADOOP-1502:
-------------------------------------

    Component/s: dfs

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


[jira] Commented: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "Michael Bieniosek (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12507547 ] 

Michael Bieniosek commented on HADOOP-1502:
-------------------------------------------

So, in my case, the problem was that the user of the fs client didn't have write permission to dfs.client.buffer.dir.  In this case, we should probably make the error message more clear.

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


[jira] Commented: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506975 ] 

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

The code that shows this problem is in Configuration.getFile()

{noformat}

 public File getFile(String dirsProp, String path) {
    ...
    if (dir.exists() || dir.mkdirs()) {  .......... (A)
        return file;                            ...........(B)
      }
}
{noformat}

The Statement A is intended to create a directory if it does not exists. However, when executed by multiple threads/processes at the same time, Statement (A) returns false and Statement (B) does not gets executed.

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


[jira] Commented: (HADOOP-1502) IOException at task startup "No valid local directories in property: dfs.client.buffer.dir"

Posted by "Devaraj Das (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12507130 ] 

Devaraj Das commented on HADOOP-1502:
-------------------------------------

A comment here is that the File class's mkdirs( ) is vulnerable to the problem (as I commented https://issues.apache.org/jira/browse/HADOOP-1513#action_12507128), but File.mkdir( ) is not. File class doesn't cache updates (as is true with HDFS). 
File.mkdir( ), File.exists( ) calls are passed on to the OS. So for example, if a thread calls File.mkdir(), and, if another thread makes a call to File.exists() while the mkdir( ) is in progress, that exists( ) call will return false. So if a call to File.exists() returns true, we can be sure that the native filesystem contains that path.

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