Browse Source

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 năm trước cách đây
mục cha
commit
3d5c443a7d

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

@@ -15,6 +15,9 @@ Release 0.23.1 - Unreleased
 
   IMPROVEMENTS
 
+    HADOOP-8015. ChRootFileSystem should extend FilterFileSystem 
+    (Daryn Sharp via bobby)
+
     HADOOP-7801. HADOOP_PREFIX cannot be overriden. (Bruno Mahé via tomwhite)
 
     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;
   }
 
+  /**
+   * Get the raw file system 
+   * @return FileSystem being filtered
+   */
+  public FileSystem getRawFileSystem() {
+    return fs;
+  }
+
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
    *   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.net.URI;
 import java.net.URISyntaxException;
-import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 
 /**
@@ -60,15 +59,14 @@ import org.apache.hadoop.util.Progressable;
 
 @InterfaceAudience.Private
 @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 Path chRootPathPart; // the root below the root of the base
   private final String chRootPathPartString;
   private Path workingDir;
   
   protected FileSystem getMyFs() {
-    return myFs;
+    return getRawFileSystem();
   }
   
   /**
@@ -90,8 +88,8 @@ class ChRootedFileSystem extends FileSystem {
    */
   public ChRootedFileSystem(final FileSystem fs, final Path theRoot)
     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); 
                             // but not public
     chRootPathPart = new Path(theRoot.toUri().getPath());
@@ -111,8 +109,8 @@ class ChRootedFileSystem extends FileSystem {
      */
     // Handle the two cases:
     //              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));
 
     workingDir = getHomeDirectory();
@@ -127,7 +125,6 @@ class ChRootedFileSystem extends FileSystem {
    */
   public void initialize(final URI name, final Configuration conf)
       throws IOException {
-    myFs.initialize(name, conf);
     super.initialize(name, conf);
     setConf(conf);
   }
@@ -137,12 +134,6 @@ class ChRootedFileSystem extends FileSystem {
     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.
    * @param p - fully qualified path p
@@ -175,7 +166,7 @@ class ChRootedFileSystem extends FileSystem {
   
   public Path getResolvedQualifiedPath(final Path f)
       throws FileNotFoundException {
-    return myFs.makeQualified(
+    return makeQualified(
         new Path(chRootPathPartString + f.toUri().toString()));
   }
   
@@ -199,14 +190,14 @@ class ChRootedFileSystem extends FileSystem {
   public FSDataOutputStream create(final Path f, final FsPermission permission,
       final boolean overwrite, final int bufferSize, final short replication,
       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);
   }
 
   @Override
   public boolean delete(final Path f, final boolean recursive) 
       throws IOException {
-    return myFs.delete(fullPath(f), recursive);
+    return super.delete(fullPath(f), recursive);
   }
   
 
@@ -219,95 +210,90 @@ class ChRootedFileSystem extends FileSystem {
   @Override
   public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
       final long len) throws IOException {
-    return myFs.getFileBlockLocations(
+    return super.getFileBlockLocations(
         new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
   }
 
   @Override
   public FileChecksum getFileChecksum(final Path f) 
       throws IOException {
-    return myFs.getFileChecksum(fullPath(f));
+    return super.getFileChecksum(fullPath(f));
   }
 
   @Override
   public FileStatus getFileStatus(final Path f) 
       throws IOException {
-    return myFs.getFileStatus(fullPath(f));
+    return super.getFileStatus(fullPath(f));
   }
 
   @Override
   public FsStatus getStatus(Path p) throws IOException {
-    return myFs.getStatus(fullPath(p));
+    return super.getStatus(fullPath(p));
   }
 
   @Override
   public FsServerDefaults getServerDefaults() throws IOException {
-    return myFs.getServerDefaults();
+    return super.getServerDefaults();
   }
 
   @Override
   public FileStatus[] listStatus(final Path f) 
       throws IOException {
-    return myFs.listStatus(fullPath(f));
+    return super.listStatus(fullPath(f));
   }
   
   @Override
   public boolean mkdirs(final Path f, final FsPermission permission)
       throws IOException {
-    return myFs.mkdirs(fullPath(f), permission);
+    return super.mkdirs(fullPath(f), permission);
   }
 
   @Override
   public FSDataInputStream open(final Path f, final int bufferSize) 
     throws IOException {
-    return myFs.open(fullPath(f), bufferSize);
+    return super.open(fullPath(f), bufferSize);
   }
   
   @Override
   public FSDataOutputStream append(final Path f, final int bufferSize,
       final Progressable progress) throws IOException {
-    return myFs.append(fullPath(f), bufferSize, progress);
+    return super.append(fullPath(f), bufferSize, progress);
   }
 
   @Override
   public boolean rename(final Path src, final Path dst) throws IOException {
     // note fullPath will check that paths are relative to this FileSystem.
     // 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
   public void setOwner(final Path f, final String username,
       final String groupname)
     throws IOException {
-    myFs.setOwner(fullPath(f), username, groupname);
+    super.setOwner(fullPath(f), username, groupname);
   }
 
   @Override
   public void setPermission(final Path f, final FsPermission permission)
     throws IOException {
-    myFs.setPermission(fullPath(f), permission);
+    super.setPermission(fullPath(f), permission);
   }
 
   @Override
   public boolean setReplication(final Path f, final short replication)
     throws IOException {
-    return myFs.setReplication(fullPath(f), replication);
+    return super.setReplication(fullPath(f), replication);
   }
 
   @Override
   public void setTimes(final Path f, final long mtime, final long atime) 
       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
-  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));
   }
 }