浏览代码

HADOOP-4007. Remove DFSFileInfo. Contributed by Sanjay Radia.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@696095 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 年之前
父节点
当前提交
1324beef68

+ 3 - 0
CHANGES.txt

@@ -62,6 +62,9 @@ Trunk (unreleased changes)
     JobTracker start time to disinguish attempts of the same TIP across 
     JobTracker start time to disinguish attempts of the same TIP across 
     restarts. (Amar Ramesh Kamat via ddas)
     restarts. (Amar Ramesh Kamat via ddas)
 
 
+    HADOOP-4007. REMOVE DFSFileInfo - FileStatus is sufficient. 
+    (Sanjay Radia via hairong)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-3341. Allow streaming jobs to specify the field separator for map
     HADOOP-3341. Allow streaming jobs to specify the field separator for map

+ 4 - 4
src/hdfs/org/apache/hadoop/hdfs/DFSClient.java

@@ -470,7 +470,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   OutputStream append(String src, int buffersize, Progressable progress
   OutputStream append(String src, int buffersize, Progressable progress
       ) throws IOException {
       ) throws IOException {
     checkOpen();
     checkOpen();
-    DFSFileInfo stat = null;
+    FileStatus stat = null;
     LocatedBlock lastBlock = null;
     LocatedBlock lastBlock = null;
     try {
     try {
       stat = getFileInfo(src);
       stat = getFileInfo(src);
@@ -561,7 +561,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
 
   /**
   /**
    */
    */
-  public DFSFileInfo[] listPaths(String src) throws IOException {
+  public FileStatus[] listPaths(String src) throws IOException {
     checkOpen();
     checkOpen();
     try {
     try {
       return namenode.getListing(src);
       return namenode.getListing(src);
@@ -570,7 +570,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     }
     }
   }
   }
 
 
-  public DFSFileInfo getFileInfo(String src) throws IOException {
+  public FileStatus getFileInfo(String src) throws IOException {
     checkOpen();
     checkOpen();
     try {
     try {
       return namenode.getFileInfo(src);
       return namenode.getFileInfo(src);
@@ -2546,7 +2546,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
      * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
      * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
      */
      */
     DFSOutputStream(String src, int buffersize, Progressable progress,
     DFSOutputStream(String src, int buffersize, Progressable progress,
-        LocatedBlock lastBlock, DFSFileInfo stat,
+        LocatedBlock lastBlock, FileStatus stat,
         int bytesPerChecksum) throws IOException {
         int bytesPerChecksum) throws IOException {
       this(src, stat.getBlockSize(), progress, bytesPerChecksum);
       this(src, stat.getBlockSize(), progress, bytesPerChecksum);
 
 

+ 2 - 3
src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -24,7 +24,6 @@ import java.net.*;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -234,7 +233,7 @@ public class DistributedFileSystem extends FileSystem {
   }
   }
 
 
   public FileStatus[] listStatus(Path p) throws IOException {
   public FileStatus[] listStatus(Path p) throws IOException {
-    DFSFileInfo[] infos = dfs.listPaths(getPathName(p));
+    FileStatus[] infos = dfs.listPaths(getPathName(p));
     if (infos == null) return null;
     if (infos == null) return null;
     FileStatus[] stats = new FileStatus[infos.length];
     FileStatus[] stats = new FileStatus[infos.length];
     for (int i = 0; i < infos.length; i++) {
     for (int i = 0; i < infos.length; i++) {
@@ -398,7 +397,7 @@ public class DistributedFileSystem extends FileSystem {
    * @throws FileNotFoundException if the file does not exist.
    * @throws FileNotFoundException if the file does not exist.
    */
    */
   public FileStatus getFileStatus(Path f) throws IOException {
   public FileStatus getFileStatus(Path f) throws IOException {
-    DFSFileInfo fi = dfs.getFileInfo(getPathName(f));
+    FileStatus fi = dfs.getFileInfo(getPathName(f));
     if (fi != null) {
     if (fi != null) {
       return makeQualified(fi);
       return makeQualified(fi);
     } else {
     } else {

+ 5 - 5
src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
 
 
 /**********************************************************************
 /**********************************************************************
  * ClientProtocol is used by user code via 
  * ClientProtocol is used by user code via 
@@ -38,10 +39,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * Compared to the previous version the following changes have been introduced:
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
    * 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
   // File contents
@@ -253,7 +253,7 @@ public interface ClientProtocol extends VersionedProtocol {
   /**
   /**
    * Get a listing of the indicated directory
    * 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
   // System issues and management
@@ -398,7 +398,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         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.
    * Get {@link ContentSummary} rooted at the specified directory.

+ 0 - 84
src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java

@@ -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();
-  }
-}

+ 24 - 7
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -21,13 +21,13 @@ import java.io.*;
 import java.util.*;
 import java.util.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.MetricsContext;
 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.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -657,7 +657,7 @@ class FSDirectory implements FSConstants, Closeable {
    * This function is admittedly very inefficient right now.  We'll
    * This function is admittedly very inefficient right now.  We'll
    * make it better later.
    * make it better later.
    */
    */
-  public DFSFileInfo[] getListing(String src) {
+  public FileStatus[] getListing(String src) {
     String srcs = normalizePath(src);
     String srcs = normalizePath(src);
 
 
     synchronized (rootDir) {
     synchronized (rootDir) {
@@ -665,15 +665,15 @@ class FSDirectory implements FSConstants, Closeable {
       if (targetNode == null)
       if (targetNode == null)
         return null;
         return null;
       if (!targetNode.isDirectory()) {
       if (!targetNode.isDirectory()) {
-        return new DFSFileInfo[]{new DFSFileInfo(srcs, targetNode)};
+        return new FileStatus[]{createFileStatus(srcs, targetNode)};
       }
       }
       List<INode> contents = ((INodeDirectory)targetNode).getChildren();
       List<INode> contents = ((INodeDirectory)targetNode).getChildren();
-      DFSFileInfo listing[] = new DFSFileInfo[contents.size()];
+      FileStatus listing[] = new FileStatus[contents.size()];
       if(! srcs.endsWith(Path.SEPARATOR))
       if(! srcs.endsWith(Path.SEPARATOR))
         srcs += Path.SEPARATOR;
         srcs += Path.SEPARATOR;
       int i = 0;
       int i = 0;
       for (INode cur : contents) {
       for (INode cur : contents) {
-        listing[i] = new DFSFileInfo(srcs+cur.getLocalName(), cur);
+        listing[i] = createFileStatus(srcs+cur.getLocalName(), cur);
         i++;
         i++;
       }
       }
       return listing;
       return listing;
@@ -685,7 +685,7 @@ class FSDirectory implements FSConstants, Closeable {
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         or null if file not found
    */
    */
-  DFSFileInfo getFileInfo(String src) {
+  FileStatus getFileInfo(String src) {
     String srcs = normalizePath(src);
     String srcs = normalizePath(src);
     synchronized (rootDir) {
     synchronized (rootDir) {
       INode targetNode = rootDir.getNode(srcs);
       INode targetNode = rootDir.getNode(srcs);
@@ -693,7 +693,7 @@ class FSDirectory implements FSConstants, Closeable {
         return null;
         return null;
       }
       }
       else {
       else {
-        return new DFSFileInfo(srcs, targetNode);
+        return createFileStatus(srcs, targetNode);
       }
       }
     }
     }
   }
   }
@@ -1106,4 +1106,21 @@ class FSDirectory implements FSConstants, Closeable {
     } 
     } 
     return status;
     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));
+  }
 }
 }

+ 2 - 2
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -41,8 +41,8 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 
 
 /**
 /**
  * FSEditLog maintains a log of the namespace modifications.
  * FSEditLog maintains a log of the namespace modifications.
@@ -652,7 +652,7 @@ public class FSEditLog {
           String s = FSImage.readString(in);
           String s = FSImage.readString(in);
           String d = FSImage.readString(in);
           String d = FSImage.readString(in);
           timestamp = readLong(in);
           timestamp = readLong(in);
-          DFSFileInfo dinfo = fsDir.getFileInfo(d);
+          FileStatus dinfo = fsDir.getFileInfo(d);
           fsDir.unprotectedRenameTo(s, d, timestamp);
           fsDir.unprotectedRenameTo(s, d, timestamp);
           fsNamesys.changeLease(s, d, dinfo);
           fsNamesys.changeLease(s, d, dinfo);
           break;
           break;

+ 4 - 4
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1596,7 +1596,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       checkAncestorAccess(actualdst, FsAction.WRITE);
       checkAncestorAccess(actualdst, FsAction.WRITE);
     }
     }
 
 
-    DFSFileInfo dinfo = dir.getFileInfo(dst);
+    FileStatus dinfo = dir.getFileInfo(dst);
     if (dir.renameTo(src, dst)) {
     if (dir.renameTo(src, dst)) {
       changeLease(src, dst, dinfo);     // update lease with new filename
       changeLease(src, dst, dinfo);     // update lease with new filename
       return true;
       return true;
@@ -1663,7 +1663,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         or null if file not found
    */
    */
-  DFSFileInfo getFileInfo(String src) throws IOException {
+  FileStatus getFileInfo(String src) throws IOException {
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
       checkTraverse(src);
       checkTraverse(src);
     }
     }
@@ -1927,7 +1927,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * Get a listing of all files at 'src'.  The Object[] array
    * Get a listing of all files at 'src'.  The Object[] array
    * exists so we can return file attributes (soon to be implemented)
    * 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 (isPermissionEnabled) {
       if (dir.isDir(src)) {
       if (dir.isDir(src)) {
         checkPathAccess(src, FsAction.READ_EXECUTE);
         checkPathAccess(src, FsAction.READ_EXECUTE);
@@ -4444,7 +4444,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   // rename was successful. If any part of the renamed subtree had
   // rename was successful. If any part of the renamed subtree had
   // files that were being written to, update with new filename.
   // 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 {
                    throws IOException {
     String overwrite;
     String overwrite;
     String replaceBy;
     String replaceBy;

+ 4 - 9
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -20,16 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 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.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 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.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -40,7 +35,7 @@ import org.apache.hadoop.security.UnixUserGroupInformation;
  */
  */
 public class FileDataServlet extends DfsServlet {
 public class FileDataServlet extends DfsServlet {
 
 
-  private URI createUri(DFSFileInfo i, UnixUserGroupInformation ugi,
+  private URI createUri(FileStatus i, UnixUserGroupInformation ugi,
       ClientProtocol nnproxy, String scheme)
       ClientProtocol nnproxy, String scheme)
       throws IOException, URISyntaxException {
       throws IOException, URISyntaxException {
     final DatanodeID host = pickSrcDatanode(i, nnproxy);
     final DatanodeID host = pickSrcDatanode(i, nnproxy);
@@ -63,7 +58,7 @@ public class FileDataServlet extends DfsServlet {
    * Currently, this looks at no more than the first five blocks of a file,
    * Currently, this looks at no more than the first five blocks of a file,
    * selecting a datanode randomly from the most represented.
    * selecting a datanode randomly from the most represented.
    */
    */
-  private static DatanodeID pickSrcDatanode(DFSFileInfo i,
+  private static DatanodeID pickSrcDatanode(FileStatus i,
       ClientProtocol nnproxy) throws IOException {
       ClientProtocol nnproxy) throws IOException {
     final LocatedBlocks blks = nnproxy.getBlockLocations(
     final LocatedBlocks blks = nnproxy.getBlockLocations(
         i.getPath().toUri().getPath(), 0, 1);
         i.getPath().toUri().getPath(), 0, 1);
@@ -89,7 +84,7 @@ public class FileDataServlet extends DfsServlet {
     try {
     try {
       final String path = request.getPathInfo() != null
       final String path = request.getPathInfo() != null
         ? request.getPathInfo() : "/";
         ? request.getPathInfo() : "/";
-      DFSFileInfo info = nnproxy.getFileInfo(path);
+      FileStatus info = nnproxy.getFileInfo(path);
       if ((info != null) && !info.isDir()) {
       if ((info != null) && !info.isDir()) {
         response.sendRedirect(createUri(info, ugi, nnproxy,
         response.sendRedirect(createUri(info, ugi, nnproxy,
               request.getScheme()).toURL().toString());
               request.getScheme()).toURL().toString());

+ 6 - 6
src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java

@@ -17,8 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 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.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
@@ -58,7 +58,7 @@ public class ListPathsServlet extends DfsServlet {
    * Node information includes path, modification, permission, owner and group.
    * Node information includes path, modification, permission, owner and group.
    * For files, it also includes size, replication and block-size. 
    * 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.startTag(i.isDir() ? "directory" : "file");
     doc.attribute("path", i.getPath().toUri().getPath());
     doc.attribute("path", i.getPath().toUri().getPath());
     doc.attribute("modified", df.format(new Date(i.getModificationTime())));
     doc.attribute("modified", df.format(new Date(i.getModificationTime())));
@@ -140,7 +140,7 @@ public class ListPathsServlet extends DfsServlet {
         doc.attribute(m.getKey(), m.getValue());
         doc.attribute(m.getKey(), m.getValue());
       }
       }
 
 
-      DFSFileInfo base = nnproxy.getFileInfo(path);
+      FileStatus base = nnproxy.getFileInfo(path);
       if ((base != null) && base.isDir()) {
       if ((base != null) && base.isDir()) {
         writeInfo(base, doc);
         writeInfo(base, doc);
       }
       }
@@ -150,9 +150,9 @@ public class ListPathsServlet extends DfsServlet {
       while (!pathstack.empty()) {
       while (!pathstack.empty()) {
         String p = pathstack.pop();
         String p = pathstack.pop();
         try {
         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;
               continue;
             }
             }
             if (recur && i.isDir()) {
             if (recur && i.isDir()) {

+ 4 - 3
src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
 
 
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -478,8 +479,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
 
 
   /**
   /**
    */
    */
-  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) {
     if (files != null) {
       myMetrics.numGetListingOps.inc();
       myMetrics.numGetListingOps.inc();
     }
     }
@@ -493,7 +494,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         or null if file not found
    */
    */
-  public DFSFileInfo getFileInfo(String src)  throws IOException {
+  public FileStatus getFileInfo(String src)  throws IOException {
     return namesystem.getFileInfo(src);
     return namesystem.getFileInfo(src);
   }
   }
 
 

+ 5 - 5
src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -35,12 +35,12 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 
 
 /**
 /**
@@ -124,7 +124,7 @@ public class NamenodeFsck {
    */
    */
   public void fsck() throws IOException {
   public void fsck() throws IOException {
     try {
     try {
-      DFSFileInfo[] files = nn.namesystem.dir.getListing(path);
+      FileStatus[] files = nn.namesystem.dir.getListing(path);
       FsckResult res = new FsckResult();
       FsckResult res = new FsckResult();
       res.totalRacks = nn.getNetworkTopology().getNumOfRacks();
       res.totalRacks = nn.getNetworkTopology().getNumOfRacks();
       res.totalDatanodes = nn.namesystem.getNumberOfDatanodes(
       res.totalDatanodes = nn.namesystem.getNumberOfDatanodes(
@@ -151,13 +151,13 @@ public class NamenodeFsck {
     }
     }
   }
   }
   
   
-  private void check(DFSFileInfo file, FsckResult res) throws IOException {
+  private void check(FileStatus file, FsckResult res) throws IOException {
     int minReplication = nn.namesystem.getMinReplication();
     int minReplication = nn.namesystem.getMinReplication();
     String path = file.getPath().toString();
     String path = file.getPath().toString();
     boolean isOpen = false;
     boolean isOpen = false;
 
 
     if (file.isDir()) {
     if (file.isDir()) {
-      DFSFileInfo[] files = nn.namesystem.dir.getListing(path);
+      FileStatus[] files = nn.namesystem.dir.getListing(path);
       if (files == null) {
       if (files == null) {
         return;
         return;
       }
       }
@@ -301,7 +301,7 @@ public class NamenodeFsck {
     }
     }
   }
   }
   
   
-  private void lostFoundMove(DFSFileInfo file, LocatedBlocks blocks)
+  private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
     throws IOException {
     throws IOException {
     DFSClient dfs = new DFSClient(conf);
     DFSClient dfs = new DFSClient(conf);
     if (!lfInited) {
     if (!lfInited) {

+ 3 - 3
src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -28,8 +28,8 @@ import java.util.zip.CRC32;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 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;
 import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
 import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -130,10 +130,10 @@ public class TestDFSUpgradeFromImage extends TestCase {
   private void verifyDir(DFSClient client, String dir) 
   private void verifyDir(DFSClient client, String dir) 
                                            throws IOException {
                                            throws IOException {
     
     
-    DFSFileInfo[] fileArr = client.listPaths(dir);
+    FileStatus[] fileArr = client.listPaths(dir);
     TreeMap<String, Boolean> fileMap = new TreeMap<String, Boolean>();
     TreeMap<String, Boolean> fileMap = new TreeMap<String, Boolean>();
     
     
-    for(DFSFileInfo file : fileArr) {
+    for(FileStatus file : fileArr) {
       String path = file.getPath().toString();
       String path = file.getPath().toString();
       fileMap.put(path, Boolean.valueOf(file.isDir()));
       fileMap.put(path, Boolean.valueOf(file.isDir()));
     }
     }

+ 2 - 5
src/test/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -22,13 +22,10 @@ import java.io.*;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.
  * This class tests the FileStatus API.
@@ -83,7 +80,7 @@ public class TestFileStatus extends TestCase {
                  fs.getFileStatus(path).isDir() == true);
                  fs.getFileStatus(path).isDir() == true);
       
       
       // make sure getFileInfo returns null for files which do not exist
       // make sure getFileInfo returns null for files which do not exist
-      DFSFileInfo fileInfo = dfsClient.getFileInfo("/noSuchFile");
+      FileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
       assertTrue(fileInfo == null);
       assertTrue(fileInfo == null);
 
 
       // create a file in home directory
       // create a file in home directory

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

@@ -76,7 +76,7 @@
         return;
         return;
       }
       }
       // directory
       // directory
-      DFSFileInfo[] files = dfs.listPaths(target);
+      FileStatus[] files = dfs.listPaths(target);
       //generate a table and dump the info
       //generate a table and dump the info
       String [] headings = { "Name", "Type", "Size", "Replication", 
       String [] headings = { "Name", "Type", "Size", "Replication", 
                               "Block Size", "Modification Time",
                               "Block Size", "Modification Time",
@@ -120,7 +120,7 @@
           String datanodeUrl = req.getRequestURL()+"?dir="+
           String datanodeUrl = req.getRequestURL()+"?dir="+
               URLEncoder.encode(files[i].getPath().toString(), "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].getPath().getName()+"</a>";
           cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
           cols[6] = files[i].getPermission().toString();
           cols[6] = files[i].getPermission().toString();
           cols[7] = files[i].getOwner();
           cols[7] = files[i].getOwner();