Browse Source

HADOOP-1621. Make FileStatus a concrete class.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@564699 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 18 years ago
parent
commit
6f2b6e21c4

+ 3 - 0
CHANGES.txt

@@ -13,6 +13,9 @@ Trunk (unreleased changes)
     earlier deprecated) are removed.  Client Protocol version changed 
     earlier deprecated) are removed.  Client Protocol version changed 
     from 15 to 16.  (Raghu Angadi via dhruba)
     from 15 to 16.  (Raghu Angadi via dhruba)
 
 
+    HADOOP-1621.  FileStatus is now a concrete class and FileSystem.listPaths
+    is deprecated and replaced with listStatus. (Chris Douglas via omalley)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-1636.  Allow configuration of the number of jobs kept in
     HADOOP-1636.  Allow configuration of the number of jobs kept in

+ 6 - 5
src/java/org/apache/hadoop/dfs/BlockCrcUpgrade.java

@@ -1956,13 +1956,13 @@ class BlockCrcUpgradeObjectNamenode extends UpgradeObjectNamenode {
     
     
     for (DFSFileInfo file:fileArr) {
     for (DFSFileInfo file:fileArr) {
       if (file.isDir()) {
       if (file.isDir()) {
-        updateBlockLevelStats(file.getPath(), stats);
+        updateBlockLevelStats(file.getPath().toString(), stats);
       } else {
       } else {
         // Get the all the blocks.
         // Get the all the blocks.
         LocatedBlocks blockLoc = null;
         LocatedBlocks blockLoc = null;
         try {
         try {
-          blockLoc = getFSNamesystem().getBlockLocations(null, file.getPath(), 0, 
-                                                  file.getLen());
+          blockLoc = getFSNamesystem().getBlockLocations(null,
+              file.getPath().toString(), 0, file.getLen());
           int numBlocks = blockLoc.locatedBlockCount();
           int numBlocks = blockLoc.locatedBlockCount();
           for (int i=0; i<numBlocks; i++) {
           for (int i=0; i<numBlocks; i++) {
             LocatedBlock loc = blockLoc.get(i);
             LocatedBlock loc = blockLoc.get(i);
@@ -1990,7 +1990,8 @@ class BlockCrcUpgradeObjectNamenode extends UpgradeObjectNamenode {
           }
           }
         } catch (IOException e) {
         } catch (IOException e) {
           LOG.error("BlockCrcUpgrade: could not get block locations for " +
           LOG.error("BlockCrcUpgrade: could not get block locations for " +
-                    file.getPath() + " : " + StringUtils.stringifyException(e));
+                    file.getPath().toString() + " : " +
+                    StringUtils.stringifyException(e));
           stats.errors++;
           stats.errors++;
         }
         }
       }
       }
@@ -2069,7 +2070,7 @@ class BlockCrcUpgradeObjectNamenode extends UpgradeObjectNamenode {
     
     
     for ( DFSFileInfo file:fileArr ) {
     for ( DFSFileInfo file:fileArr ) {
       if ( file.isDir() ) {
       if ( file.isDir() ) {
-        numFilesDeleted += deleteCrcFiles(file.getPath());
+        numFilesDeleted += deleteCrcFiles(file.getPath().toString());
       }
       }
     }
     }
     
     

+ 8 - 81
src/java/org/apache/hadoop/dfs/DFSFileInfo.java

@@ -21,8 +21,6 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 
 
-import java.io.*;
-
 /******************************************************
 /******************************************************
  * DFSFileInfo tracks info about remote files, including
  * DFSFileInfo tracks info about remote files, including
  * name, size, etc.
  * name, size, etc.
@@ -31,7 +29,7 @@ import java.io.*;
  * Block locations are sorted by the distance to the current client.
  * Block locations are sorted by the distance to the current client.
  * 
  * 
  ******************************************************/
  ******************************************************/
-class DFSFileInfo implements Writable, FileStatus {
+class DFSFileInfo extends FileStatus {
   static {                                      // register a ctor
   static {                                      // register a ctor
     WritableFactories.setFactory
     WritableFactories.setFactory
       (DFSFileInfo.class,
       (DFSFileInfo.class,
@@ -40,13 +38,6 @@ class DFSFileInfo implements Writable, FileStatus {
        });
        });
   }
   }
 
 
-  Path path;
-  long len;
-  boolean isDir;
-  short blockReplication;
-  long blockSize;
-  long modificationTime;
-
   /**
   /**
    */
    */
   public DFSFileInfo() {
   public DFSFileInfo() {
@@ -56,36 +47,22 @@ class DFSFileInfo implements Writable, FileStatus {
    * Create DFSFileInfo by file INode 
    * Create DFSFileInfo by file INode 
    */
    */
   public DFSFileInfo(FSDirectory.INode node) {
   public DFSFileInfo(FSDirectory.INode node) {
-    this.path = new Path(node.getAbsoluteName());
-    this.isDir = node.isDir();
-    this.len = isDir ? node.computeContentsLength() : node.computeFileLength();
-    this.blockReplication = node.getReplication();
-    blockSize = node.getBlockSize();
-    modificationTime = node.getModificationTime();
-  }
-
-  /**
-   */
-  public String getPath() {
-    return path.toString();
+    // XXX This should probably let length == 0 for directories
+    super(node.isDir() ? node.computeContentsLength() : node.computeFileLength(),
+          node.isDir(), node.getReplication(), node.getBlockSize(),
+          node.getModificationTime(), new Path(node.getAbsoluteName()));
   }
   }
 
 
   /**
   /**
    */
    */
   public String getName() {
   public String getName() {
-    return path.getName();
+    return getPath().getName();
   }
   }
   
   
   /**
   /**
    */
    */
   public String getParent() {
   public String getParent() {
-    return path.getParent().toString();
-  }
-
-  /**
-   */
-  public long getLen() {
-    return len;
+    return getPath().getParent().toString();
   }
   }
 
 
   /**
   /**
@@ -93,56 +70,6 @@ class DFSFileInfo implements Writable, FileStatus {
    */
    */
   public long getContentsLen() {
   public long getContentsLen() {
     assert isDir() : "Must be a directory";
     assert isDir() : "Must be a directory";
-    return len;
-  }
-
-  /**
-   */
-  public boolean isDir() {
-    return isDir;
-  }
-
-  /**
-   */
-  public short getReplication() {
-    return this.blockReplication;
-  }
-
-  /**
-   * Get the block size of the file.
-   * @return the number of bytes
-   */
-  public long getBlockSize() {
-    return blockSize;
-  }
-
-  /**
-   * Get the last modification time of the file.
-   * @return the number of milliseconds since January 1, 1970 UTC.
-   */
-  public long getModificationTime() {
-    return modificationTime;
-  }
-    
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, getPath());
-    out.writeLong(len);
-    out.writeBoolean(isDir);
-    out.writeShort(blockReplication);
-    out.writeLong(blockSize);
-    out.writeLong(modificationTime);
-  }
-  
-  public void readFields(DataInput in) throws IOException {
-    String strPath = Text.readString(in);
-    this.path = new Path(strPath);
-    this.len = in.readLong();
-    this.isDir = in.readBoolean();
-    this.blockReplication = in.readShort();
-    blockSize = in.readLong();
-    modificationTime = in.readLong();
+    return getLen();
   }
   }
 }
 }

+ 1 - 1
src/java/org/apache/hadoop/dfs/DfsPath.java

@@ -32,7 +32,7 @@ class DfsPath extends Path {
    * DfsPaths are fully qualified with scheme and authority.
    * DfsPaths are fully qualified with scheme and authority.
    */
    */
   public DfsPath(DFSFileInfo info, FileSystem fs) {
   public DfsPath(DFSFileInfo info, FileSystem fs) {
-    super((new Path(info.getPath())).makeQualified(fs).toString());
+    super((new Path(info.getPath().toString())).makeQualified(fs).toString());
     this.info = info;
     this.info = info;
   }
   }
 
 

+ 4 - 0
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -171,6 +171,10 @@ public class DistributedFileSystem extends FileSystem {
     return (info == null) ? 0 : info[0].getLen();
     return (info == null) ? 0 : info[0].getLen();
   }
   }
 
 
+  public FileStatus[] listStatus(Path f) throws IOException {
+    return dfs.listPaths(getPath(f));
+  }
+
   public Path[] listPaths(Path f) throws IOException {
   public Path[] listPaths(Path f) throws IOException {
     DFSFileInfo info[] = dfs.listPaths(getPath(f));
     DFSFileInfo info[] = dfs.listPaths(getPath(f));
     if (info == null) {
     if (info == null) {

+ 10 - 8
src/java/org/apache/hadoop/dfs/NamenodeFsck.java

@@ -139,10 +139,10 @@ public class NamenodeFsck {
   private void check(DFSFileInfo file, FsckResult res) throws IOException {
   private void check(DFSFileInfo file, FsckResult res) throws IOException {
     if (file.isDir()) {
     if (file.isDir()) {
       if (showFiles) {
       if (showFiles) {
-        out.println(file.getPath() + " <dir>");
+        out.println(file.getPath().toString() + " <dir>");
       }
       }
       res.totalDirs++;
       res.totalDirs++;
-      DFSFileInfo[] files = nn.getListing(file.getPath());
+      DFSFileInfo[] files = nn.getListing(file.getPath().toString());
       for (int i = 0; i < files.length; i++) {
       for (int i = 0; i < files.length; i++) {
         check(files[i], res);
         check(files[i], res);
       }
       }
@@ -151,10 +151,12 @@ public class NamenodeFsck {
     res.totalFiles++;
     res.totalFiles++;
     long fileLen = file.getLen();
     long fileLen = file.getLen();
     res.totalSize += fileLen;
     res.totalSize += fileLen;
-    LocatedBlocks blocks = nn.getBlockLocations(file.getPath(), 0, fileLen);
+    LocatedBlocks blocks = nn.getBlockLocations(file.getPath().toString(),
+        0, fileLen);
     res.totalBlocks += blocks.locatedBlockCount();
     res.totalBlocks += blocks.locatedBlockCount();
     if (showFiles) {
     if (showFiles) {
-      out.print(file.getPath() + " " + fileLen + ", " + res.totalBlocks + " block(s): ");
+      out.print(file.getPath().toString() + " " + fileLen + ", " +
+          res.totalBlocks + " block(s): ");
     }  else {
     }  else {
       out.print('.');
       out.print('.');
       out.flush();
       out.flush();
@@ -179,7 +181,7 @@ public class NamenodeFsck {
         res.numUnderReplicatedBlocks += 1;
         res.numUnderReplicatedBlocks += 1;
         under++;
         under++;
         if (!showFiles) {
         if (!showFiles) {
-          out.print("\n" + file.getPath() + ": ");
+          out.print("\n" + file.getPath().toString() + ": ");
         }
         }
         out.println(" Under replicated " + block.getBlockName() +
         out.println(" Under replicated " + block.getBlockName() +
                     ". Target Replicas is " +
                     ". Target Replicas is " +
@@ -209,7 +211,7 @@ public class NamenodeFsck {
     }
     }
     if (missing > 0) {
     if (missing > 0) {
       if (!showFiles) {
       if (!showFiles) {
-        out.println("\n" + file.getPath() + ": " +
+        out.println("\n" + file.getPath().toString() + ": " +
                     "MISSING " + missing + " blocks of total size " + 
                     "MISSING " + missing + " blocks of total size " + 
                     missize + " B.");
                     missize + " B.");
       }
       }
@@ -221,7 +223,7 @@ public class NamenodeFsck {
         lostFoundMove(file, blocks);
         lostFoundMove(file, blocks);
         break;
         break;
       case FIXING_DELETE:
       case FIXING_DELETE:
-        nn.delete(file.getPath());
+        nn.delete(file.getPath().toString());
       }
       }
     }
     }
     if (showFiles) {
     if (showFiles) {
@@ -292,7 +294,7 @@ public class NamenodeFsck {
       }
       }
       if (fos != null) fos.close();
       if (fos != null) fos.close();
       LOG.warn("\n - moved corrupted file " + file.getPath() + " to /lost+found");
       LOG.warn("\n - moved corrupted file " + file.getPath() + " to /lost+found");
-      dfs.delete(new UTF8(file.getPath()));
+      dfs.delete(new UTF8(file.getPath().toString()));
     }  catch (Exception e) {
     }  catch (Exception e) {
       e.printStackTrace();
       e.printStackTrace();
       LOG.warn(errmsg + ": " + e.getMessage());
       LOG.warn(errmsg + ": " + e.getMessage());

+ 74 - 8
src/java/org/apache/hadoop/fs/FileStatus.java

@@ -17,39 +17,105 @@
  */
  */
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
 
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 
 
 /** Interface that represents the client side information for a file.
 /** Interface that represents the client side information for a file.
  */
  */
-public interface FileStatus {
+public class FileStatus implements Writable {
+
+  private Path path;
+  private long length;
+  private boolean isdir;
+  private short block_replication;
+  private long blocksize;
+  private long modification_time;
+
+  public FileStatus() { this(0, false, 0, 0, 0, null); }
+
+  public FileStatus(long length, boolean isdir, int block_replication,
+             long blocksize, long modification_time, Path path) {
+    this(length, isdir, (short)block_replication, blocksize,
+         modification_time, path);
+  }
+
+  public FileStatus(long length, boolean isdir, short block_replication,
+             long blocksize, long modification_time, Path path) {
+    this.length = length;
+    this.isdir = isdir;
+    this.block_replication = block_replication;
+    this.blocksize = blocksize;
+    this.modification_time = modification_time;
+    this.path = path;
+  }
 
 
   /* 
   /* 
    * @return the length of this file, in blocks
    * @return the length of this file, in blocks
    */
    */
-  public long getLen();
+  public long getLen() {
+    return length;
+  }
 
 
   /**
   /**
    * Is this a directory?
    * Is this a directory?
    * @return true if this is a directory
    * @return true if this is a directory
    */
    */
-  public boolean isDir();
+  public boolean isDir() {
+    return isdir;
+  }
 
 
   /**
   /**
    * Get the block size of the file.
    * Get the block size of the file.
    * @return the number of bytes
    * @return the number of bytes
    */
    */
-  public long getBlockSize();
+  public long getBlockSize() {
+    return blocksize;
+  }
 
 
   /**
   /**
    * Get the replication factor of a file.
    * Get the replication factor of a file.
    * @return the replication factor of a file.
    * @return the replication factor of a file.
    */
    */
-  public short getReplication();
+  public short getReplication() {
+    return block_replication;
+  }
 
 
   /**
   /**
    * Get the modification time of the file.
    * Get the modification time of the file.
-   * @return the modification time of file.
+   * @return the modification time of file in milliseconds since January 1, 1970 UTC.
    */
    */
-  public long getModificationTime();
+  public long getModificationTime() {
+    return modification_time;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  //////////////////////////////////////////////////
+  // Writable
+  //////////////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, getPath().toString());
+    out.writeLong(length);
+    out.writeBoolean(isdir);
+    out.writeShort(block_replication);
+    out.writeLong(blocksize);
+    out.writeLong(modification_time);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    String strPath = Text.readString(in);
+    this.path = new Path(strPath);
+    this.length = in.readLong();
+    this.isdir = in.readBoolean();
+    this.block_replication = in.readShort();
+    blocksize = in.readLong();
+    modification_time = in.readLong();
+  }
+
 }
 }

+ 14 - 2
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -475,7 +475,19 @@ public abstract class FileSystem extends Configured {
     };
     };
     
     
   /** List files in a directory. */
   /** List files in a directory. */
-  public abstract Path[] listPaths(Path f) throws IOException;
+  @Deprecated
+  public Path[] listPaths(Path f) throws IOException {
+    FileStatus[] stat = listStatus(f);
+    if (stat == null) return null;
+    Path[] ret = new Path[stat.length];
+    for (int i = 0; i < stat.length; ++i) {
+      ret[i] = stat[i].getPath();
+    }
+    return ret;
+  }
+
+  /** */
+  public abstract FileStatus[] listStatus(Path f) throws IOException;
     
     
   /** 
   /** 
    * Filter files in the given pathes using the default checksum filter. 
    * Filter files in the given pathes using the default checksum filter. 
@@ -504,7 +516,7 @@ public abstract class FileSystem extends Configured {
   public Path[] listPaths(Path f, PathFilter filter) throws IOException {
   public Path[] listPaths(Path f, PathFilter filter) throws IOException {
     ArrayList<Path> results = new ArrayList<Path>();
     ArrayList<Path> results = new ArrayList<Path>();
     listPaths(results, f, filter);
     listPaths(results, f, filter);
-    return (Path[]) results.toArray(new Path[results.size()]);
+    return results.toArray(new Path[results.size()]);
   }
   }
     
     
   /** 
   /** 

+ 2 - 2
src/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -157,8 +157,8 @@ public class FilterFileSystem extends FileSystem {
   }
   }
 
 
   /** List files in a directory. */
   /** List files in a directory. */
-  public Path[] listPaths(Path f) throws IOException {
-    return fs.listPaths(f);
+  public FileStatus[] listStatus(Path f) throws IOException {
+    return fs.listStatus(f);
   }
   }
   
   
   /**
   /**

+ 5 - 22
src/java/org/apache/hadoop/fs/InMemoryFileSystem.java

@@ -262,7 +262,7 @@ public class InMemoryFileSystem extends ChecksumFileSystem {
     /**
     /**
      * Directory operations are not supported
      * Directory operations are not supported
      */
      */
-    public Path[] listPaths(Path f) throws IOException {
+    public FileStatus[] listStatus(Path f) throws IOException {
       return null;
       return null;
     }
     }
 
 
@@ -280,7 +280,7 @@ public class InMemoryFileSystem extends ChecksumFileSystem {
   
   
     public FileStatus getFileStatus(Path f) throws IOException {
     public FileStatus getFileStatus(Path f) throws IOException {
       synchronized (this) {
       synchronized (this) {
-        return new InMemoryFileStatus(pathToFileAttribs.get(getPath(f)));
+        return new InMemoryFileStatus(f, pathToFileAttribs.get(getPath(f)));
       }
       }
     }
     }
   
   
@@ -373,26 +373,9 @@ public class InMemoryFileSystem extends ChecksumFileSystem {
       }
       }
     }
     }
 
 
-    private class InMemoryFileStatus implements FileStatus {
-      private long length;
-
-      InMemoryFileStatus(FileAttributes attr) throws IOException {
-        length = attr.size;
-      }
-      public long getLen() {
-        return length;
-      }
-      public boolean isDir() {
-        return false;
-      }
-      public long getBlockSize() {
-        return getDefaultBlockSize();
-      }
-      public short getReplication() {
-        return 1;
-      }
-      public long getModificationTime() {
-        return 0;  // not supported yet
+    private class InMemoryFileStatus extends FileStatus {
+      InMemoryFileStatus(Path f, FileAttributes attr) throws IOException {
+        super(attr.size, false, 1, getDefaultBlockSize(), 0, f);
       }
       }
     }
     }
   }
   }

+ 22 - 43
src/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -191,30 +191,29 @@ public class RawLocalFileSystem extends FileSystem {
   public boolean exists(Path f) throws IOException {
   public boolean exists(Path f) throws IOException {
     return pathToFile(f).exists();
     return pathToFile(f).exists();
   }
   }
-  
-  public Path[] listPaths(Path f) throws IOException {
+
+  public FileStatus[] listStatus(Path f) throws IOException {
     File localf = pathToFile(f);
     File localf = pathToFile(f);
-    Path[] results;
-    
-    if (!localf.exists())
+    FileStatus[] results;
+
+    if (!localf.exists()) {
+      return null;
+    }
+    if (localf.isFile()) {
+      return new FileStatus[] { new RawLocalFileStatus(localf) };
+    }
+
+    String[] names = localf.list();
+    if (names == null) {
       return null;
       return null;
-    else if (localf.isFile()) {
-      results = new Path[1];
-      results[0] = f;
-      return results;
-    } else { // directory
-      String[] names = localf.list();
-      if (names == null) {
-        return null;
-      }
-      results = new Path[names.length];
-      for (int i = 0; i < names.length; i++) {
-        results[i] = new Path(f, names[i]);
-      }
-      return results;
     }
     }
+    results = new FileStatus[names.length];
+    for (int i = 0; i < names.length; i++) {
+      results[i] = getFileStatus(new Path(f, names[i]));
+    }
+    return results;
   }
   }
-  
+
   /**
   /**
    * Creates the specified directory hierarchy. Does not
    * Creates the specified directory hierarchy. Does not
    * treat existence as an error.
    * treat existence as an error.
@@ -319,30 +318,10 @@ public class RawLocalFileSystem extends FileSystem {
     return new RawLocalFileStatus(pathToFile(f));
     return new RawLocalFileStatus(pathToFile(f));
   }
   }
 
 
-  private class RawLocalFileStatus implements FileStatus {
-    private long length;
-    private boolean isDir;
-    private long mtime;
-
+  private class RawLocalFileStatus extends FileStatus {
     RawLocalFileStatus(File f) throws IOException {
     RawLocalFileStatus(File f) throws IOException {
-      length = f.length();
-      isDir = f.isDirectory();
-      mtime = f.lastModified();
-    }
-    public long getLen() {
-      return length;
-    }
-    public boolean isDir() {
-      return isDir;
-    }
-    public long getBlockSize() {
-      return getDefaultBlockSize();
-    }
-    public short getReplication() {
-      return 1;
-    }
-    public long getModificationTime() {
-      return mtime;
+      super(f.length(), f.isDirectory(), 1, getDefaultBlockSize(),
+            f.lastModified(), new Path(f.toURI().toString()));
     }
     }
   }
   }
 }
 }

+ 27 - 31
src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
@@ -161,17 +162,20 @@ public class S3FileSystem extends FileSystem {
   }
   }
 
 
   @Override
   @Override
-  public Path[] listPaths(Path path) throws IOException {
-    Path absolutePath = makeAbsolute(path);
+  public FileStatus[] listStatus(Path f) throws IOException {
+    Path absolutePath = makeAbsolute(f);
     INode inode = store.retrieveINode(absolutePath);
     INode inode = store.retrieveINode(absolutePath);
     if (inode == null) {
     if (inode == null) {
       return null;
       return null;
-    } else if (inode.isFile()) {
-      return new Path[] { absolutePath };
-    } else { // directory
-      Set<Path> paths = store.listSubPaths(absolutePath);
-      return paths.toArray(new Path[0]);
     }
     }
+    if (inode.isFile()) {
+      return new FileStatus[] { new S3FileStatus(f, inode) };
+    }
+    ArrayList<FileStatus> ret = new ArrayList<FileStatus>();
+    for (Path p : store.listSubPaths(absolutePath)) {
+      ret.add(getFileStatus(p));
+    }
+    return ret.toArray(new FileStatus[0]);
   }
   }
 
 
   @Override
   @Override
@@ -288,7 +292,7 @@ public class S3FileSystem extends FileSystem {
     if (inode == null) {
     if (inode == null) {
       throw new IOException(f.toString() + ": No such file or directory.");
       throw new IOException(f.toString() + ": No such file or directory.");
     }
     }
-    return new S3FileStatus(inode);
+    return new S3FileStatus(f, inode);
   }
   }
 
 
   // diagnostic methods
   // diagnostic methods
@@ -301,35 +305,27 @@ public class S3FileSystem extends FileSystem {
     store.purge();
     store.purge();
   }
   }
 
 
-  private static class S3FileStatus implements FileStatus {
-    private long length = 0, blockSize = 0;
-    private boolean isDir;
+  private static class S3FileStatus extends FileStatus {
+
+    S3FileStatus(Path f, INode inode) throws IOException {
+      super(findLength(inode), inode.isDirectory(), 1,
+            findBlocksize(inode), 0, f);
+    }
 
 
-    S3FileStatus(INode inode) throws IOException {
-      isDir = inode.isDirectory();
-      if (!isDir) {
+    private static long findLength(INode inode) {
+      if (!inode.isDirectory()) {
+        long length = 0L;
         for (Block block : inode.getBlocks()) {
         for (Block block : inode.getBlocks()) {
           length += block.getLength();
           length += block.getLength();
-          if (blockSize == 0) {
-            blockSize = block.getLength();
-          }
         }
         }
+        return length;
       }
       }
+      return 0;
     }
     }
-    public long getLen() {
-      return length;
-    }
-    public boolean isDir() {
-      return isDir;
-    }
-    public long getBlockSize() {
-      return blockSize;
-    }
-    public short getReplication() {
-      return 1;
-    }
-    public long getModificationTime() {
-      return 0;  // not supported yet
+
+    private static long findBlocksize(INode inode) {
+      final Block[] ret = inode.getBlocks();
+      return ret == null ? 0L : ret[0].getLength();
     }
     }
   }
   }
 }
 }

+ 4 - 4
src/webapps/datanode/browseDirectory.jsp

@@ -96,16 +96,16 @@
         String cols [] = new String[6];
         String cols [] = new String[6];
         for (int i = 0; i < files.length; i++) {
         for (int i = 0; i < files.length; i++) {
           //Get the location of the first block of the file
           //Get the location of the first block of the file
-          if (files[i].getPath().endsWith(".crc")) continue;
+          if (files[i].getPath().toString().endsWith(".crc")) continue;
           if (!files[i].isDir()) {
           if (!files[i].isDir()) {
             List<LocatedBlock> blocks = 
             List<LocatedBlock> blocks = 
-              dfs.namenode.getBlockLocations(files[i].getPath(), 0, 1).getLocatedBlocks();
+              dfs.namenode.getBlockLocations(files[i].getPath().toString(), 0, 1).getLocatedBlocks();
             DatanodeInfo [] locations = blocks.get(0).getLocations();
             DatanodeInfo [] locations = blocks.get(0).getLocations();
             if (locations.length == 0) {
             if (locations.length == 0) {
               cols[0] = files[i].getName();
               cols[0] = files[i].getName();
             } else {
             } else {
               String datanodeUrl = req.getRequestURL()+"?dir="+
               String datanodeUrl = req.getRequestURL()+"?dir="+
-                URLEncoder.encode(files[i].getPath(), "UTF-8") + 
+                URLEncoder.encode(files[i].getPath().toString(), "UTF-8") + 
                 "&namenodeInfoPort=" + namenodeInfoPort;
                 "&namenodeInfoPort=" + namenodeInfoPort;
 		            
 		            
               cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getName()+"</a>";
               cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getName()+"</a>";
@@ -118,7 +118,7 @@
           }
           }
           else {
           else {
             String datanodeUrl = req.getRequestURL()+"?dir="+
             String datanodeUrl = req.getRequestURL()+"?dir="+
-              URLEncoder.encode(files[i].getPath(), "UTF-8") + 
+              URLEncoder.encode(files[i].getPath().toString(), "UTF-8") + 
               "&namenodeInfoPort=" + namenodeInfoPort;
               "&namenodeInfoPort=" + namenodeInfoPort;
             cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getName()+"</a>";
             cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getName()+"</a>";
             cols[1] = "dir";
             cols[1] = "dir";