hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sanjay Dahiya (JIRA)" <j...@apache.org>
Subject [jira] Created: (HADOOP-612) DFs copyFromLocal fails with NullPointerException for a single file
Date Wed, 18 Oct 2006 11:34:39 GMT
DFs copyFromLocal fails with NullPointerException for a single file
-------------------------------------------------------------------

                 Key: HADOOP-612
                 URL: http://issues.apache.org/jira/browse/HADOOP-612
             Project: Hadoop
          Issue Type: Bug
          Components: dfs
    Affects Versions: 0.7.1
            Reporter: Sanjay Dahiya
            Priority: Blocker


DFS copyFromLocal fails with NullPointerException when copying a single file to DFS. Copying
a directory works. 

  public File getFile(String dirsProp, String path)
    throws IOException {
    String[] dirs = getStrings(dirsProp);                     <===== returns null for a
single file
    int hashCode = path.hashCode();
    for (int i = 0; i < dirs.length; i++) {  // try each local dir                    
<==== Throws NullPointerException
      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
      File file = new File(dirs[index], path);
      File dir = file.getParentFile();
      if (dir.exists() || dir.mkdirs()) {
        return file;
      }
    }
    throw new IOException("No valid local directories in property: "+dirsProp);
  }



Exception in thread "main" java.lang.NullPointerException
        at org.apache.hadoop.conf.Configuration.getFile(Configuration.java:397)
        at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.newBackupFile(DFSClient.java:913)
        at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.<init>(DFSClient.java:903)
        at org.apache.hadoop.dfs.DFSClient.create(DFSClient.java:276)
        at org.apache.hadoop.dfs.DistributedFileSystem.createRaw(DistributedFileSystem.java:104)
        at org.apache.hadoop.fs.FSDataOutputStream$Summer.<init>(FSDataOutputStream.java:56)
        at org.apache.hadoop.fs.FSDataOutputStream$Summer.<init>(FSDataOutputStream.java:45)
        at org.apache.hadoop.fs.FSDataOutputStream.<init>(FSDataOutputStream.java:146)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:271)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:178)
        at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:77)
        at org.apache.hadoop.dfs.DistributedFileSystem.copyFromLocalFile(DistributedFileSystem.java:186)
        at org.apache.hadoop.dfs.DFSShell.copyFromLocal(DFSShell.java:45)
        at org.apache.hadoop.dfs.DFSShell.run(DFSShell.java:516)
        at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:187)
        at org.apache.hadoop.dfs.DFSShell.main(DFSShell.java:570)


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

        

Mime
View raw message