You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2007/06/23 00:55:35 UTC

svn commit: r549977 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/s3/ src/test/org/apache/hadoop/dfs/ src/webapps/datanode/

Author: cutting
Date: Fri Jun 22 15:55:34 2007
New Revision: 549977

URL: http://svn.apache.org/viewvc?view=rev&rev=549977
Log:
HADOOP-1377.  Add support for modification time to FileSystem.  Contributed by Dhruba.

Added:
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestModTime.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DfsPath.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSImage.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
    lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri Jun 22 15:55:34 2007
@@ -246,6 +246,11 @@
      a temporary name, then rename them to their final name, so that
      failures don't leave partial files.  (Tsz Wo Sze via cutting)
 
+ 76. HADOOP-1377.  Add support for modification time to FileSystem and
+     implement in HDFS and local implementations.  Also, alter access
+     to file properties to be through a new FileStatus interface.
+     (Dhruba Borthakur via cutting)
+
 
 Release 0.13.0 - 2007-06-08
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java Fri Jun 22 15:55:34 2007
@@ -30,10 +30,8 @@
 
   /**
    * Compared to the previous version the following changes have been introduced:
-   * 12: open() prototype changed; 
-   *     getBlockLocations() added; 
-   *     DFSFileInfo format changed;
-   *     getHints() removed.
+   * 13: getListing returns file creation times and modification times.
+   *     getFileInfo added.
    *     DatanodeInfo serialization has hostname.
    */
   public static final long versionID = 13L;
@@ -370,4 +368,11 @@
    * @throws IOException
    */
   public void metaSave(String filename) throws IOException;
+
+  /* Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   * @throws IOException if file does not exist
+   * @return object containing information regarding the file
+   */
+  public DFSFileInfo getFileInfo(String src) throws IOException;
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Fri Jun 22 15:55:34 2007
@@ -399,6 +399,11 @@
     return namenode.getListing(src.toString());
   }
 
+  public DFSFileInfo getFileInfo(UTF8 src) throws IOException {
+    checkOpen();
+    return namenode.getFileInfo(src.toString());
+  }
+
   /**
    */
   public long totalRawCapacity() throws IOException {

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java Fri Jun 22 15:55:34 2007
@@ -19,6 +19,7 @@
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
 
 import java.io.*;
 
@@ -30,7 +31,7 @@
  * Block locations are sorted by the distance to the current client.
  * 
  ******************************************************/
-class DFSFileInfo implements Writable {
+class DFSFileInfo implements Writable, FileStatus {
   static {                                      // register a ctor
     WritableFactories.setFactory
       (DFSFileInfo.class,
@@ -44,7 +45,8 @@
   boolean isDir;
   short blockReplication;
   long blockSize;
-  
+  long modificationTime;
+
   /**
    */
   public DFSFileInfo() {
@@ -59,6 +61,7 @@
     this.len = isDir ? node.computeContentsLength() : node.computeFileLength();
     this.blockReplication = node.getReplication();
     blockSize = node.getBlockSize();
+    modificationTime = node.getModificationTime();
   }
 
   /**
@@ -112,6 +115,14 @@
   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
@@ -122,6 +133,7 @@
     out.writeBoolean(isDir);
     out.writeShort(blockReplication);
     out.writeLong(blockSize);
+    out.writeLong(modificationTime);
   }
   
   public void readFields(DataInput in) throws IOException {
@@ -131,5 +143,6 @@
     this.isDir = in.readBoolean();
     this.blockReplication = in.readShort();
     blockSize = in.readLong();
+    modificationTime = in.readLong();
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DfsPath.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DfsPath.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DfsPath.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DfsPath.java Fri Jun 22 15:55:34 2007
@@ -55,4 +55,7 @@
   public long getBlockSize() {
     return info.getBlockSize();
   }
+  public long getModificationTime() {
+    return info.getModificationTime();
+  }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java Fri Jun 22 15:55:34 2007
@@ -76,14 +76,6 @@
       return dfs.getDefaultBlockSize();
     }
     
-    public long getBlockSize(Path f) throws IOException {
-      // if we already know the answer, use it.
-      if (f instanceof DfsPath) {
-        return ((DfsPath) f).getBlockSize();
-      }
-      return dfs.getBlockSize(getPath(f));
-    }
-    
     public short getDefaultReplication() {
       return dfs.getDefaultReplication();
     }
@@ -172,22 +164,6 @@
       return dfs.exists(getPath(f));
     }
 
-    public boolean isDirectory(Path f) throws IOException {
-      if (f instanceof DfsPath) {
-        return ((DfsPath)f).isDirectory();
-      }
-      return dfs.isDirectory(getPath(f));
-    }
-
-    public long getLength(Path f) throws IOException {
-      if (f instanceof DfsPath) {
-        return ((DfsPath)f).length();
-      }
-
-      DFSFileInfo info[] = dfs.listPaths(getPath(f));
-      return (info == null) ? 0 : info[0].getLen();
-    }
-
     public long getContentLength(Path f) throws IOException {
       if (f instanceof DfsPath) {
         return ((DfsPath)f).getContentsLength();
@@ -197,15 +173,6 @@
       return (info == null) ? 0 : info[0].getLen();
     }
 
-    public short getReplication(Path f) throws IOException {
-      if (f instanceof DfsPath) {
-        return ((DfsPath)f).getReplication();
-      }
-
-      DFSFileInfo info[] = dfs.listPaths(getPath(f));
-      return info[0].getReplication();
-    }
-
     public Path[] listPaths(Path f) throws IOException {
       DFSFileInfo info[] = dfs.listPaths(getPath(f));
       if (info == null) {
@@ -367,6 +334,20 @@
 
       return true;
     }
+
+    /**
+     * Returns the stat information about the file.
+     */
+    public FileStatus getFileStatus(Path f) throws IOException {
+      if (f instanceof DfsPath) {
+        DfsPath p = (DfsPath) f;
+        return p.info;
+      }
+      else {
+        DFSFileInfo p = dfs.getFileInfo(getPath(f));
+        return p;
+      }
+    }
   }
 
   public DistributedFileSystem() {
@@ -443,5 +424,13 @@
                                        FSDataInputStream sums, long sumsPos) {
     return ((RawDistributedFileSystem)fs).reportChecksumFailure(
                                                                 f, in, inPos, sums, sumsPos);
+  }
+
+  /**
+   * Returns the stat information about the file.
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    return ((RawDistributedFileSystem)fs).getFileStatus(f);
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java Fri Jun 22 15:55:34 2007
@@ -136,8 +136,7 @@
   // Version is reflected in the data storage file.
   // Versions are negative.
   // Decrement LAYOUT_VERSION to define a new version.
-  public static final int LAYOUT_VERSION = -4;
+  public static final int LAYOUT_VERSION = -5;
   // Current version: 
-  // Top level directory is reorganized to allow file system state 
-  // transitions: upgrade, rollback, and finalize.
+  // File modification times added.
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java Fri Jun 22 15:55:34 2007
@@ -48,6 +48,7 @@
     private TreeMap<String, INode> children = null;
     private Block blocks[] = null;
     private short blockReplication;
+    private long modificationTime;
 
     /**
      */
@@ -56,6 +57,7 @@
       this.parent = null;
       this.blocks = blocks;
       this.blockReplication = replication;
+      this.modificationTime = 0;
     }
 
     /**
@@ -65,6 +67,26 @@
       this.parent = null;
       this.blocks = null;
       this.blockReplication = 0;
+      this.modificationTime = 0;
+    }
+
+    INode(String name, long modifictionTime) {
+      this.name = name;
+      this.parent = null;
+      this.blocks = null;
+      this.blockReplication = 0;
+      this.modificationTime = modifictionTime;
+    }
+
+    /**
+     */
+    INode(String name, Block blocks[], short replication,
+          long modificationTime) {
+      this.name = name;
+      this.parent = null;
+      this.blocks = blocks;
+      this.blockReplication = replication;
+      this.modificationTime = modificationTime;
     }
 
     /**
@@ -107,6 +129,24 @@
     }
 
     /**
+     * Get last modification time of inode.
+     * @return access time
+     */
+    long getModificationTime() {
+      return this.modificationTime;
+    }
+
+    /**
+     * Set last modification time of inode.
+     */
+    void setModificationTime(long modtime) {
+      assert isDir();
+      if (this.modificationTime <= modtime) {
+        this.modificationTime = modtime;
+      }
+    }
+
+    /**
      * Get children iterator
      * @return Iterator of children
      */
@@ -215,7 +255,7 @@
         return true;
       }
     }
-          
+
     /**
      * Collect all the blocks at this INode and all its children.
      * This operation is performed after a node is removed from the tree,
@@ -385,12 +425,14 @@
   public boolean addFile(String path, Block[] blocks, short replication) {
     waitForReady();
 
-    // Always do an implicit mkdirs for parent directory tree
-    if (!mkdirs(new Path(path).getParent().toString())) {
+    // Always do an implicit mkdirs for parent directory tree.
+    long modTime = FSNamesystem.now();
+    if (!mkdirs(new Path(path).getParent().toString(), modTime)) {
       return false;
     }
-    INode newNode = new INode(new File(path).getName(), blocks, replication);
-    if (!unprotectedAddFile(path, newNode)) {
+    INode newNode = new INode(new File(path).getName(), blocks, 
+                              replication, modTime);
+    if (!unprotectedAddFile(path, newNode, modTime)) {
       NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
                                    +"failed to add "+path+" with "
                                    +blocks.length+" blocks to the file system");
@@ -405,14 +447,17 @@
     
   /**
    */
-  boolean unprotectedAddFile(String path, INode newNode) {
+  private boolean unprotectedAddFile(String path, INode newNode, long parentModTime) {
     synchronized (rootDir) {
       try {
         if (rootDir.addNode(path, newNode) != null) {
           int nrBlocks = (newNode.blocks == null) ? 0 : newNode.blocks.length;
           // Add file->block mapping
-          for (int i = 0; i < nrBlocks; i++)
+          for (int i = 0; i < nrBlocks; i++) {
             namesystem.blocksMap.addINode(newNode.blocks[i], newNode);
+          }
+          // update modification time of parent directory
+          newNode.getParent().setModificationTime(parentModTime);
           return true;
         } else {
           return false;
@@ -423,9 +468,12 @@
     }
   }
     
-  boolean unprotectedAddFile(String path, Block[] blocks, short replication) {
+  boolean unprotectedAddFile(String path, Block[] blocks, short replication,
+                             long modificationTime) {
     return unprotectedAddFile(path,  
-                              new INode(path, blocks, replication));
+                              new INode(path, blocks, replication,
+                                        modificationTime),
+                              modificationTime);
   }
 
   /**
@@ -435,15 +483,16 @@
     NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
                                   +src+" to "+dst);
     waitForReady();
-    if (!unprotectedRenameTo(src, dst))
+    long now = namesystem.now();
+    if (!unprotectedRenameTo(src, dst, now))
       return false;
-    fsImage.getEditLog().logRename(src, dst);
+    fsImage.getEditLog().logRename(src, dst, now);
     return true;
   }
 
   /**
    */
-  boolean unprotectedRenameTo(String src, String dst) {
+  boolean unprotectedRenameTo(String src, String dst, long timestamp) {
     synchronized(rootDir) {
       INode renamedNode = rootDir.getNode(src);
       if (renamedNode == null) {
@@ -459,6 +508,7 @@
                                      +"failed to rename "+src+" to "+dst+ " because destination exists");
         return false;
       }
+      INode oldParent = renamedNode.getParent();
       renamedNode.removeNode();
             
       // the renamed node can be reused now
@@ -466,6 +516,10 @@
         if (rootDir.addNode(dst, renamedNode) != null) {
           NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
                                         +src+" is renamed to "+dst);
+
+          // update modification time of old parent as well as new parent dir
+          oldParent.setModificationTime(timestamp);
+          renamedNode.getParent().setModificationTime(timestamp);
           return true;
         }
       } catch (FileNotFoundException e) {
@@ -476,7 +530,6 @@
         }catch(FileNotFoundException e2) {                
         }
       }
-
       return false;
     }
   }
@@ -549,15 +602,16 @@
     NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: "
                                   +src);
     waitForReady();
-    Block[] blocks = unprotectedDelete(src); 
+    long now = namesystem.now();
+    Block[] blocks = unprotectedDelete(src, now); 
     if (blocks != null)
-      fsImage.getEditLog().logDelete(src);
+      fsImage.getEditLog().logDelete(src, now);
     return blocks;
   }
 
   /**
    */
-  Block[] unprotectedDelete(String src) {
+  Block[] unprotectedDelete(String src, long modificationTime) {
     synchronized (rootDir) {
       INode targetNode = rootDir.getNode(src);
       if (targetNode == null) {
@@ -576,6 +630,7 @@
         } else {
           NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
                                         +src+" is removed");
+          targetNode.getParent().setModificationTime(modificationTime);
           Vector<Block> v = new Vector<Block>();
           targetNode.collectSubtreeBlocks(v);
           for (Block b : v) {
@@ -648,6 +703,24 @@
     }
   }
 
+  /* Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @throws IOException if file does not exist
+   * @return object containing information regarding the file
+   */
+  DFSFileInfo getFileInfo(String src) throws IOException {
+    String srcs = normalizePath(src);
+    synchronized (rootDir) {
+      INode targetNode = rootDir.getNode(srcs);
+      if (targetNode == null) {
+        throw new IOException("File does not exist");
+      }
+      else {
+        return new DFSFileInfo(targetNode);
+      }
+    }
+  }
+
   /**
    * Get the blocks associated with the file.
    */
@@ -702,7 +775,7 @@
   /**
    * Create directory entries for every item
    */
-  boolean mkdirs(String src) {
+  boolean mkdirs(String src, long now) {
     src = normalizePath(src);
 
     // Use this to collect all the dirs we need to construct
@@ -724,7 +797,7 @@
     for (int i = numElts - 1; i >= 0; i--) {
       String cur = v.elementAt(i);
       try {
-        INode inserted = unprotectedMkdir(cur);
+        INode inserted = unprotectedMkdir(cur, now);
         if (inserted != null) {
           NameNode.stateChangeLog.debug("DIR* FSDirectory.mkdirs: "
                                         +"created directory "+cur);
@@ -747,9 +820,10 @@
 
   /**
    */
-  INode unprotectedMkdir(String src) throws FileNotFoundException {
+  INode unprotectedMkdir(String src, long timestamp) throws FileNotFoundException {
     synchronized (rootDir) {
-      return rootDir.addNode(src, new INode(new File(src).getName()));
+      return rootDir.addNode(src, new INode(new File(src).getName(),
+                                            timestamp));
     }
   }
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java Fri Jun 22 15:55:34 2007
@@ -227,6 +227,9 @@
       short replication = fsNamesys.getDefaultReplication();
       try {
         while (true) {
+          long timestamp = 0;
+          long ctime = 0; 
+          long mtime = 0;
           byte opcode = -1;
           try {
             opcode = in.readByte();
@@ -247,13 +250,18 @@
               aw = new ArrayWritable(UTF8.class);
               aw.readFields(in);
               writables = aw.get(); 
-              if (writables.length != 2)
-                throw new IOException("Incorrect data fortmat. " 
-                                      + "Name & replication pair expected");
+              if (logVersion >= -4 && writables.length != 2 ||
+                  logVersion < -4 && writables.length != 3) {
+                  throw new IOException("Incorrect data fortmat. " 
+                                        + "Name & replication pair expected");
+              }
               name = (UTF8) writables[0];
               replication = Short.parseShort(
                                              ((UTF8)writables[1]).toString());
               replication = adjustReplication(replication);
+              if (logVersion < -4) {
+                mtime = Long.parseLong(((UTF8)writables[2]).toString());
+              }
             }
             // get blocks
             aw = new ArrayWritable(Block.class);
@@ -262,7 +270,7 @@
             Block blocks[] = new Block[writables.length];
             System.arraycopy(writables, 0, blocks, 0, blocks.length);
             // add to the file tree
-            fsDir.unprotectedAddFile(name.toString(), blocks, replication);
+            fsDir.unprotectedAddFile(name.toString(), blocks, replication, mtime);
             break;
           }
           case OP_SET_REPLICATION: {
@@ -277,23 +285,70 @@
             break;
           } 
           case OP_RENAME: {
-            UTF8 src = new UTF8();
-            UTF8 dst = new UTF8();
-            src.readFields(in);
-            dst.readFields(in);
-            fsDir.unprotectedRenameTo(src.toString(), dst.toString());
+            UTF8 src = null;
+            UTF8 dst = null;
+            if (logVersion >= -4) {
+              src = new UTF8();
+              dst = new UTF8();
+              src.readFields(in);
+              dst.readFields(in);
+            } else {
+              ArrayWritable aw = null;
+              Writable writables[];
+              aw = new ArrayWritable(UTF8.class);
+              aw.readFields(in);
+              writables = aw.get(); 
+              if (writables.length != 3) {
+                throw new IOException("Incorrect data fortmat. " 
+                                      + "Mkdir operation.");
+              }
+              src = (UTF8) writables[0];
+              dst = (UTF8) writables[1];
+              timestamp = Long.parseLong(((UTF8)writables[2]).toString());
+            }
+            fsDir.unprotectedRenameTo(src.toString(), dst.toString(), timestamp);
             break;
           }
           case OP_DELETE: {
-            UTF8 src = new UTF8();
-            src.readFields(in);
-            fsDir.unprotectedDelete(src.toString());
+            UTF8 src = null;
+            if (logVersion >= -4) {
+              src = new UTF8();
+              src.readFields(in);
+            } else {
+              ArrayWritable aw = null;
+              Writable writables[];
+              aw = new ArrayWritable(UTF8.class);
+              aw.readFields(in);
+              writables = aw.get(); 
+              if (writables.length != 2) {
+                throw new IOException("Incorrect data fortmat. " 
+                                      + "delete operation.");
+              }
+              src = (UTF8) writables[0];
+              timestamp = Long.parseLong(((UTF8)writables[1]).toString());
+            }
+            fsDir.unprotectedDelete(src.toString(), timestamp);
             break;
           }
           case OP_MKDIR: {
-            UTF8 src = new UTF8();
-            src.readFields(in);
-            fsDir.unprotectedMkdir(src.toString());
+            UTF8 src = null;
+            if (logVersion >= -4) {
+              src = new UTF8();
+              src.readFields(in);
+            } else {
+              ArrayWritable aw = null;
+              Writable writables[];
+              aw = new ArrayWritable(UTF8.class);
+              aw.readFields(in);
+              writables = aw.get(); 
+              if (writables.length != 2) {
+                throw new IOException("Incorrect data fortmat. " 
+                                      + "Mkdir operation.");
+              }
+              src = (UTF8) writables[0];
+              timestamp = Long.parseLong(((UTF8)writables[1]).toString());
+            }
+            fsDir.unprotectedMkdir(src.toString(), timestamp);
             break;
           }
           case OP_DATANODE_ADD: {
@@ -422,7 +477,8 @@
   void logCreateFile(FSDirectory.INode newNode) {
     UTF8 nameReplicationPair[] = new UTF8[] { 
       new UTF8(newNode.computeName()), 
-      FSEditLog.toLogReplication(newNode.getReplication())};
+      FSEditLog.toLogReplication(newNode.getReplication()),
+      FSEditLog.toLogTimeStamp(newNode.getModificationTime())};
     logEdit(OP_ADD,
             new ArrayWritable(UTF8.class, nameReplicationPair), 
             new ArrayWritable(Block.class, newNode.getBlocks()));
@@ -432,15 +488,23 @@
    * Add create directory record to edit log
    */
   void logMkDir(FSDirectory.INode newNode) {
-    logEdit(OP_MKDIR, new UTF8(newNode.computeName()), null);
+    UTF8 info[] = new UTF8[] {
+      new UTF8(newNode.computeName()),
+      FSEditLog.toLogTimeStamp(newNode.getModificationTime())
+    };
+    logEdit(OP_MKDIR, new ArrayWritable(UTF8.class, info), null);
   }
   
   /** 
    * Add rename record to edit log
    * TODO: use String parameters until just before writing to disk
    */
-  void logRename(String src, String dst) {
-    logEdit(OP_RENAME, new UTF8(src), new UTF8(dst));
+  void logRename(String src, String dst, long timestamp) {
+    UTF8 info[] = new UTF8[] { 
+      new UTF8(src),
+      new UTF8(dst),
+      FSEditLog.toLogTimeStamp(timestamp)};
+    logEdit(OP_RENAME, new ArrayWritable(UTF8.class, info), null);
   }
   
   /** 
@@ -455,8 +519,11 @@
   /** 
    * Add delete file record to edit log
    */
-  void logDelete(String src) {
-    logEdit(OP_DELETE, new UTF8(src), null);
+  void logDelete(String src, long timestamp) {
+    UTF8 info[] = new UTF8[] { 
+      new UTF8(src),
+      FSEditLog.toLogTimeStamp(timestamp)};
+    logEdit(OP_DELETE, new ArrayWritable(UTF8.class, info), null);
   }
   
   /** 
@@ -481,6 +548,10 @@
   
   static short fromLogReplication(UTF8 replication) {
     return Short.parseShort(replication.toString());
+  }
+
+  static UTF8 toLogTimeStamp(long timestamp) {
+    return new UTF8(Long.toString(timestamp));
   }
 
   /**

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSImage.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSImage.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSImage.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSImage.java Fri Jun 22 15:55:34 2007
@@ -647,12 +647,16 @@
       short replication = FSNamesystem.getFSNamesystem().getDefaultReplication();
       for (int i = 0; i < numFiles; i++) {
         UTF8 name = new UTF8();
+        long modificationTime = 0;
         name.readFields(in);
         // version 0 does not support per file replication
         if (!(imgVersion >= 0)) {
           replication = in.readShort(); // other versions do
           replication = FSEditLog.adjustReplication(replication);
         }
+        if (imgVersion <= -5) {
+          modificationTime = in.readLong();
+        }
         int numBlocks = in.readInt();
         Block blocks[] = null;
         if (numBlocks > 0) {
@@ -662,7 +666,8 @@
             blocks[j].readFields(in);
           }
         }
-        fsDir.unprotectedAddFile(name.toString(), blocks, replication);
+        fsDir.unprotectedAddFile(name.toString(), blocks, replication,
+                                 modificationTime);
       }
       
       // load datanode info
@@ -784,6 +789,7 @@
       fullName = parentPrefix + "/" + root.getLocalName();
       new UTF8(fullName).write(out);
       out.writeShort(root.getReplication());
+      out.writeLong(root.getModificationTime());
       if (root.isDir()) {
         out.writeInt(0);
       } else {

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Fri Jun 22 15:55:34 2007
@@ -1169,6 +1169,15 @@
     return dir.isDir(src);
   }
 
+  /* Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @throws IOException if file does not exist
+   * @return object containing information regarding the file
+   */
+  DFSFileInfo getFileInfo(String src) throws IOException {
+    return dir.getFileInfo(src);
+  }
+
   /**
    * Whether the pathname is valid.  Currently prohibits relative paths, 
    * and names which contain a ":" or "/" 
@@ -1213,7 +1222,7 @@
     if (!isValidName(src)) {
       throw new IOException("Invalid directory name: " + src);
     }
-    success = dir.mkdirs(src);
+    success = dir.mkdirs(src, now());
     if (!success) {
       throw new IOException("Invalid directory name: " + src);
     }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java Fri Jun 22 15:55:34 2007
@@ -470,6 +470,15 @@
     return files;
   }
 
+  /* Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @throws IOException if file does not exist
+   * @return object containing information regarding the file
+   */
+  public DFSFileInfo getFileInfo(String src)  throws IOException {
+    return namesystem.getFileInfo(src);
+  }
+
   /**
    */
   public long[] getStats() throws IOException {

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java?view=auto&rev=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileStatus.java Fri Jun 22 15:55:34 2007
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+
+import org.apache.hadoop.io.*;
+
+/** Interface that represents the client side information for a file.
+ */
+public interface FileStatus {
+
+  /* 
+   * @return the length of this file, in blocks
+   */
+  public long getLen();
+
+  /**
+   * Is this a directory?
+   * @return true if this is a directory
+   */
+  public boolean isDir();
+
+  /**
+   * Get the block size of the file.
+   * @return the number of bytes
+   */
+  public long getBlockSize();
+
+  /**
+   * Get the replication factor of a file.
+   * @return the replication factor of a file.
+   */
+  public short getReplication();
+
+  /**
+   * Get the modification time of the file.
+   * @return the modification time of file.
+   */
+  public long getModificationTime();
+}

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Fri Jun 22 15:55:34 2007
@@ -368,11 +368,15 @@
   /**
    * Get replication.
    * 
+   * @deprecated Use getFileStatus() instead
    * @param src file name
    * @return file replication
    * @throws IOException
-   */
-  public abstract short getReplication(Path src) throws IOException;
+   */ 
+  @Deprecated
+  public short getReplication(Path src) throws IOException {
+    return getFileStatus(src).getReplication();
+  }
 
   /**
    * Set replication for an existing file.
@@ -400,7 +404,14 @@
   public abstract boolean exists(Path f) throws IOException;
 
   /** True iff the named path is a directory. */
-  public abstract boolean isDirectory(Path f) throws IOException;
+  /** @deprecated Use getFileStatus() instead */ @Deprecated
+  public boolean isDirectory(Path f) throws IOException {
+    try {
+      return getFileStatus(f).isDir();
+    } catch (IOException e) {
+      return false;               // f does not exist
+    }
+  }
 
   /** True iff the named path is a regular file. */
   public boolean isFile(Path f) throws IOException {
@@ -412,7 +423,10 @@
   }
     
   /** The number of bytes in a file. */
-  public abstract long getLength(Path f) throws IOException;
+  /** @deprecated Use getFileStatus() instead */ @Deprecated
+  public long getLength(Path f) throws IOException {
+    return getFileStatus(f).getLen();
+  }
     
   /** Return the number of bytes of the given path 
    * If <i>f</i> is a file, return the size of the file;
@@ -817,7 +831,10 @@
    * @param f the filename
    * @return the number of bytes in a block
    */
-  public abstract long getBlockSize(Path f) throws IOException;
+  /** @deprecated Use getFileStatus() instead */ @Deprecated
+  public long getBlockSize(Path f) throws IOException {
+    return getFileStatus(f).getBlockSize();
+  }
     
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
@@ -831,4 +848,12 @@
    */
   public abstract short getDefaultReplication();
 
+  /* 
+   * Return a file status object that represents the
+   * file.
+   * @param f The path to the file we want information from
+   * @return filestatus object
+   * @throws IOException see specific implementation
+   */
+  public abstract FileStatus getFileStatus(Path f) throws IOException;
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java Fri Jun 22 15:55:34 2007
@@ -124,17 +124,6 @@
   }
 
   /**
-   * Get replication.
-   * 
-   * @param src file name
-   * @return file replication
-   * @throws IOException
-   */
-  public short getReplication(Path src) throws IOException {
-    return fs.getReplication(src);
-  }
-
-  /**
    * Set replication for an existing file.
    * 
    * @param src file name
@@ -167,16 +156,6 @@
     return fs.exists(f);
   }
 
-  /** True iff the named path is a directory. */
-  public boolean isDirectory(Path f) throws IOException {
-    return fs.isDirectory(f);
-  }
-
-  /** The number of bytes in a file. */
-  public long getLength(Path f) throws IOException {
-    return fs.getLength(f);
-  }
-  
   /** List files in a directory. */
   public Path[] listPaths(Path f) throws IOException {
     return fs.listPaths(f);
@@ -272,15 +251,6 @@
     fs.completeLocalOutput(fsOutputFile, tmpLocalFile);
   }
 
-  /**
-   * Get the block size for a particular file.
-   * @param f the filename
-   * @return the number of bytes in a block
-   */
-  public long getBlockSize(Path f) throws IOException {
-    return fs.getBlockSize(f);
-  }
-  
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
   public long getDefaultBlockSize() {
@@ -292,6 +262,13 @@
    */
   public short getDefaultReplication() {
     return fs.getDefaultReplication();
+  }
+
+  /**
+   * Get file status.
+   */
+  public FileStatus getFileStatus(Path f) throws IOException {
+    return fs.getFileStatus(f);
   }
 
   @Override

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java Fri Jun 22 15:55:34 2007
@@ -19,6 +19,7 @@
 
 import java.io.*;
 import java.util.*;
+import java.text.SimpleDateFormat;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.ipc.*;
@@ -29,6 +30,8 @@
 
   protected FileSystem fs;
   private Trash trash;
+  public static final SimpleDateFormat dateForm = 
+    new SimpleDateFormat("yyyy-MM-dd HH:mm");
 
   /**
    */
@@ -405,12 +408,15 @@
       }
       for (int i = 0; i < items.length; i++) {
         Path cur = items[i];
+        FileStatus stat = fs.getFileStatus(cur);
+        String mdate = dateForm.format(new Date(stat.getModificationTime()));
         System.out.println(cur.toUri().getPath() + "\t" 
-                           + (fs.isDirectory(cur) ? 
-                              "<dir>" : 
-                              ("<r " + fs.getReplication(cur) 
-                               + ">\t" + fs.getLength(cur))));
-        if (recursive && fs.isDirectory(cur)) {
+                           + (stat.isDir() ? 
+                              "<dir>\t" : 
+                              ("<r " + stat.getReplication() 
+                               + ">\t" + stat.getLen()))
+                           + "\t" + mdate);
+        if (recursive && stat.isDir()) {
           ls(cur, recursive, printHeader);
         }
       }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java Fri Jun 22 15:55:34 2007
@@ -233,7 +233,7 @@
     /**
      * Replication is not supported for the inmemory file system.
      */
-    public short getReplication(Path src) throws IOException {
+    public short getDefaultReplication() {
       return 1;
     }
 
@@ -275,23 +275,6 @@
     /**
      * Directory operations are not supported
      */
-    public boolean isDirectory(Path f) throws IOException {
-      return !isFile(f);
-    }
-
-    public boolean isFile(Path f) throws IOException {
-      return exists(f);
-    }
-
-    public long getLength(Path f) throws IOException {
-      synchronized (this) {
-        return pathToFileAttribs.get(getPath(f)).size;
-      }
-    }
-  
-    /**
-     * Directory operations are not supported
-     */
     public Path[] listPaths(Path f) throws IOException {
       return null;
     }
@@ -330,16 +313,14 @@
       throws IOException {
     }
 
-    public long getBlockSize(Path f) throws IOException {
-      return getDefaultBlockSize();
-    }
-
     public long getDefaultBlockSize() {
       return 32 * 1024; //some random large number. can be anything actually
     }
 
-    public short getDefaultReplication() {
-      return 1;
+    public FileStatus getFileStatus(Path f) throws IOException {
+      synchronized (this) {
+        return new InMemoryFileStatus(pathToFileAttribs.get(getPath(f)));
+      }
     }
   
     /** Some APIs exclusively for InMemoryFileSystem */
@@ -428,6 +409,29 @@
       public FileAttributes(int size) {
         this.size = size;
         this.data = new byte[size];
+      }
+    }
+
+    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
       }
     }
   }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Fri Jun 22 15:55:34 2007
@@ -195,10 +195,6 @@
   /**
    * Replication is not supported for the local file system.
    */
-  public short getReplication(Path f) throws IOException {
-    return 1;
-  }
-  
   /** Set the replication of the given file */
   public boolean setReplication(Path src,
                                 short replication
@@ -223,14 +219,6 @@
     return pathToFile(f).exists();
   }
   
-  public boolean isDirectory(Path f) throws IOException {
-    return pathToFile(f).isDirectory();
-  }
-  
-  public long getLength(Path f) throws IOException {
-    return pathToFile(f).length();
-  }
-  
   public Path[] listPaths(Path f) throws IOException {
     File localf = pathToFile(f);
     Path[] results;
@@ -366,12 +354,38 @@
     return "LocalFS";
   }
   
-  public long getBlockSize(Path filename) {
-    // local doesn't really do blocks, so just use the global number
-    return getDefaultBlockSize();
+  public FileStatus getFileStatus(Path f) throws IOException {
+    return new RawLocalFileStatus(pathToFile(f));
   }
-  
+
   public short getDefaultReplication() {
     return 1;
+  }
+  
+  private class RawLocalFileStatus implements FileStatus {
+    private long length;
+    private boolean isDir;
+    private long mtime;
+
+    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;
+    }
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java Fri Jun 22 15:55:34 2007
@@ -13,6 +13,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -122,15 +123,6 @@
   }
 
   @Override
-  public boolean isDirectory(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      return false;
-    }
-    return inode.isDirectory();
-  }
-
-  @Override
   public boolean isFile(Path path) throws IOException {
     INode inode = store.retrieveINode(makeAbsolute(path));
     if (inode == null) {
@@ -270,28 +262,25 @@
     return true;
   }
 
-  @Override
-  public long getLength(Path path) throws IOException {
-    INode inode = checkFile(path);
-    long length = 0;
-    for (Block block : inode.getBlocks()) {
-      length += block.getLength();
-    }
-    return length;
-  }
-
   /**
    * Replication is not supported for S3 file systems since S3 handles it for
    * us.
    */
   @Override
-  public short getReplication(Path path) throws IOException {
+  public short getDefaultReplication() {
     return 1;
   }
 
+  /**
+   * FileStatus for S3 file systems. 
+   */
   @Override
-  public short getDefaultReplication() {
-    return 1;
+  public FileStatus getFileStatus(Path f)  throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(f));
+    if (inode == null) {
+      throw new IOException(f.toString() + ": No such file or directory.");
+    }
+    return new S3FileStatus(inode);
   }
 
   /**
@@ -305,19 +294,6 @@
   }
 
   @Override
-  public long getBlockSize(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      throw new IOException(path.toString() + ": No such file or directory.");
-    }
-    Block[] blocks = inode.getBlocks();
-    if (blocks == null || blocks.length == 0) {
-      return 0;
-    }
-    return blocks[0].getLength();
-  }
-
-  @Override
   public long getDefaultBlockSize() {
     return getConf().getLong("fs.s3.block.size", DEFAULT_BLOCK_SIZE);
   }
@@ -379,4 +355,35 @@
     store.purge();
   }
 
+  private static class S3FileStatus implements FileStatus {
+    private long length = 0, blockSize = 0;
+    private boolean isDir;
+
+    S3FileStatus(INode inode) throws IOException {
+      isDir = inode.isDirectory();
+      if (!isDir) {
+        for (Block block : inode.getBlocks()) {
+          length += block.getLength();
+          if (blockSize == 0) {
+            blockSize = block.getLength();
+          }
+        }
+      }
+    }
+    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
+    }
+  }
 }

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestModTime.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestModTime.java?view=auto&rev=549977
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestModTime.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestModTime.java Fri Jun 22 15:55:34 2007
@@ -0,0 +1,181 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs;
+
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.Random;
+import java.net.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * This class tests the decommissioning of nodes.
+ * @author Dhruba Borthakur
+ */
+public class TestModTime extends TestCase {
+  static final long seed = 0xDEADBEEFL;
+  static final int blockSize = 8192;
+  static final int fileSize = 16384;
+  static final int numDatanodes = 6;
+
+
+  Random myrand = new Random();
+  Path hostsFile;
+  Path excludeFile;
+
+  private void writeFile(FileSystem fileSys, Path name, int repl)
+    throws IOException {
+    // create and write a file that contains three blocks of data
+    FSDataOutputStream stm = fileSys.create(name, true, 
+                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),
+                                            (short)repl, (long)blockSize);
+    byte[] buffer = new byte[fileSize];
+    Random rand = new Random(seed);
+    rand.nextBytes(buffer);
+    stm.write(buffer);
+    stm.close();
+  }
+  
+  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
+    assertTrue(fileSys.exists(name));
+    fileSys.delete(name);
+    assertTrue(!fileSys.exists(name));
+  }
+
+  private void printDatanodeReport(DatanodeInfo[] info) {
+    System.out.println("-------------------------------------------------");
+    for (int i = 0; i < info.length; i++) {
+      System.out.println(info[i].getDatanodeReport());
+      System.out.println();
+    }
+  }
+
+  /**
+   * Tests modification time in DFS.
+   */
+  public void testModTime() throws IOException {
+    Configuration conf = new Configuration();
+
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    cluster.waitActive();
+    InetSocketAddress addr = new InetSocketAddress("localhost", 
+                                                   cluster.getNameNodePort());
+    DFSClient client = new DFSClient(addr, conf);
+    DatanodeInfo[] info = client.datanodeReport();
+    assertEquals("Number of Datanodes ", numDatanodes, info.length);
+    FileSystem fileSys = cluster.getFileSystem();
+    int replicas = numDatanodes - 1;
+    assertTrue(fileSys instanceof DistributedFileSystem);
+
+    try {
+
+     //
+     // create file and record ctime and mtime of test file
+     //
+     System.out.println("Creating testdir1 and testdir1/test1.dat.");
+     Path dir1 = new Path("testdir1");
+     Path file1 = new Path(dir1, "test1.dat");
+     writeFile(fileSys, file1, replicas);
+     FileStatus stat = fileSys.getFileStatus(file1);
+     long mtime1 = stat.getModificationTime();
+     assertTrue(mtime1 != 0);
+     //
+     // record dir times
+     //
+     stat = fileSys.getFileStatus(dir1);
+     long mdir1 = stat.getModificationTime();
+
+     //
+     // create second test file
+     //
+     System.out.println("Creating testdir1/test2.dat.");
+     Path file2 = new Path(dir1, "test2.dat");
+     writeFile(fileSys, file2, replicas);
+     stat = fileSys.getFileStatus(file2);
+
+     //
+     // verify that mod time of dir remains the same
+     // as before. modification time of directory has increased.
+     //
+     stat = fileSys.getFileStatus(dir1);
+     assertTrue(stat.getModificationTime() >= mdir1);
+     mdir1 = stat.getModificationTime();
+     //
+     // create another directory
+     //
+     Path dir2 = (new Path("testdir2/")).makeQualified(fileSys);
+     System.out.println("Creating testdir2 " + dir2);
+     assertTrue(fileSys.mkdirs(dir2));
+     stat = fileSys.getFileStatus(dir2);
+     long mdir2 = stat.getModificationTime();
+     //
+     // rename file1 from testdir into testdir2
+     //
+     Path newfile = new Path(dir2, "testnew.dat");
+     System.out.println("Moving " + file1 + " to " + newfile);
+     fileSys.rename(file1, newfile);
+     //
+     // verify that modification time of file1 did not change.
+     //
+     stat = fileSys.getFileStatus(newfile);
+     assertTrue(stat.getModificationTime() == mtime1);
+     //
+     // verify that modification time of  testdir1 and testdir2
+     // were changed. 
+     //
+     stat = fileSys.getFileStatus(dir1);
+     assertTrue(stat.getModificationTime() != mdir1);
+     mdir1 = stat.getModificationTime();
+
+     stat = fileSys.getFileStatus(dir2);
+     assertTrue(stat.getModificationTime() != mdir2);
+     mdir2 = stat.getModificationTime();
+     //
+     // delete newfile
+     //
+     System.out.println("Deleting testdir2/testnew.dat.");
+     assertTrue(fileSys.delete(newfile));
+     //
+     // verify that modification time of testdir1 has not changed.
+     //
+     stat = fileSys.getFileStatus(dir1);
+     assertTrue(stat.getModificationTime() == mdir1);
+     //
+     // verify that modification time of testdir2 has changed.
+     //
+     stat = fileSys.getFileStatus(dir2);
+     assertTrue(stat.getModificationTime() != mdir2);
+     mdir2 = stat.getModificationTime();
+
+     cleanupFile(fileSys, file2);
+     cleanupFile(fileSys, dir1);
+     cleanupFile(fileSys, dir2);
+    } catch (IOException e) {
+      info = client.datanodeReport();
+      printDatanodeReport(info);
+      throw e;
+    } finally {
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+}

Modified: lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp?view=diff&rev=549977&r1=549976&r2=549977
==============================================================================
--- lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/datanode/browseDirectory.jsp Fri Jun 22 15:55:34 2007
@@ -69,9 +69,10 @@
       // directory
       DFSFileInfo[] files = dfs.listPaths(target);
       //generate a table and dump the info
-      String [] headings = new String[5];
+      String [] headings = new String[7];
       headings[0] = "Name"; headings[1] = "Type"; headings[2] = "Size";
       headings[3] = "Replication"; headings[4] = "BlockSize";
+      headings[5] = "Creation Time"; headings[6] = "Modification Time";
       out.print("<h3>Contents of directory ");
       JspHelper.printPathWithLinks(dir, out, namenodeInfoPort);
       out.print("</h3><hr>");
@@ -92,7 +93,7 @@
         jspHelper.addTableHeader(out);
         int row=0;
         jspHelper.addTableRow(out, headings, row++);
-        String cols [] = new String[5];
+        String cols [] = new String[6];
         for (int i = 0; i < files.length; i++) {
           //Get the location of the first block of the file
           if (files[i].getPath().endsWith(".crc")) continue;
@@ -113,6 +114,7 @@
             cols[2] = FsShell.byteDesc(files[i].getLen());
             cols[3] = Short.toString(files[i].getReplication());
             cols[4] = FsShell.byteDesc(files[i].getBlockSize());
+            cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           }
           else {
             String datanodeUrl = req.getRequestURL()+"?dir="+
@@ -123,6 +125,7 @@
             cols[2] = "";
             cols[3] = "";
             cols[4] = "";
+            cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           }
           jspHelper.addTableRow(out, cols, row++);
         }