浏览代码

svn merge -c 1239924 fixes HADOOP-8015

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1239925 13f79535-47bb-0310-9956-ffa450edef68
Robert Joseph Evans 13 年之前
父节点
当前提交
3d5c443a7d

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -15,6 +15,9 @@ Release 0.23.1 - Unreleased
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    HADOOP-8015. ChRootFileSystem should extend FilterFileSystem 
+    (Daryn Sharp via bobby)
+
     HADOOP-7801. HADOOP_PREFIX cannot be overriden. (Bruno Mahé via tomwhite)
     HADOOP-7801. HADOOP_PREFIX cannot be overriden. (Bruno Mahé via tomwhite)
 
 
     HADOOP-7802. Hadoop scripts unconditionally source
     HADOOP-7802. Hadoop scripts unconditionally source

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -63,6 +63,14 @@ public class FilterFileSystem extends FileSystem {
     this.statistics = fs.statistics;
     this.statistics = fs.statistics;
   }
   }
 
 
+  /**
+   * Get the raw file system 
+   * @return FileSystem being filtered
+   */
+  public FileSystem getRawFileSystem() {
+    return fs;
+  }
+
   /** Called after a new FileSystem instance is constructed.
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
    * @param name a uri whose authority section names the host, port, etc.
    *   for this FileSystem
    *   for this FileSystem

+ 26 - 40
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java

@@ -20,7 +20,6 @@ import java.io.FileNotFoundException;
 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.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -31,11 +30,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 
 
 /**
 /**
@@ -60,15 +59,14 @@ import org.apache.hadoop.util.Progressable;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
-class ChRootedFileSystem extends FileSystem {
-  private final FileSystem myFs; // the base file system whose root is changed
+class ChRootedFileSystem extends FilterFileSystem {
   private final URI myUri; // the base URI + the chRoot
   private final URI myUri; // the base URI + the chRoot
   private final Path chRootPathPart; // the root below the root of the base
   private final Path chRootPathPart; // the root below the root of the base
   private final String chRootPathPartString;
   private final String chRootPathPartString;
   private Path workingDir;
   private Path workingDir;
   
   
   protected FileSystem getMyFs() {
   protected FileSystem getMyFs() {
-    return myFs;
+    return getRawFileSystem();
   }
   }
   
   
   /**
   /**
@@ -90,8 +88,8 @@ class ChRootedFileSystem extends FileSystem {
    */
    */
   public ChRootedFileSystem(final FileSystem fs, final Path theRoot)
   public ChRootedFileSystem(final FileSystem fs, final Path theRoot)
     throws URISyntaxException {
     throws URISyntaxException {
-    myFs = fs;
-    myFs.makeQualified(theRoot); //check that root is a valid path for fs
+    super(fs);
+    makeQualified(theRoot); //check that root is a valid path for fs
                             // Would like to call myFs.checkPath(theRoot); 
                             // Would like to call myFs.checkPath(theRoot); 
                             // but not public
                             // but not public
     chRootPathPart = new Path(theRoot.toUri().getPath());
     chRootPathPart = new Path(theRoot.toUri().getPath());
@@ -111,8 +109,8 @@ class ChRootedFileSystem extends FileSystem {
      */
      */
     // Handle the two cases:
     // Handle the two cases:
     //              scheme:/// and scheme://authority/
     //              scheme:/// and scheme://authority/
-    myUri = new URI(myFs.getUri().toString() + 
-        (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
+    myUri = new URI(fs.getUri().toString() + 
+        (fs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
           chRootPathPart.toString().substring(1));
           chRootPathPart.toString().substring(1));
 
 
     workingDir = getHomeDirectory();
     workingDir = getHomeDirectory();
@@ -127,7 +125,6 @@ class ChRootedFileSystem extends FileSystem {
    */
    */
   public void initialize(final URI name, final Configuration conf)
   public void initialize(final URI name, final Configuration conf)
       throws IOException {
       throws IOException {
-    myFs.initialize(name, conf);
     super.initialize(name, conf);
     super.initialize(name, conf);
     setConf(conf);
     setConf(conf);
   }
   }
@@ -137,12 +134,6 @@ class ChRootedFileSystem extends FileSystem {
     return myUri;
     return myUri;
   }
   }
   
   
-  @Override
-  public Path makeQualified(final Path path) {
-    return myFs.makeQualified(path);
-    // NOT myFs.makeQualified(fullPath(path));
-  }
- 
   /**
   /**
    * Strip out the root from the path.
    * Strip out the root from the path.
    * @param p - fully qualified path p
    * @param p - fully qualified path p
@@ -175,7 +166,7 @@ class ChRootedFileSystem extends FileSystem {
   
   
   public Path getResolvedQualifiedPath(final Path f)
   public Path getResolvedQualifiedPath(final Path f)
       throws FileNotFoundException {
       throws FileNotFoundException {
-    return myFs.makeQualified(
+    return makeQualified(
         new Path(chRootPathPartString + f.toUri().toString()));
         new Path(chRootPathPartString + f.toUri().toString()));
   }
   }
   
   
@@ -199,14 +190,14 @@ class ChRootedFileSystem extends FileSystem {
   public FSDataOutputStream create(final Path f, final FsPermission permission,
   public FSDataOutputStream create(final Path f, final FsPermission permission,
       final boolean overwrite, final int bufferSize, final short replication,
       final boolean overwrite, final int bufferSize, final short replication,
       final long blockSize, final Progressable progress) throws IOException {
       final long blockSize, final Progressable progress) throws IOException {
-    return myFs.create(fullPath(f), permission, overwrite, bufferSize,
+    return super.create(fullPath(f), permission, overwrite, bufferSize,
         replication, blockSize, progress);
         replication, blockSize, progress);
   }
   }
 
 
   @Override
   @Override
   public boolean delete(final Path f, final boolean recursive) 
   public boolean delete(final Path f, final boolean recursive) 
       throws IOException {
       throws IOException {
-    return myFs.delete(fullPath(f), recursive);
+    return super.delete(fullPath(f), recursive);
   }
   }
   
   
 
 
@@ -219,95 +210,90 @@ class ChRootedFileSystem extends FileSystem {
   @Override
   @Override
   public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
   public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
       final long len) throws IOException {
       final long len) throws IOException {
-    return myFs.getFileBlockLocations(
+    return super.getFileBlockLocations(
         new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
         new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
   }
   }
 
 
   @Override
   @Override
   public FileChecksum getFileChecksum(final Path f) 
   public FileChecksum getFileChecksum(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.getFileChecksum(fullPath(f));
+    return super.getFileChecksum(fullPath(f));
   }
   }
 
 
   @Override
   @Override
   public FileStatus getFileStatus(final Path f) 
   public FileStatus getFileStatus(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.getFileStatus(fullPath(f));
+    return super.getFileStatus(fullPath(f));
   }
   }
 
 
   @Override
   @Override
   public FsStatus getStatus(Path p) throws IOException {
   public FsStatus getStatus(Path p) throws IOException {
-    return myFs.getStatus(fullPath(p));
+    return super.getStatus(fullPath(p));
   }
   }
 
 
   @Override
   @Override
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
-    return myFs.getServerDefaults();
+    return super.getServerDefaults();
   }
   }
 
 
   @Override
   @Override
   public FileStatus[] listStatus(final Path f) 
   public FileStatus[] listStatus(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.listStatus(fullPath(f));
+    return super.listStatus(fullPath(f));
   }
   }
   
   
   @Override
   @Override
   public boolean mkdirs(final Path f, final FsPermission permission)
   public boolean mkdirs(final Path f, final FsPermission permission)
       throws IOException {
       throws IOException {
-    return myFs.mkdirs(fullPath(f), permission);
+    return super.mkdirs(fullPath(f), permission);
   }
   }
 
 
   @Override
   @Override
   public FSDataInputStream open(final Path f, final int bufferSize) 
   public FSDataInputStream open(final Path f, final int bufferSize) 
     throws IOException {
     throws IOException {
-    return myFs.open(fullPath(f), bufferSize);
+    return super.open(fullPath(f), bufferSize);
   }
   }
   
   
   @Override
   @Override
   public FSDataOutputStream append(final Path f, final int bufferSize,
   public FSDataOutputStream append(final Path f, final int bufferSize,
       final Progressable progress) throws IOException {
       final Progressable progress) throws IOException {
-    return myFs.append(fullPath(f), bufferSize, progress);
+    return super.append(fullPath(f), bufferSize, progress);
   }
   }
 
 
   @Override
   @Override
   public boolean rename(final Path src, final Path dst) throws IOException {
   public boolean rename(final Path src, final Path dst) throws IOException {
     // note fullPath will check that paths are relative to this FileSystem.
     // note fullPath will check that paths are relative to this FileSystem.
     // Hence both are in same file system and a rename is valid
     // Hence both are in same file system and a rename is valid
-    return myFs.rename(fullPath(src), fullPath(dst)); 
+    return super.rename(fullPath(src), fullPath(dst)); 
   }
   }
   
   
   @Override
   @Override
   public void setOwner(final Path f, final String username,
   public void setOwner(final Path f, final String username,
       final String groupname)
       final String groupname)
     throws IOException {
     throws IOException {
-    myFs.setOwner(fullPath(f), username, groupname);
+    super.setOwner(fullPath(f), username, groupname);
   }
   }
 
 
   @Override
   @Override
   public void setPermission(final Path f, final FsPermission permission)
   public void setPermission(final Path f, final FsPermission permission)
     throws IOException {
     throws IOException {
-    myFs.setPermission(fullPath(f), permission);
+    super.setPermission(fullPath(f), permission);
   }
   }
 
 
   @Override
   @Override
   public boolean setReplication(final Path f, final short replication)
   public boolean setReplication(final Path f, final short replication)
     throws IOException {
     throws IOException {
-    return myFs.setReplication(fullPath(f), replication);
+    return super.setReplication(fullPath(f), replication);
   }
   }
 
 
   @Override
   @Override
   public void setTimes(final Path f, final long mtime, final long atime) 
   public void setTimes(final Path f, final long mtime, final long atime) 
       throws IOException {
       throws IOException {
-    myFs.setTimes(fullPath(f), mtime, atime);
-  }
-
-  @Override
-  public void setVerifyChecksum(final boolean verifyChecksum)  {
-    myFs.setVerifyChecksum(verifyChecksum);
+    super.setTimes(fullPath(f), mtime, atime);
   }
   }
   
   
   @Override
   @Override
-  public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
-    return myFs.getDelegationTokens(renewer);
+  public Path resolvePath(final Path p) throws IOException {
+    return super.resolvePath(fullPath(p));
   }
   }
 }
 }