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 ha...@apache.org on 2008/09/17 01:15:44 UTC

svn commit: r696095 - in /hadoop/core/trunk: ./ src/hdfs/org/apache/hadoop/hdfs/ src/hdfs/org/apache/hadoop/hdfs/protocol/ src/hdfs/org/apache/hadoop/hdfs/server/namenode/ src/test/org/apache/hadoop/hdfs/ src/webapps/datanode/

Author: hairong
Date: Tue Sep 16 16:15:44 2008
New Revision: 696095

URL: http://svn.apache.org/viewvc?rev=696095&view=rev
Log:
HADOOP-4007. Remove DFSFileInfo. Contributed by Sanjay Radia.

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
    hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
    hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestFileStatus.java
    hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue Sep 16 16:15:44 2008
@@ -62,6 +62,9 @@
     JobTracker start time to disinguish attempts of the same TIP across 
     restarts. (Amar Ramesh Kamat via ddas)
 
+    HADOOP-4007. REMOVE DFSFileInfo - FileStatus is sufficient. 
+    (Sanjay Radia via hairong)
+
   NEW FEATURES
 
     HADOOP-3341. Allow streaming jobs to specify the field separator for map

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java Tue Sep 16 16:15:44 2008
@@ -470,7 +470,7 @@
   OutputStream append(String src, int buffersize, Progressable progress
       ) throws IOException {
     checkOpen();
-    DFSFileInfo stat = null;
+    FileStatus stat = null;
     LocatedBlock lastBlock = null;
     try {
       stat = getFileInfo(src);
@@ -561,7 +561,7 @@
 
   /**
    */
-  public DFSFileInfo[] listPaths(String src) throws IOException {
+  public FileStatus[] listPaths(String src) throws IOException {
     checkOpen();
     try {
       return namenode.getListing(src);
@@ -570,7 +570,7 @@
     }
   }
 
-  public DFSFileInfo getFileInfo(String src) throws IOException {
+  public FileStatus getFileInfo(String src) throws IOException {
     checkOpen();
     try {
       return namenode.getFileInfo(src);
@@ -2546,7 +2546,7 @@
      * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
      */
     DFSOutputStream(String src, int buffersize, Progressable progress,
-        LocatedBlock lastBlock, DFSFileInfo stat,
+        LocatedBlock lastBlock, FileStatus stat,
         int bytesPerChecksum) throws IOException {
       this(src, stat.getBlockSize(), progress, bytesPerChecksum);
 

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java Tue Sep 16 16:15:44 2008
@@ -24,7 +24,6 @@
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -234,7 +233,7 @@
   }
 
   public FileStatus[] listStatus(Path p) throws IOException {
-    DFSFileInfo[] infos = dfs.listPaths(getPathName(p));
+    FileStatus[] infos = dfs.listPaths(getPathName(p));
     if (infos == null) return null;
     FileStatus[] stats = new FileStatus[infos.length];
     for (int i = 0; i < infos.length; i++) {
@@ -398,7 +397,7 @@
    * @throws FileNotFoundException if the file does not exist.
    */
   public FileStatus getFileStatus(Path f) throws IOException {
-    DFSFileInfo fi = dfs.getFileInfo(getPathName(f));
+    FileStatus fi = dfs.getFileInfo(getPathName(f));
     if (fi != null) {
       return makeQualified(fi);
     } else {

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Tue Sep 16 16:15:44 2008
@@ -24,6 +24,7 @@
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
 
 /**********************************************************************
  * ClientProtocol is used by user code via 
@@ -38,10 +39,9 @@
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
-   * 38 : getEditLogSize(), rollEditLog(), rollFSImage() 
-   * moved to NamenodeProtocol.
+   * 39: removed DFSFileInfo (Use FileStatus instead)
    */
-  public static final long versionID = 38L;
+  public static final long versionID = 39L;
   
   ///////////////////////////////////////
   // File contents
@@ -253,7 +253,7 @@
   /**
    * Get a listing of the indicated directory
    */
-  public DFSFileInfo[] getListing(String src) throws IOException;
+  public FileStatus[] getListing(String src) throws IOException;
 
   ///////////////////////////////////////
   // System issues and management
@@ -398,7 +398,7 @@
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  public DFSFileInfo getFileInfo(String src) throws IOException;
+  public FileStatus getFileInfo(String src) throws IOException;
 
   /**
    * Get {@link ContentSummary} rooted at the specified directory.

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java Tue Sep 16 16:15:44 2008
@@ -1,84 +0,0 @@
-/**
- * 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.hdfs.protocol;
-
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
-
-/******************************************************
- * DFSFileInfo tracks info about remote files, including
- * name, size, etc.
- * 
- * Includes partial information about its blocks.
- * Block locations are sorted by the distance to the current client.
- * 
- ******************************************************/
-public class DFSFileInfo extends FileStatus {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DFSFileInfo.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DFSFileInfo(); }
-       });
-  }
-
-  /**
-   */
-  public DFSFileInfo() {
-  }
-
-  /**
-   * Create DFSFileInfo by file INode 
-   */
-  public DFSFileInfo(String path, INode node) {
-    // length is zero for directories
-    super(node.isDirectory() ? 0 : node.computeContentSummary().getLength(), 
-          node.isDirectory(), 
-          node.isDirectory() ? 0 : ((INodeFile)node).getReplication(), 
-          node.isDirectory() ? 0 : ((INodeFile)node).getPreferredBlockSize(),
-          node.getModificationTime(),
-          node.getAccessTime(),
-          node.getFsPermission(),
-          node.getUserName(),
-          node.getGroupName(),
-          new Path(path));
-  }
-
-  /**
-   */
-  public String getName() {
-    return getPath().getName();
-  }
-  
-  /**
-   */
-  public String getParent() {
-    return getPath().getParent().toString();
-  }
-
-  /**
-   * @deprecated use {@link #getLen()} instead
-   */
-  public long getContentsLen() {
-    assert isDir() : "Must be a directory";
-    return getLen();
-  }
-}

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Tue Sep 16 16:15:44 2008
@@ -21,13 +21,13 @@
 import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -657,7 +657,7 @@
    * This function is admittedly very inefficient right now.  We'll
    * make it better later.
    */
-  public DFSFileInfo[] getListing(String src) {
+  public FileStatus[] getListing(String src) {
     String srcs = normalizePath(src);
 
     synchronized (rootDir) {
@@ -665,15 +665,15 @@
       if (targetNode == null)
         return null;
       if (!targetNode.isDirectory()) {
-        return new DFSFileInfo[]{new DFSFileInfo(srcs, targetNode)};
+        return new FileStatus[]{createFileStatus(srcs, targetNode)};
       }
       List<INode> contents = ((INodeDirectory)targetNode).getChildren();
-      DFSFileInfo listing[] = new DFSFileInfo[contents.size()];
+      FileStatus listing[] = new FileStatus[contents.size()];
       if(! srcs.endsWith(Path.SEPARATOR))
         srcs += Path.SEPARATOR;
       int i = 0;
       for (INode cur : contents) {
-        listing[i] = new DFSFileInfo(srcs+cur.getLocalName(), cur);
+        listing[i] = createFileStatus(srcs+cur.getLocalName(), cur);
         i++;
       }
       return listing;
@@ -685,7 +685,7 @@
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  DFSFileInfo getFileInfo(String src) {
+  FileStatus getFileInfo(String src) {
     String srcs = normalizePath(src);
     synchronized (rootDir) {
       INode targetNode = rootDir.getNode(srcs);
@@ -693,7 +693,7 @@
         return null;
       }
       else {
-        return new DFSFileInfo(srcs, targetNode);
+        return createFileStatus(srcs, targetNode);
       }
     }
   }
@@ -1106,4 +1106,21 @@
     } 
     return status;
   }
+  
+  /**
+   * Create FileStatus by file INode 
+   */
+   private static FileStatus createFileStatus(String path, INode node) {
+    // length is zero for directories
+    return new FileStatus(node.isDirectory() ? 0 : node.computeContentSummary().getLength(), 
+        node.isDirectory(), 
+        node.isDirectory() ? 0 : ((INodeFile)node).getReplication(), 
+        node.isDirectory() ? 0 : ((INodeFile)node).getPreferredBlockSize(),
+        node.getModificationTime(),
+        node.getAccessTime(),
+        node.getFsPermission(),
+        node.getUserName(),
+        node.getGroupName(),
+        new Path(path));
+  }
 }

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Tue Sep 16 16:15:44 2008
@@ -41,8 +41,8 @@
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 
 /**
  * FSEditLog maintains a log of the namespace modifications.
@@ -652,7 +652,7 @@
           String s = FSImage.readString(in);
           String d = FSImage.readString(in);
           timestamp = readLong(in);
-          DFSFileInfo dinfo = fsDir.getFileInfo(d);
+          FileStatus dinfo = fsDir.getFileInfo(d);
           fsDir.unprotectedRenameTo(s, d, timestamp);
           fsNamesys.changeLease(s, d, dinfo);
           break;

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue Sep 16 16:15:44 2008
@@ -1596,7 +1596,7 @@
       checkAncestorAccess(actualdst, FsAction.WRITE);
     }
 
-    DFSFileInfo dinfo = dir.getFileInfo(dst);
+    FileStatus dinfo = dir.getFileInfo(dst);
     if (dir.renameTo(src, dst)) {
       changeLease(src, dst, dinfo);     // update lease with new filename
       return true;
@@ -1663,7 +1663,7 @@
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  DFSFileInfo getFileInfo(String src) throws IOException {
+  FileStatus getFileInfo(String src) throws IOException {
     if (isPermissionEnabled) {
       checkTraverse(src);
     }
@@ -1927,7 +1927,7 @@
    * Get a listing of all files at 'src'.  The Object[] array
    * exists so we can return file attributes (soon to be implemented)
    */
-  public DFSFileInfo[] getListing(String src) throws IOException {
+  public FileStatus[] getListing(String src) throws IOException {
     if (isPermissionEnabled) {
       if (dir.isDir(src)) {
         checkPathAccess(src, FsAction.READ_EXECUTE);
@@ -4444,7 +4444,7 @@
   // rename was successful. If any part of the renamed subtree had
   // files that were being written to, update with new filename.
   //
-  void changeLease(String src, String dst, DFSFileInfo dinfo) 
+  void changeLease(String src, String dst, FileStatus dinfo) 
                    throws IOException {
     String overwrite;
     String replaceBy;

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java Tue Sep 16 16:15:44 2008
@@ -20,16 +20,11 @@
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -40,7 +35,7 @@
  */
 public class FileDataServlet extends DfsServlet {
 
-  private URI createUri(DFSFileInfo i, UnixUserGroupInformation ugi,
+  private URI createUri(FileStatus i, UnixUserGroupInformation ugi,
       ClientProtocol nnproxy, String scheme)
       throws IOException, URISyntaxException {
     final DatanodeID host = pickSrcDatanode(i, nnproxy);
@@ -63,7 +58,7 @@
    * Currently, this looks at no more than the first five blocks of a file,
    * selecting a datanode randomly from the most represented.
    */
-  private static DatanodeID pickSrcDatanode(DFSFileInfo i,
+  private static DatanodeID pickSrcDatanode(FileStatus i,
       ClientProtocol nnproxy) throws IOException {
     final LocatedBlocks blks = nnproxy.getBlockLocations(
         i.getPath().toUri().getPath(), 0, 1);
@@ -89,7 +84,7 @@
     try {
       final String path = request.getPathInfo() != null
         ? request.getPathInfo() : "/";
-      DFSFileInfo info = nnproxy.getFileInfo(path);
+      FileStatus info = nnproxy.getFileInfo(path);
       if ((info != null) && !info.isDir()) {
         response.sendRedirect(createUri(info, ugi, nnproxy,
               request.getScheme()).toURL().toString());

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java Tue Sep 16 16:15:44 2008
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.VersionInfo;
@@ -58,7 +58,7 @@
    * Node information includes path, modification, permission, owner and group.
    * For files, it also includes size, replication and block-size. 
    */
-  static void writeInfo(DFSFileInfo i, XMLOutputter doc) throws IOException {
+  static void writeInfo(FileStatus i, XMLOutputter doc) throws IOException {
     doc.startTag(i.isDir() ? "directory" : "file");
     doc.attribute("path", i.getPath().toUri().getPath());
     doc.attribute("modified", df.format(new Date(i.getModificationTime())));
@@ -140,7 +140,7 @@
         doc.attribute(m.getKey(), m.getValue());
       }
 
-      DFSFileInfo base = nnproxy.getFileInfo(path);
+      FileStatus base = nnproxy.getFileInfo(path);
       if ((base != null) && base.isDir()) {
         writeInfo(base, doc);
       }
@@ -150,9 +150,9 @@
       while (!pathstack.empty()) {
         String p = pathstack.pop();
         try {
-          for (DFSFileInfo i : nnproxy.getListing(p)) {
-            if (exclude.matcher(i.getName()).matches()
-                || !filter.matcher(i.getName()).matches()) {
+          for (FileStatus i : nnproxy.getListing(p)) {
+            if (exclude.matcher(i.getPath().getName()).matches()
+                || !filter.matcher(i.getPath().getName()).matches()) {
               continue;
             }
             if (recur && i.isDir()) {

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Tue Sep 16 16:15:44 2008
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.FileSystem;
@@ -478,8 +479,8 @@
 
   /**
    */
-  public DFSFileInfo[] getListing(String src) throws IOException {
-    DFSFileInfo[] files = namesystem.getListing(src);
+  public FileStatus[] getListing(String src) throws IOException {
+    FileStatus[] files = namesystem.getListing(src);
     if (files != null) {
       myMetrics.numGetListingOps.inc();
     }
@@ -493,7 +494,7 @@
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  public DFSFileInfo getFileInfo(String src)  throws IOException {
+  public FileStatus getFileInfo(String src)  throws IOException {
     return namesystem.getFileInfo(src);
   }
 

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java Tue Sep 16 16:15:44 2008
@@ -35,12 +35,12 @@
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 
 /**
@@ -124,7 +124,7 @@
    */
   public void fsck() throws IOException {
     try {
-      DFSFileInfo[] files = nn.namesystem.dir.getListing(path);
+      FileStatus[] files = nn.namesystem.dir.getListing(path);
       FsckResult res = new FsckResult();
       res.totalRacks = nn.getNetworkTopology().getNumOfRacks();
       res.totalDatanodes = nn.namesystem.getNumberOfDatanodes(
@@ -151,13 +151,13 @@
     }
   }
   
-  private void check(DFSFileInfo file, FsckResult res) throws IOException {
+  private void check(FileStatus file, FsckResult res) throws IOException {
     int minReplication = nn.namesystem.getMinReplication();
     String path = file.getPath().toString();
     boolean isOpen = false;
 
     if (file.isDir()) {
-      DFSFileInfo[] files = nn.namesystem.dir.getListing(path);
+      FileStatus[] files = nn.namesystem.dir.getListing(path);
       if (files == null) {
         return;
       }
@@ -301,7 +301,7 @@
     }
   }
   
-  private void lostFoundMove(DFSFileInfo file, LocatedBlocks blocks)
+  private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
     throws IOException {
     DFSClient dfs = new DFSClient(conf);
     if (!lfInited) {

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java Tue Sep 16 16:15:44 2008
@@ -28,8 +28,8 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
 import org.apache.commons.logging.Log;
@@ -130,10 +130,10 @@
   private void verifyDir(DFSClient client, String dir) 
                                            throws IOException {
     
-    DFSFileInfo[] fileArr = client.listPaths(dir);
+    FileStatus[] fileArr = client.listPaths(dir);
     TreeMap<String, Boolean> fileMap = new TreeMap<String, Boolean>();
     
-    for(DFSFileInfo file : fileArr) {
+    for(FileStatus file : fileArr) {
       String path = file.getPath().toString();
       fileMap.put(path, Boolean.valueOf(file.isDir()));
     }

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestFileStatus.java?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestFileStatus.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/hdfs/TestFileStatus.java Tue Sep 16 16:15:44 2008
@@ -22,13 +22,10 @@
 import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 
 /**
  * This class tests the FileStatus API.
@@ -83,7 +80,7 @@
                  fs.getFileStatus(path).isDir() == true);
       
       // make sure getFileInfo returns null for files which do not exist
-      DFSFileInfo fileInfo = dfsClient.getFileInfo("/noSuchFile");
+      FileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
       assertTrue(fileInfo == null);
 
       // create a file in home directory

Modified: hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp?rev=696095&r1=696094&r2=696095&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp (original)
+++ hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp Tue Sep 16 16:15:44 2008
@@ -76,7 +76,7 @@
         return;
       }
       // directory
-      DFSFileInfo[] files = dfs.listPaths(target);
+      FileStatus[] files = dfs.listPaths(target);
       //generate a table and dump the info
       String [] headings = { "Name", "Type", "Size", "Replication", 
                               "Block Size", "Modification Time",
@@ -120,7 +120,7 @@
           String datanodeUrl = req.getRequestURL()+"?dir="+
               URLEncoder.encode(files[i].getPath().toString(), "UTF-8") + 
               "&namenodeInfoPort=" + namenodeInfoPort;
-          cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getName()+"</a>";
+          cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath().getName()+"</a>";
           cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           cols[6] = files[i].getPermission().toString();
           cols[7] = files[i].getOwner();