瀏覽代碼

commit 6ccd8b0a2322d5f46ea61f9e4da077ccfa3031bf
Author: Sanjay Radia <sradia@yahoo-inc.com>
Date: Wed Dec 8 12:18:00 2010 -0800

Added ViewFs (Client side mount table)

+++ b/YAHOO-CHANGES.txt
+ Port and extend viewfs (client-side mountables) from Fred
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/yahoo-merge@1079150 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年之前
父節點
當前提交
6eb7b7986e
共有 28 個文件被更改,包括 4879 次插入24 次删除
  1. 17 1
      src/java/core-default.xml
  2. 9 1
      src/java/org/apache/hadoop/fs/FileContext.java
  3. 12 0
      src/java/org/apache/hadoop/fs/FileSystem.java
  4. 4 1
      src/java/org/apache/hadoop/fs/FilterFileSystem.java
  5. 18 19
      src/java/org/apache/hadoop/fs/FsShell.java
  6. 20 1
      src/java/org/apache/hadoop/fs/Trash.java
  7. 303 0
      src/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
  8. 287 0
      src/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java
  9. 70 0
      src/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
  10. 63 0
      src/java/org/apache/hadoop/fs/viewfs/Constants.java
  11. 421 0
      src/java/org/apache/hadoop/fs/viewfs/InodeTree.java
  12. 626 0
      src/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
  13. 816 0
      src/java/org/apache/hadoop/fs/viewfs/ViewFs.java
  14. 98 0
      src/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
  15. 0 1
      src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
  16. 291 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java
  17. 292 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestChRootedFs.java
  18. 60 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestFSMainOperationsLocalFileSystem.java
  19. 42 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestFcCreateMkdirLocalFs.java
  20. 87 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestFcMainOperationsLocalFs.java
  21. 41 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestFcPermissionsLocalFs.java
  22. 45 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java
  23. 57 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
  24. 43 0
      src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsLocalFs.java
  25. 479 0
      src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
  26. 78 0
      src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
  27. 528 0
      src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
  28. 72 0
      src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java

+ 17 - 1
src/java/core-default.xml

@@ -245,6 +245,14 @@
   <description>The FileSystem for hdfs: uris.</description>
 </property>
 
+
+<property>
+  <name>fs.viewfs.impl</name>
+  <value>org.apache.hadoop.fs.viewfs.ViewFileSystem</value>
+  <description>The FileSystem for view file system for viewfs: uris
+  (ie client side mount table:).</description>
+</property>
+
 <property>
   <name>fs.AbstractFileSystem.file.impl</name>
   <value>org.apache.hadoop.fs.local.LocalFs</value>
@@ -255,7 +263,15 @@
 <property>
   <name>fs.AbstractFileSystem.hdfs.impl</name>
   <value>org.apache.hadoop.fs.Hdfs</value>
-  <description>The FileSystem for hdfs: uris.</description>
+  <description>The AbstractFileSystem for hdfs: uris.</description>
+</property>
+
+
+<property>
+  <name>fs.AbstractFileSystem.viewfs.impl</name>
+  <value>org.apache.hadoop.fs.viewfs.ViewFs</value>
+  <description>The AbstractFileSystem for view file system for viewfs: uris
+  (ie client side mount table:).</description>
 </property>
 
 <property>

+ 9 - 1
src/java/org/apache/hadoop/fs/FileContext.java

@@ -511,7 +511,15 @@ public final class FileContext {
   }
   
   /**
-   * 
+   * Return the current user's home directory in this file system.
+   * The default implementation returns "/user/$USER/".
+   * @return the home directory
+   */
+  public Path getHomeDirectory() {
+    return defaultFS.getHomeDirectory();
+  }
+  
+  /**
    * @return the umask of this FileContext
    */
   public FsPermission getUMask() {

+ 12 - 0
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -518,6 +518,18 @@ public abstract class FileSystem extends Configured implements Closeable {
         conf.getInt("io.file.buffer.size", 4096));
   }
 
+  /**
+   * Return the fully-qualified path of path f resolving the path
+   * through any symlinks or mount point
+   * @param p path to be resolved
+   * @return fully qualified path 
+   * @throws FileNotFoundException
+   */
+   public Path resolvePath(final Path p) throws FileNotFoundException {
+     checkPath(p);
+     return p; // default is to return the path
+   }
+
   /**
    * Opens an FSDataInputStream at the indicated Path.
    * @param f the file name to open

+ 4 - 1
src/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.net.URI;
 import java.util.EnumSet;
-import java.util.Iterator;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -94,6 +93,10 @@ public class FilterFileSystem extends FileSystem {
       return fs.getFileBlockLocations(file, start, len);
   }
   
+  @Override
+  public Path resolvePath(final Path p) throws FileNotFoundException {
+    return fs.resolvePath(p);
+  }
   /**
    * Opens an FSDataInputStream at the indicated Path.
    * @param f the file name to open

+ 18 - 19
src/java/org/apache/hadoop/fs/FsShell.java

@@ -154,9 +154,9 @@ public class FsShell extends Configured implements Tool {
    * and copy them to the local name. srcf is kept.
    * When copying multiple files, the destination must be a directory. 
    * Otherwise, IOException is thrown.
-   * @param argv: arguments
-   * @param pos: Ignore everything before argv[pos]  
-   * @exception: IOException  
+   * @param argv arguments
+   * @param pos Ignore everything before argv[pos]  
+   * @throws IOException on error
    * @see org.apache.hadoop.fs.FileSystem.globStatus 
    */
   void copyToLocal(String[]argv, int pos) throws IOException {
@@ -289,9 +289,9 @@ public class FsShell extends Configured implements Tool {
    * Get all the files in the directories that match the source file 
    * pattern and merge and sort them to only one file on local fs 
    * srcf is kept.
-   * @param srcf: a file pattern specifying source files
-   * @param dstf: a destination local file/directory 
-   * @exception: IOException  
+   * @param srcf a file pattern specifying source files
+   * @param dstf a destination local file/directory 
+   * @exception IOException  
    * @see org.apache.hadoop.fs.FileSystem.globStatus 
    */
   void copyMergeToLocal(String srcf, Path dst) throws IOException {
@@ -306,10 +306,10 @@ public class FsShell extends Configured implements Tool {
    * 
    * Also adds a string between the files (useful for adding \n
    * to a text file)
-   * @param srcf: a file pattern specifying source files
-   * @param dstf: a destination local file/directory
-   * @param endline: if an end of line character is added to a text file 
-   * @exception: IOException  
+   * @param srcf a file pattern specifying source files
+   * @param dstf a destination local file/directory
+   * @param endline if an end of line character is added to a text file 
+   * @throws IOException on error
    * @see org.apache.hadoop.fs.FileSystem.globStatus 
    */
   void copyMergeToLocal(String srcf, Path dst, boolean endline) throws IOException {
@@ -339,8 +339,8 @@ public class FsShell extends Configured implements Tool {
   /**
    * Fetch all files that match the file pattern <i>srcf</i> and display
    * their content on stdout. 
-   * @param srcf: a file pattern specifying source files
-   * @exception: IOException
+   * @param srcf a file pattern specifying source files
+   * @throws IOException on error
    * @see org.apache.hadoop.fs.FileSystem.globStatus 
    */
   void cat(String src, boolean verifyChecksum) throws IOException {
@@ -572,9 +572,9 @@ public class FsShell extends Configured implements Tool {
   /**
    * Actually set the replication for this file
    * If it fails either throw IOException or print an error msg
-   * @param file: a file/directory
-   * @param newRep: new replication factor
-   * @throws IOException
+   * @param file a file/directory
+   * @param newRep new replication factor
+   * @throws IOException on error
    */
   private void setFileReplication(Path file, FileSystem srcFs, short newRep, List<Path> waitList)
     throws IOException {
@@ -970,7 +970,7 @@ public class FsShell extends Configured implements Tool {
    * the argvp[] array.
    * If multiple source files are specified, then the destination 
    * must be a directory. Otherwise, IOException is thrown.
-   * @exception: IOException  
+   * @throws IOException on error
    */
   private int rename(String argv[], Configuration conf) throws IOException {
     int i = 0;
@@ -1055,7 +1055,7 @@ public class FsShell extends Configured implements Tool {
    * the argvp[] array.
    * If multiple source files are specified, then the destination 
    * must be a directory. Otherwise, IOException is thrown.
-   * @exception: IOException  
+   * @throws IOException on error
    */
   private int copy(String argv[], Configuration conf) throws IOException {
     int i = 0;
@@ -1153,8 +1153,7 @@ public class FsShell extends Configured implements Tool {
     
     if(!skipTrash) {
       try {
-	      Trash trashTmp = new Trash(srcFs, getConf());
-        if (trashTmp.moveToTrash(src)) {
+        if (Trash.moveToAppropriateTrash(srcFs, src, getConf())) {
           System.out.println("Moved to trash: " + src);
           return;
         }

+ 20 - 1
src/java/org/apache/hadoop/fs/Trash.java

@@ -86,7 +86,26 @@ public class Trash extends Configured {
                                          FS_TRASH_INTERVAL_DEFAULT) *
                                 MSECS_PER_MINUTE);
   }
-
+  
+  /**
+   * In case of the symlinks or mount points, one has to move the appropriate
+   * trashbin in the actual volume of the path p being deleted.
+   * 
+   * Hence we get the file system of the fully-qualified resolved-path and
+   * then move the path p to the trashbin in that volume,
+   * @param fs - the filesystem of path p
+   * @param p - the  path being deleted - to be moved to trasg
+   * @param conf - configuration
+   * @return false if the item is already in the trash or trash is disabled
+   * @throws IOException on error
+   */
+  public static boolean moveToAppropriateTrash(FileSystem fs, Path p,
+      Configuration conf) throws IOException {
+    Path fullyResolvedPath = fs.resolvePath(p);
+    Trash trash = new Trash(FileSystem.get(fullyResolvedPath.toUri(), conf), conf);
+    return trash.moveToTrash(fullyResolvedPath);
+  }
+  
   private Trash(Path home, Configuration conf) throws IOException {
     super(conf);
     this.fs = home.getFileSystem(conf);

+ 303 - 0
src/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java

@@ -0,0 +1,303 @@
+/**
+ * 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.viewfs;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+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.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * <code>ChRootedFileSystem</code> is a file system with its root some path
+ * below the root of its base file system. 
+ * 
+ * Example: For a base file system hdfs://nn1/ with chRoot at /usr/foo, the
+ * members will be setup as shown below.
+ * <ul>
+ * <li>myFs is the base file system and points to hdfs at nn1</li>
+ * <li>myURI is hdfs://nn1/user/foo</li>
+ * <li>chRootPathPart is /user/foo</li>
+ * <li>workingDir is a directory related to chRoot</li>
+ * </ul>
+ * 
+ * The paths are resolved as follows by ChRootedFileSystem:
+ * <ul>
+ * <li> Absolute path /a/b/c is resolved to /user/foo/a/b/c at myFs</li>
+ * <li> Relative path x/y is resolved to /user/foo/<workingDir>/x/y</li>
+ * </ul>
+ */
+
+@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
+  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;
+  }
+  
+  /**
+   * @param path
+   * @return  full path including the chroot 
+   */
+  protected Path fullPath(final Path path) {
+    super.checkPath(path);
+    return path.isAbsolute() ? 
+        new Path(chRootPathPartString + path.toUri().getPath()) :
+        new Path(chRootPathPartString + workingDir.toUri().getPath(), path);
+  }
+  
+  /**
+   * Constructor
+   * @param fs base file system
+   * @param theRoot chRoot for this file system
+   * @throws URISyntaxException
+   */
+  public ChRootedFileSystem(final FileSystem fs, final Path theRoot)
+    throws URISyntaxException {
+    myFs = fs;
+    myFs.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());
+    chRootPathPartString = chRootPathPart.toUri().getPath();
+    try {
+      initialize(fs.getUri(), fs.getConf());
+    } catch (IOException e) { // This exception should not be thrown
+      throw new RuntimeException("This should not occur");
+    }
+    
+    /*
+     * We are making URI include the chrootedPath: e.g. file:///chrootedPath.
+     * This is questionable since Path#makeQualified(uri, path) ignores
+     * the pathPart of a uri. Since this class is internal we can ignore
+     * this issue but if we were to make it external then this needs
+     * to be resolved.
+     */
+    myUri = new URI(myFs.getUri().toString() //+ Path.SEPARATOR
+        + chRootPathPart.toString().substring(1));
+
+    workingDir = getHomeDirectory();
+    // We don't use the wd of the myFs bu set it to root.
+    myFs.setWorkingDirectory(chRootPathPart);
+  }
+  
+  /** 
+   * Called after a new FileSystem instance is constructed.
+   * @param name a uri whose authority section names the host, port, etc.
+   *   for this FileSystem
+   * @param conf the configuration
+   */
+  public void initialize(final URI name, final Configuration conf)
+      throws IOException {
+    myFs.initialize(name, conf);
+    super.initialize(name, conf);
+    setConf(conf);
+  }
+
+  @Override
+  public URI getUri() {
+    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
+   * @return -  the remaining path  without the begining /
+   * @throws IOException if the p is not prefixed with root
+   */
+  String stripOutRoot(final Path p) throws IOException {
+    try {
+     checkPath(p);
+    } catch (IllegalArgumentException e) {
+      throw new IOException("Internal Error - path " + p +
+          " should have been with URI: " + myUri);
+    }
+    String pathPart = p.toUri().getPath();
+    return (pathPart.length() == chRootPathPartString.length()) ? "" : pathPart
+        .substring(chRootPathPartString.length() + 1);   
+  }
+  
+  @Override
+  protected Path getInitialWorkingDirectory() {
+    /*
+     * 3 choices here: 
+     *     null or / or /user/<uname> or strip out the root out of myFs's
+     *  inital wd. 
+     * Only reasonable choice for initialWd for chrooted fds is null 
+     * so that the default rule for wd is applied
+     */
+    return null;
+  }
+  
+  public Path getResolvedQualifiedPath(final Path f)
+      throws FileNotFoundException {
+    return myFs.makeQualified(
+        new Path(chRootPathPartString + f.toUri().toString()));
+  }
+  
+  @Override
+  public Path getHomeDirectory() {
+    return  new Path("/user/"+System.getProperty("user.name")).makeQualified(
+          getUri(), null);
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+  
+  @Override
+  public void setWorkingDirectory(final Path new_dir) {
+    workingDir = new_dir.isAbsolute() ? new_dir : new Path(workingDir, new_dir);
+  }
+
+  @Override
+  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,
+        replication, blockSize, progress);
+  }
+
+  @Override
+  public boolean delete(final Path f, final boolean recursive) 
+      throws IOException {
+    return myFs.delete(fullPath(f), recursive);
+  }
+  
+
+  @Override
+  public boolean delete(Path f) throws IOException {
+   return delete(f, true);
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
+      final long len) throws IOException {
+    return myFs.getFileBlockLocations(
+        new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(final Path f) 
+      throws IOException {
+    return myFs.getFileChecksum(fullPath(f));
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) 
+      throws IOException {
+    return myFs.getFileStatus(fullPath(f));
+  }
+
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    return myFs.getStatus(fullPath(p));
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return myFs.getServerDefaults();
+  }
+
+  @Override
+  public FileStatus[] listStatus(final Path f) 
+      throws IOException {
+    return myFs.listStatus(fullPath(f));
+  }
+  
+  @Override
+  public boolean mkdirs(final Path f, final FsPermission permission)
+      throws IOException {
+    return myFs.mkdirs(fullPath(f), permission);
+  }
+
+  @Override
+  public FSDataInputStream open(final Path f, final int bufferSize) 
+    throws IOException {
+    return myFs.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);
+  }
+
+  @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)); 
+  }
+  
+  @Override
+  public void setOwner(final Path f, final String username,
+      final String groupname)
+    throws IOException {
+    myFs.setOwner(fullPath(f), username, groupname);
+  }
+
+  @Override
+  public void setPermission(final Path f, final FsPermission permission)
+    throws IOException {
+    myFs.setPermission(fullPath(f), permission);
+  }
+
+  @Override
+  public boolean setReplication(final Path f, final short replication)
+    throws IOException {
+    return myFs.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);
+  }
+}

+ 287 - 0
src/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java

@@ -0,0 +1,287 @@
+/**
+ * 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.viewfs;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * <code>ChrootedFs</code> is a file system with its root some path
+ * below the root of its base file system.
+ * Example: For a base file system hdfs://nn1/ with chRoot at /usr/foo, the
+ * members will be setup as shown below.
+ * <ul>
+ * <li>myFs is the base file system and points to hdfs at nn1</li>
+ * <li>myURI is hdfs://nn1/user/foo</li>
+ * <li>chRootPathPart is /user/foo</li>
+ * <li>workingDir is a directory related to chRoot</li>
+ * </ul>
+ * 
+ * The paths are resolved as follows by ChRootedFileSystem:
+ * <ul>
+ * <li> Absolute path /a/b/c is resolved to /user/foo/a/b/c at myFs</li>
+ * <li> Relative path x/y is resolved to /user/foo/<workingDir>/x/y</li>
+ * </ul>
+
+ * 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
+class ChRootedFs extends AbstractFileSystem {
+  private final AbstractFileSystem myFs;  // the base file system whose root is changed
+  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;
+  
+  protected AbstractFileSystem getMyFs() {
+    return myFs;
+  }
+  
+  /**
+   * 
+   * @param path
+   * @return return full path including the chroot
+   */
+  protected Path fullPath(final Path path) {
+    super.checkPath(path);
+    return new Path(chRootPathPartString + path.toUri().getPath());
+  }
+  
+  public ChRootedFs(final AbstractFileSystem fs, final Path theRoot)
+    throws URISyntaxException {
+    super(fs.getUri(), fs.getUri().getScheme(),
+        fs.getUri().getAuthority() != null, fs.getUriDefaultPort());
+    myFs = fs;
+    myFs.checkPath(theRoot);
+    chRootPathPart = new Path(myFs.getUriPath(theRoot));
+    chRootPathPartString = chRootPathPart.toUri().getPath();
+    /*
+     * We are making URI include the chrootedPath: e.g. file:///chrootedPath.
+     * This is questionable since Path#makeQualified(uri, path) ignores
+     * the pathPart of a uri. Since this class is internal we can ignore
+     * this issue but if we were to make it external then this needs
+     * to be resolved.
+     */
+    myUri = new URI(myFs.getUri().toString() + chRootPathPart.toString().substring(1));
+    super.checkPath(theRoot);
+  }
+  
+  @Override
+  public URI getUri() {
+    return myUri;
+  }
+
+  
+  /**
+   *  
+   * Strip out the root from the path.
+   * 
+   * @param p - fully qualified path p
+   * @return -  the remaining path  without the begining /
+   */
+  public String stripOutRoot(final Path p) {
+    try {
+     checkPath(p);
+    } catch (IllegalArgumentException e) {
+      throw new RuntimeException("Internal Error - path " + p +
+          " should have been with URI" + myUri);
+    }
+    String pathPart = p.toUri().getPath();
+    return  (pathPart.length() == chRootPathPartString.length()) ?
+        "" : pathPart.substring(chRootPathPartString.length() + 1);   
+  }
+  
+
+  @Override
+  public Path getHomeDirectory() {
+    return myFs.getHomeDirectory();
+  }
+  
+  @Override
+  public Path getInitialWorkingDirectory() {
+    /*
+     * 3 choices here: return null or / or strip out the root out of myFs's
+     *  inital wd. 
+     * Only reasonable choice for initialWd for chrooted fds is null 
+     */
+    return null;
+  }
+  
+  @Override
+  public FSDataOutputStream createInternal(final Path f,
+      final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress, final int bytesPerChecksum,
+      final boolean createParent) throws IOException, UnresolvedLinkException {
+    return myFs.createInternal(fullPath(f), flag,
+        absolutePermission, bufferSize,
+        replication, blockSize, progress, bytesPerChecksum, createParent);
+  }
+
+  @Override
+  public boolean delete(final Path f, final boolean recursive) 
+      throws IOException, UnresolvedLinkException {
+    return myFs.delete(fullPath(f), recursive);
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final Path f, final long start,
+      final long len) throws IOException, UnresolvedLinkException {
+    return myFs.getFileBlockLocations(fullPath(f), start, len);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(final Path f) 
+      throws IOException, UnresolvedLinkException {
+    return myFs.getFileChecksum(fullPath(f));
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) 
+      throws IOException, UnresolvedLinkException {
+    return myFs.getFileStatus(fullPath(f));
+  }
+
+  @Override
+  public FileStatus getFileLinkStatus(final Path f) 
+    throws IOException, UnresolvedLinkException {
+    return myFs.getFileLinkStatus(fullPath(f));
+  }
+  
+  @Override
+  public FsStatus getFsStatus() throws IOException {
+    return myFs.getFsStatus();
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return myFs.getServerDefaults();
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return myFs.getUriDefaultPort();
+  }
+
+  @Override
+  public FileStatus[] listStatus(final Path f) 
+      throws IOException, UnresolvedLinkException {
+    return myFs.listStatus(fullPath(f));
+  }
+
+  @Override
+  public void mkdir(final Path dir, final FsPermission permission,
+      final boolean createParent) throws IOException, UnresolvedLinkException {
+    myFs.mkdir(fullPath(dir), permission, createParent);
+    
+  }
+
+  @Override
+  public FSDataInputStream open(final Path f, final int bufferSize) 
+    throws IOException, UnresolvedLinkException {
+    return myFs.open(fullPath(f), bufferSize);
+  }
+
+  @Override
+  public void renameInternal(final Path src, final Path dst)
+    throws IOException, UnresolvedLinkException {
+    // note fullPath will check that paths are relative to this FileSystem.
+    // Hence both are in same file system and a rename is valid
+    myFs.renameInternal(fullPath(src), fullPath(dst));
+  }
+  
+  @Override
+  public void renameInternal(final Path src, final Path dst, 
+      final boolean overwrite)
+    throws IOException, UnresolvedLinkException {
+    // note fullPath will check that paths are relative to this FileSystem.
+    // Hence both are in same file system and a rename is valid
+    myFs.renameInternal(fullPath(src), fullPath(dst), overwrite);
+  }
+
+  @Override
+  public void setOwner(final Path f, final String username,
+      final String groupname)
+    throws IOException, UnresolvedLinkException {
+    myFs.setOwner(fullPath(f), username, groupname);
+    
+  }
+
+  @Override
+  public void setPermission(final Path f, final FsPermission permission)
+    throws IOException, UnresolvedLinkException {
+    myFs.setPermission(fullPath(f), permission);
+  }
+
+  @Override
+  public boolean setReplication(final Path f, final short replication)
+    throws IOException, UnresolvedLinkException {
+    return myFs.setReplication(fullPath(f), replication);
+  }
+
+  @Override
+  public void setTimes(final Path f, final long mtime, final long atime) 
+      throws IOException, UnresolvedLinkException {
+    myFs.setTimes(fullPath(f), mtime, atime);
+  }
+
+  @Override
+  public void setVerifyChecksum(final boolean verifyChecksum) 
+      throws IOException, UnresolvedLinkException {
+    myFs.setVerifyChecksum(verifyChecksum);
+  }
+
+  @Override
+  public boolean supportsSymlinks() {
+    return myFs.supportsSymlinks();
+  }
+
+  @Override
+  public void createSymlink(final Path target, final Path link,
+      final boolean createParent) throws IOException, UnresolvedLinkException {
+    /*
+     * We leave the link alone:
+     * If qualified or link relative then of course it is okay.
+     * If absolute (ie / relative) then the link has to be resolved
+     * relative to the changed root.
+     */
+    myFs.createSymlink(fullPath(target), link, createParent);
+  }
+
+  @Override
+  public Path getLinkTarget(final Path f) throws IOException {
+    return myFs.getLinkTarget(fullPath(f));
+  }
+}

+ 70 - 0
src/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java

@@ -0,0 +1,70 @@
+/**
+ * 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.viewfs;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Utilities for config variables of the viewFs See {@link ViewFs}
+ */
+public class ConfigUtil {
+  /**
+   * Get the config variable prefix for the specified mount table
+   * @param mountTableName - the name of the mount table
+   * @return the config variable prefix for the specified mount table
+   */
+  public static String getConfigViewFsPrefix(final String mountTableName) {
+    return Constants.CONFIG_VIEWFS_PREFIX + "." + mountTableName;
+  }
+  
+  /**
+   * Get the config variable prefix for the default mount table
+   * @return the config variable prefix for the default mount table
+   */
+  public static String getConfigViewFsPrefix() {
+    return 
+      getConfigViewFsPrefix(Constants.CONFIG_VIEWFS_PREFIX_DEFAULT_MOUNT_TABLE);
+  }
+  
+  /**
+   * Add a link to the config for the specified mount table
+   * @param conf - add the link to this conf
+   * @param mountTableName
+   * @param src - the src path name
+   * @param target - the target URI link
+   */
+  public static void addLink(Configuration conf, final String mountTableName, 
+      final String src, final URI target) {
+    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
+        Constants.CONFIG_VIEWFS_LINK + "." + src, target.toString());  
+  }
+  
+  /**
+   * Add a link to the config for the default mount table
+   * @param conf - add the link to this conf
+   * @param src - the src path name
+   * @param target - the target URI link
+   */
+  public static void addLink(final Configuration conf, final String src,
+      final URI target) {
+    addLink( conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, 
+        src, target);   
+  }
+}

+ 63 - 0
src/java/org/apache/hadoop/fs/viewfs/Constants.java

@@ -0,0 +1,63 @@
+/**
+ * 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.viewfs;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Config variable prefixes for ViewFs -
+ *     see {@link org.apache.hadoop.fs.viewfs.ViewFs} for examples.
+ * The mount table is specified in the config using these prefixes.
+ * See {@link org.apache.hadoop.fs.viewfs.ConfigUtil} for convenience lib.
+ */
+public interface Constants {
+  /**
+   * Prefix for the config variable prefix for the ViewFs mount-table
+   */
+  public static final String CONFIG_VIEWFS_PREFIX = "fs.viewfs.mounttable";
+  
+  /**
+   * Config variable name for the default mount table.
+   */
+  public static final String CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE = "default";
+  
+  /**
+   * Config variable full prefix for the default mount table.
+   */
+  public static final String CONFIG_VIEWFS_PREFIX_DEFAULT_MOUNT_TABLE = 
+          CONFIG_VIEWFS_PREFIX + "." + CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE;
+  
+  /**
+   * Config variable for specifying a simple link
+   */
+  public static final String CONFIG_VIEWFS_LINK = "link";
+  
+  /**
+   * Config variable for specifying a merge link
+   */
+  public static final String CONFIG_VIEWFS_LINK_MERGE = "linkMerge";
+  
+  /**
+   * Config variable for specifying a merge of the root of the mount-table
+   *  with the root of another file system. 
+   */
+  public static final String CONFIG_VIEWFS_LINK_MERGE_SLASH = "linkMergeSlash";
+
+  static public final FsPermission PERMISSION_RRR = 
+    new FsPermission((short) 0444);
+}

+ 421 - 0
src/java/org/apache/hadoop/fs/viewfs/InodeTree.java

@@ -0,0 +1,421 @@
+/**
+ * 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.viewfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+
+
+/**
+ * InodeTree implements a mount-table as a tree of inodes.
+ * It is used to implement ViewFs and ViewFileSystem.
+ * In order to use it the caller must subclass it and implement
+ * the abstract methods {@link #getTargetFileSystem(INodeDir)}, etc.
+ * 
+ * The mountable is initialized from the config variables as 
+ * specified in {@link ViewFs}
+ *
+ * @param <T> is AbstractFileSystem or FileSystem
+ * 
+ * The three main methods are
+ * {@link #InodeTreel(Configuration)} // constructor
+ * {@link #InodeTree(Configuration, String)} // constructor
+ * {@link #resolve(String, boolean)} 
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable 
+abstract class InodeTree<T> {
+  static enum ResultKind {isInternalDir, isExternalDir;};
+  static final Path SlashPath = new Path("/");
+  
+  final INodeDir<T> root; // the root of the mount table
+  
+  /**
+   * Breaks file path into component names.
+   * @param path
+   * @return array of names component names
+   */
+  static String[] breakIntoPathComponents(final String path) {
+    return path == null ? null : path.split(Path.SEPARATOR);
+  } 
+  
+  /**
+   * Internal class for inode tree
+   * @param <T>
+   */
+  abstract static class INode<T> {
+    final String fullPath; // the full path to the root
+    public INode(String pathToNode, UserGroupInformation aUgi) {
+      fullPath = pathToNode;
+    }
+  };
+
+  /**
+   * Internal class to represent an internal dir of the mount table
+   * @param <T>
+   */
+  static class INodeDir<T> extends INode<T> {
+    final Map<String,INode<T>> children = new HashMap<String,INode<T>>();
+    T InodeDirFs =  null; // file system of this internal directory of mountT
+    boolean isRoot = false;
+    
+    INodeDir(final String pathToNode, final UserGroupInformation aUgi) {
+      super(pathToNode, aUgi);
+    }
+
+    INode<T> resolve(final String pathComponent) throws FileNotFoundException {
+      final INode<T> result = resolveInternal(pathComponent);
+      if (result == null) {
+        throw new FileNotFoundException();
+      }
+      return result;
+    }
+    
+    INode<T> resolveInternal(final String pathComponent)
+        throws FileNotFoundException {
+      return children.get(pathComponent);
+    }
+    
+    INodeDir<T> addDir(final String pathComponent,
+        final UserGroupInformation aUgi)
+      throws FileAlreadyExistsException {
+      if (children.containsKey(pathComponent)) {
+        throw new FileAlreadyExistsException();
+      }
+      final INodeDir<T> newDir = new INodeDir<T>(fullPath+ (isRoot ? "" : "/") + 
+          pathComponent, aUgi);
+      children.put(pathComponent, newDir);
+      return newDir;
+    }
+    
+    void addLink(final String pathComponent, final INodeLink<T> link)
+      throws FileAlreadyExistsException {
+      if (children.containsKey(pathComponent)) {
+        throw new FileAlreadyExistsException();
+      }
+      children.put(pathComponent, link);
+    }
+  }
+
+  /**
+   * In internal class to represent a mount link
+   * A mount link can be single dir link or a merge dir link.
+
+   * A merge dir link is  a merge (junction) of links to dirs:
+   * example : <merge of 2 dirs
+   *     /users -> hdfs:nn1//users
+   *     /users -> hdfs:nn2//users
+   * 
+   * For a merge, each target is checked to be dir when created but if target
+   * is changed later it is then ignored (a dir with null entries)
+   */
+  static class INodeLink<T> extends INode<T> {
+    final boolean isMergeLink; // true if MergeLink
+    final URI[] targetDirLinkList;
+    final T targetFileSystem;   // file system object created from the link.
+    
+    /**
+     * Construct a mergeLink
+     */
+    INodeLink(final String pathToNode, final UserGroupInformation aUgi,
+        final T targetMergeFs, final URI[] aTargetDirLinkList) {
+      super(pathToNode, aUgi);
+      targetFileSystem = targetMergeFs;
+      targetDirLinkList = aTargetDirLinkList;
+      isMergeLink = true;
+    }
+    
+    /**
+     * Construct a simple link (i.e. not a mergeLink)
+     */
+    INodeLink(final String pathToNode, final UserGroupInformation aUgi,
+        final T targetFs, final URI aTargetDirLink) {
+      super(pathToNode, aUgi);
+      targetFileSystem = targetFs;
+      targetDirLinkList = new URI[1];
+      targetDirLinkList[0] = aTargetDirLink;
+      isMergeLink = false;
+    }
+    
+    /**
+     * Get the target of the link
+     * If a merge link then it returned as "," separated URI list.
+     */
+    Path getTargetLink() {
+      // is merge link - use "," as separator between the merged URIs
+      String result = targetDirLinkList[0].toString();
+      for (int i=1; i < targetDirLinkList.length; ++i) {
+        result += "," + targetDirLinkList[0].toString();  
+      }
+      return new Path(result);
+    }
+  }
+
+  private void createLink(final String src, final String target,
+      final boolean isLinkMerge, final UserGroupInformation aUgi)
+      throws URISyntaxException, IOException,
+    FileAlreadyExistsException, UnsupportedFileSystemException {
+    // Validate that src is valid absolute path
+    final Path srcPath = new Path(src); 
+    if (!srcPath.isAbsoluteAndSchemeAuthorityNull()) {
+      throw new IOException("ViewFs:Non absolute mount name in config:" + src);
+    }
+ 
+    final String[] srcPaths = breakIntoPathComponents(src);
+    INodeDir<T> curInode = root;
+    int i;
+    // Ignore first initial slash, process all except last component
+    for (i = 1; i < srcPaths.length-1; i++) {
+      final String iPath = srcPaths[i];
+      INode<T> nextInode = curInode.resolveInternal(iPath);
+      if (nextInode == null) {
+        INodeDir<T> newDir = curInode.addDir(iPath, aUgi);
+        newDir.InodeDirFs = getTargetFileSystem(newDir);
+        nextInode = newDir;
+      }
+      if (nextInode instanceof INodeLink) {
+        // Error - expected a dir but got a link
+        throw new FileAlreadyExistsException("Path " + nextInode.fullPath +
+            " already exists as link");
+      } else {
+        assert(nextInode instanceof INodeDir);
+        curInode = (INodeDir<T>) nextInode;
+      }
+    }
+    
+    // Now process the last component
+    // Add the link in 2 cases: does not exist or a link exists
+    String iPath = srcPaths[i];// last component
+    if (curInode.resolveInternal(iPath) != null) {
+      //  directory/link already exists
+      String path = srcPaths[0];
+      for (int j = 1; j <= i; ++j) {
+        path += "/" + srcPaths[j];
+      }
+      throw new FileAlreadyExistsException("Path " + path +
+            " already exists as dir; cannot create link here");
+    }
+    
+    final INodeLink<T> newLink;
+    final String fullPath = curInode.fullPath + (curInode == root ? "" : "/")
+        + iPath;
+    if (isLinkMerge) { // Target is list of URIs
+      String[] targetsList = StringUtils.getStrings(target);
+      URI[] targetsListURI = new URI[targetsList.length];
+      int k = 0;
+      for (String itarget : targetsList) {
+        targetsListURI[k++] = new URI(itarget);
+      }
+      newLink = new INodeLink<T>(fullPath, aUgi,
+          getTargetFileSystem(targetsListURI), targetsListURI);
+    } else {
+      newLink = new INodeLink<T>(fullPath, aUgi,
+          getTargetFileSystem(new URI(target)), new URI(target));
+    }
+    curInode.addLink(iPath, newLink); 
+  }
+  
+  /**
+   * Below the "public" methods of InodeTree
+   */
+  
+  /**
+   * The user of this class must subclass and implement the following
+   * 3 abstract methods.
+   * @throws IOException 
+   */
+  protected abstract T getTargetFileSystem(final URI uri)
+    throws UnsupportedFileSystemException, URISyntaxException, IOException;
+  
+  protected abstract T getTargetFileSystem(final INodeDir<T> dir)
+    throws URISyntaxException;
+  
+  protected abstract T getTargetFileSystem(final URI[] mergeFsURIList)
+  throws UnsupportedFileSystemException, URISyntaxException;
+  
+  /**
+   * Create Inode Tree from the specified mount-table specified in Config
+   * @param config - the mount table keys are prefixed with 
+   *       FsConstants.CONFIG_VIEWFS_PREFIX
+   * @param viewName - the name of the mount table - if null use defaultMT name
+   * @throws UnsupportedFileSystemException
+   * @throws URISyntaxException
+   * @throws FileAlreadyExistsException
+   * @throws IOException
+   */
+  protected InodeTree(final Configuration config, String viewName)
+      throws UnsupportedFileSystemException, URISyntaxException,
+    FileAlreadyExistsException, IOException { 
+    if (viewName == null) {
+      viewName = Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE;
+    }
+    root = new INodeDir<T>("/", UserGroupInformation.getCurrentUser());
+    root.InodeDirFs = getTargetFileSystem(root);
+    root.isRoot = true;
+    
+    final String mtPrefix = Constants.CONFIG_VIEWFS_PREFIX + "." + 
+                            viewName + ".";
+    final String linkPrefix = Constants.CONFIG_VIEWFS_LINK + ".";
+    final String linkMergePrefix = Constants.CONFIG_VIEWFS_LINK_MERGE + ".";
+    boolean gotMountTableEntry = false;
+    final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    for (Entry<String, String> si : config) {
+      final String key = si.getKey();
+      if (key.startsWith(mtPrefix)) {
+        gotMountTableEntry = true;
+        boolean isMergeLink = false;
+        String src = key.substring(mtPrefix.length());
+        if (src.startsWith(linkPrefix)) {
+          src = src.substring(linkPrefix.length());
+        } else if (src.startsWith(linkMergePrefix)) { // A merge link
+          isMergeLink = true;
+          src = src.substring(linkMergePrefix.length());
+        } else {
+          throw new IOException(
+          "ViewFs: Cannot initialize: Invalid entry in Mount table in config: "+ 
+          src);
+        }
+        final String target = si.getValue(); // link or merge link
+        createLink(src, target, isMergeLink, ugi); 
+      }  
+    }
+    if (!gotMountTableEntry) {
+      throw new IOException(
+          "ViewFs: Cannot initialize: Empty Mount table in config");
+    }
+  }
+
+  /**
+   * Resolve returns ResolveResult.
+   * The caller can continue the resolution of the remainingPath
+   * in the targetFileSystem.
+   * 
+   * If the input pathname leads to link to another file system then
+   * the targetFileSystem is the one denoted by the link (except it is
+   * file system chrooted to link target.
+   * If the input pathname leads to an internal mount-table entry then
+   * the target file system is one that represents the internal inode.
+   */
+  static class ResolveResult<T> {
+    final ResultKind kind;
+    final T targetFileSystem;
+    final String resolvedPath;
+    final Path remainingPath;   // to resolve in the target FileSystem
+    
+    ResolveResult(final ResultKind k, final T targetFs, final String resolveP,
+        final Path remainingP) {
+      kind = k;
+      targetFileSystem = targetFs;
+      resolvedPath = resolveP;
+      remainingPath = remainingP;  
+    }
+    
+    // isInternalDir of path resolution completed within the mount table 
+    boolean isInternalDir() {
+      return (kind == ResultKind.isInternalDir);
+    }
+  }
+  
+  /**
+   * Resolve the pathname p relative to root InodeDir
+   * @param p - inout path
+   * @param resolveLastComponent 
+   * @return ResolveResult which allows further resolution of the remaining path
+   * @throws FileNotFoundException
+   */
+  ResolveResult<T> resolve(final String p, final boolean resolveLastComponent)
+    throws FileNotFoundException {
+    // TO DO: - more efficient to not split the path, but simply compare
+    String[] path = breakIntoPathComponents(p); 
+    if (path.length <= 1) { // special case for when path is "/"
+      ResolveResult<T> res = 
+        new ResolveResult<T>(ResultKind.isInternalDir, 
+              root.InodeDirFs, root.fullPath, SlashPath);
+      return res;
+    }
+    
+    INodeDir<T> curInode = root;
+    int i;
+    // ignore first slash
+    for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) {
+      INode<T> nextInode = curInode.resolveInternal(path[i]);
+      if (nextInode == null) {
+        String failedAt = path[0];
+        for ( int j = 1; j <=i; ++j) {
+          failedAt += "/" + path[j];
+        }
+        throw (new FileNotFoundException(failedAt));      
+      }
+
+      if (nextInode instanceof INodeLink) {
+        final INodeLink<T> link = (INodeLink<T>) nextInode;
+        final Path remainingPath;
+        if (i >= path.length-1) {
+          remainingPath = SlashPath;
+        } else {
+          String remainingPathStr = "/" + path[i+1];
+          for (int j = i+2; j< path.length; ++j) {
+            remainingPathStr += "/" + path[j];
+          }
+          remainingPath = new Path(remainingPathStr);
+        }
+        final ResolveResult<T> res = 
+          new ResolveResult<T>(ResultKind.isExternalDir,
+              link.targetFileSystem, nextInode.fullPath, remainingPath);
+        return res;
+      } else if (nextInode instanceof INodeDir) {
+        curInode = (INodeDir<T>) nextInode;
+      }
+    }
+
+    // We have resolved to an internal dir in mount table.
+    Path remainingPath;
+    if (resolveLastComponent) {
+      remainingPath = SlashPath;
+    } else {
+      // note we have taken care of when path is "/" above
+      // for internal dirs rem-path does not start with / since the lookup
+      // that follows will do a children.get(remaningPath) and will have to
+      // strip-out the initial /
+      String remainingPathStr =  "/" + path[i];
+      for (int j = i+1; j< path.length; ++j) {
+        remainingPathStr += "/" + path[j];
+      }
+      remainingPath = new Path(remainingPathStr);
+    }
+    final ResolveResult<T> res = 
+       new ResolveResult<T>(ResultKind.isInternalDir,
+           curInode.InodeDirFs, curInode.fullPath, remainingPath); 
+    return res;
+  }
+}

+ 626 - 0
src/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -0,0 +1,626 @@
+/**
+ * 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.viewfs;
+
+import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_RRR;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.StringTokenizer;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.viewfs.InodeTree.INode;
+import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * ViewFileSystem (extends the FileSystem interface) implements a client-side
+ * mount table. Its spec and implementation is identical to {@link ViewFs}.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
+public class ViewFileSystem extends FileSystem {
+  static final AccessControlException READONLY_MOUNTABLE =
+    new AccessControlException( "InternalDir of ViewFileSystem is readonly");
+  
+  final long creationTime; // of the the mount table
+  final UserGroupInformation ugi; // the user/group of user who created mtable
+  private Path workingDir;
+  Configuration config;
+  InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
+  
+  /**
+   * Prohibits names which contain a ".", "..", ":" or "/" 
+   */
+  private static boolean isValidName(final String src) {
+    // Check for ".." "." ":" "/"
+    final StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
+    while(tokens.hasMoreTokens()) {
+      String element = tokens.nextToken();
+      if (element.equals("..") || 
+          element.equals(".")  ||
+          (element.indexOf(":") >= 0)) {
+        return false;
+      }
+    }
+    return true;
+  }
+  
+  /**
+   * Make the path Absolute and get the path-part of a pathname.
+   * Checks that URI matches this file system 
+   * and that the path-part is a valid name.
+   * 
+   * @param p path
+   * @return path-part of the Path p
+   */
+  private String getUriPath(final Path p) {
+    checkPath(p);
+    String s = makeAbsolute(p).toUri().getPath();
+    if (!isValidName(s)) {
+      throw new InvalidPathException("Path part " + s + " from URI" + p
+          + " is not a valid filename.");
+    }
+    return s;
+  }
+  
+  private Path makeAbsolute(final Path f) {
+    return f.isAbsolute() ? f : new Path(workingDir, f);
+  }
+  
+  /**
+   * This is the  constructor with the signature needed by
+   * {@link FileSystem#createFileSystem(URI, Configuration)}
+   * 
+   * After this constructor is called initialize() is called.
+   * @throws IOException 
+   */
+  public ViewFileSystem() throws IOException {
+    ugi = UserGroupInformation.getCurrentUser();
+    creationTime = System.currentTimeMillis();
+  }
+
+  /**
+   * Called after a new FileSystem instance is constructed.
+   * @param theUri a uri whose authority section names the host, port, etc. for
+   *          this FileSystem
+   * @param conf the configuration
+   */
+  public void initialize(final URI theUri, final Configuration conf)
+      throws IOException {
+    super.initialize(theUri, conf);
+    setConf(conf);
+    workingDir =
+      this.makeQualified(new Path("/user/" + ugi.getShortUserName()));
+    config = conf;
+    // Now build  client side view (i.e. client side mount table) from config.
+    final String authority = theUri.getAuthority();
+    try {
+      fsState = new InodeTree<FileSystem>(conf, authority) {
+
+        @Override
+        protected
+        FileSystem getTargetFileSystem(final URI uri)
+          throws URISyntaxException, IOException {
+            return new ChRootedFileSystem(FileSystem.get(uri, config), 
+                new Path(uri.getPath()));
+        }
+
+        @Override
+        protected
+        FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
+          throws URISyntaxException {
+          return new InternalDirOfViewFs(dir, creationTime, ugi);
+        }
+
+        @Override
+        protected
+        FileSystem getTargetFileSystem(URI[] mergeFsURIList)
+            throws URISyntaxException, UnsupportedFileSystemException {
+          throw new UnsupportedFileSystemException("mergefs not implemented");
+          // return MergeFs.createMergeFs(mergeFsURIList, config);
+        }
+      };
+    } catch (URISyntaxException e) {
+      throw new IOException("URISyntax exception: " + theUri);
+    }
+
+  }
+  
+  
+  /**
+   * Convenience Constructor for apps to call directly
+   * @param theUri which must be that of ViewFileSystem
+   * @param conf
+   * @throws IOException
+   */
+  ViewFileSystem(final URI theUri, final Configuration conf)
+    throws IOException {
+    this();
+    initialize(FsConstants.VIEWFS_URI, conf);
+  }
+  
+  /**
+   * Convenience Constructor for apps to call directly
+   * @param conf
+   * @throws IOException
+   */
+  public ViewFileSystem(final Configuration conf) throws IOException {
+    this(FsConstants.VIEWFS_URI, conf);
+  }
+  
+  public Path getTrashCanLocation(final Path f) throws FileNotFoundException {
+    final InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.isInternalDir() ? null : res.targetFileSystem.getHomeDirectory();
+  }
+
+  @Override
+  public URI getUri() {
+    return FsConstants.VIEWFS_URI;
+  }
+  
+  /**
+   * Return the fully-qualified path of path f - ie follow the path
+   * through the mount point.
+   * @param f path
+   * @return resolved fully-qualified path
+   * @throws FileNotFoundException
+   */
+  public Path getResolvedQualifiedPath(final Path f)
+      throws FileNotFoundException {
+    final InodeTree.ResolveResult<FileSystem> res;
+      res = fsState.resolve(getUriPath(f), true);
+    if (res.isInternalDir()) {
+      return f;
+    }
+    final ChRootedFileSystem targetFs = 
+      (ChRootedFileSystem) res.targetFileSystem;
+    return targetFs.getResolvedQualifiedPath(res.remainingPath);
+  }
+  
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public void setWorkingDirectory(final Path new_dir) {
+    getUriPath(new_dir); // this validates the path
+    workingDir = makeAbsolute(new_dir);
+  }
+  
+  @Override
+  public FSDataOutputStream append(final Path f, final int bufferSize,
+      final Progressable progress) throws IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.append(res.remainingPath, bufferSize, progress);
+  }
+  
+  @Override
+  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 {
+    InodeTree.ResolveResult<FileSystem> res;
+    try {
+      res = fsState.resolve(getUriPath(f), false);
+    } catch (FileNotFoundException e) {
+        throw READONLY_MOUNTABLE;
+    }
+    assert(res.remainingPath != null);
+    return res.targetFileSystem.create(res.remainingPath, permission,
+         overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  
+  @Override
+  public boolean delete(final Path f, final boolean recursive)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    // If internal dir or target is a mount link (ie remainingPath is Slash)
+    if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
+      throw READONLY_MOUNTABLE;
+    }
+    return res.targetFileSystem.delete(res.remainingPath, recursive);
+  }
+  
+  @Override
+  public boolean delete(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+      return delete(f, true);
+  }
+  
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus fs, 
+      long start, long len) throws IOException {
+    final InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(fs.getPath()), true);
+    return res.targetFileSystem.getFileBlockLocations(
+          new ViewFsFileStatus(fs, res.remainingPath), start, len);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.getFileChecksum(f);
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) throws AccessControlException,
+      FileNotFoundException, IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    
+    // FileStatus#getPath is a fully qualified path relative to the root of 
+    // target file system.
+    // We need to change it to viewfs URI - relative to root of mount table.
+    
+    // The implementors of RawLocalFileSystem were trying to be very smart.
+    // They implement FileStatus#getOwener lazily -- the object
+    // returned is really a RawLocalFileSystem that expect the
+    // FileStatus#getPath to be unchanged so that it can get owner when needed.
+    // Hence we need to interpose a new ViewFileSystemFileStatus that 
+    // works around.
+    FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
+    return new ViewFsFileStatus(status, this.makeQualified(f));
+  }
+  
+  
+  @Override
+  public FileStatus[] listStatus(final Path f) throws AccessControlException,
+      FileNotFoundException, IOException {
+    InodeTree.ResolveResult<FileSystem> res =
+      fsState.resolve(getUriPath(f), true);
+    
+    FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
+    if (!res.isInternalDir()) {
+      // We need to change the name in the FileStatus as described in
+      // {@link #getFileStatus }
+      ChRootedFileSystem targetFs;
+      targetFs = (ChRootedFileSystem) res.targetFileSystem;
+      int i = 0;
+      for (FileStatus status : statusLst) {
+          String suffix = targetFs.stripOutRoot(status.getPath());
+          statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
+              suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
+      }
+    }
+    return statusLst;
+  }
+
+  @Override
+  public boolean mkdirs(final Path dir, final FsPermission permission)
+      throws IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(dir), false);
+   return  res.targetFileSystem.mkdirs(res.remainingPath, permission);
+  }
+
+  @Override
+  public FSDataInputStream open(final Path f, final int bufferSize)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+        fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.open(res.remainingPath, bufferSize);
+  }
+
+  
+  @Override
+  public boolean rename(final Path src, final Path dst) throws IOException {
+    // passing resolveLastComponet as false to catch renaming a mount point to 
+    // itself. We need to catch this as an internal operation and fail.
+    InodeTree.ResolveResult<FileSystem> resSrc = 
+      fsState.resolve(getUriPath(src), false); 
+  
+    if (resSrc.isInternalDir()) {
+      throw READONLY_MOUNTABLE;
+    }
+      
+    InodeTree.ResolveResult<FileSystem> resDst = 
+      fsState.resolve(getUriPath(dst), false);
+    if (resDst.isInternalDir()) {
+      throw new AccessControlException(
+          "Cannot Rename within internal dirs of mount table: it is readOnly");
+    }
+    /**
+    // Alternate 1: renames within same file system - valid but we disallow
+    // Alternate 2: (as described in next para - valid but we have disallowed it
+    //
+    // Note we compare the URIs. the URIs include the link targets. 
+    // hence we allow renames across mount links as long as the mount links
+    // point to the same target.
+    if (!resSrc.targetFileSystem.getUri().equals(
+              resDst.targetFileSystem.getUri())) {
+      throw new IOException("Renames across Mount points not supported");
+    }
+    */
+    
+    //
+    // Alternate 3 : renames ONLY within the the same mount links.
+    //
+    if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
+      throw new IOException("Renames across Mount points not supported");
+    }
+    return resSrc.targetFileSystem.rename(resSrc.remainingPath,
+        resDst.remainingPath);
+  }
+  
+  @Override
+  public void setOwner(final Path f, final String username,
+      final String groupname) throws AccessControlException,
+      FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
+  }
+
+  @Override
+  public void setPermission(final Path f, final FsPermission permission)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setPermission(res.remainingPath, permission); 
+  }
+
+  @Override
+  public boolean setReplication(final Path f, final short replication)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.setReplication(res.remainingPath, replication);
+  }
+
+  @Override
+  public void setTimes(final Path f, final long mtime, final long atime)
+      throws AccessControlException, FileNotFoundException,
+      IOException {
+    InodeTree.ResolveResult<FileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
+  }
+
+  @Override
+  public void setVerifyChecksum(final boolean verifyChecksum) { 
+    // This is a file system level operations, however ViewFileSystem 
+    // points to many file systems. Noop for ViewFileSystem.
+  }
+  
+  /*
+   * An instance of this class represents an internal dir of the viewFs 
+   * that is internal dir of the mount table.
+   * It is a read only mount tables and create, mkdir or delete operations
+   * are not allowed.
+   * If called on create or mkdir then this target is the parent of the
+   * directory in which one is trying to create or mkdir; hence
+   * in this case the path name passed in is the last component. 
+   * Otherwise this target is the end point of the path and hence
+   * the path name passed in is null. 
+   */
+  static class InternalDirOfViewFs extends FileSystem {
+    final InodeTree.INodeDir<FileSystem>  theInternalDir;
+    final long creationTime; // of the the mount table
+    final UserGroupInformation ugi; // the user/group of user who created mtable
+    
+    public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
+        final long cTime, final UserGroupInformation ugi)
+      throws URISyntaxException {
+      try {
+        initialize(FsConstants.VIEWFS_URI, new Configuration());
+      } catch (IOException e) {
+        throw new RuntimeException("Cannot occur");
+      }
+      theInternalDir = dir;
+      creationTime = cTime;
+      this.ugi = ugi;
+    }
+
+    static private void checkPathIsSlash(final Path f) throws IOException {
+      if (f != InodeTree.SlashPath) {
+        throw new IOException (
+        "Internal implementation error: expected file name to be /" );
+      }
+    }
+    
+    @Override
+    public URI getUri() {
+      return FsConstants.VIEWFS_URI;
+    }
+
+    @Override
+    public Path getWorkingDirectory() {
+      throw new RuntimeException (
+      "Internal impl error: getWorkingDir should not have been called" );
+    }
+
+    @Override
+    public void setWorkingDirectory(final Path new_dir) {
+      throw new RuntimeException (
+      "Internal impl error: getWorkingDir should not have been called" ); 
+    }
+
+    @Override
+    public FSDataOutputStream append(final Path f, final int bufferSize,
+        final Progressable progress) throws IOException {
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    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 AccessControlException {
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public boolean delete(final Path f, final boolean recursive)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+    
+    @Override
+    public boolean delete(final Path f)
+        throws AccessControlException, IOException {
+      return delete(f, true);
+    }
+
+    @Override
+    public BlockLocation[] getFileBlockLocations(final FileStatus fs,
+        final long start, final long len) throws 
+        FileNotFoundException, IOException {
+      checkPathIsSlash(fs.getPath());
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public FileChecksum getFileChecksum(final Path f)
+        throws FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path f) throws IOException {
+      checkPathIsSlash(f);
+      return new FileStatus(0, true, 0, 0, creationTime, creationTime,
+          PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+
+          new Path(theInternalDir.fullPath).makeQualified(
+              FsConstants.VIEWFS_URI, null));
+    }
+    
+
+    @Override
+    public FileStatus[] listStatus(Path f) throws AccessControlException,
+        FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      FileStatus[] result = new FileStatus[theInternalDir.children.size()];
+      int i = 0;
+      for (Entry<String, INode<FileSystem>> iEntry : 
+                                          theInternalDir.children.entrySet()) {
+        INode<FileSystem> inode = iEntry.getValue();
+        if (inode instanceof INodeLink ) {
+          INodeLink<FileSystem> link = (INodeLink<FileSystem>) inode;
+
+          result[i++] = new FileStatus(0, false, 0, 0,
+            creationTime, creationTime, PERMISSION_RRR,
+            ugi.getUserName(), ugi.getGroupNames()[0],
+            link.getTargetLink(),
+            new Path(inode.fullPath).makeQualified(
+                FsConstants.VIEWFS_URI, null));
+        } else {
+          result[i++] = new FileStatus(0, true, 0, 0,
+            creationTime, creationTime, PERMISSION_RRR,
+            ugi.getUserName(), ugi.getGroupNames()[0],
+            new Path(inode.fullPath).makeQualified(
+                FsConstants.VIEWFS_URI, null));
+        }
+      }
+      return result;
+    }
+
+    @Override
+    public boolean mkdirs(Path dir, FsPermission permission)
+        throws AccessControlException, FileAlreadyExistsException {
+      if (theInternalDir.isRoot & dir == null) {
+        throw new FileAlreadyExistsException("/ already exits");
+      }
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public FSDataInputStream open(Path f, int bufferSize)
+        throws AccessControlException, FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public boolean rename(Path src, Path dst) throws AccessControlException,
+        IOException {
+      checkPathIsSlash(src);
+      checkPathIsSlash(dst);
+      throw READONLY_MOUNTABLE;     
+    }
+
+    @Override
+    public void setOwner(Path f, String username, String groupname)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public void setPermission(Path f, FsPermission permission)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;    
+    }
+
+    @Override
+    public boolean setReplication(Path f, short replication)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public void setTimes(Path f, long mtime, long atime)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;    
+    }
+
+    @Override
+    public void setVerifyChecksum(boolean verifyChecksum) {
+      // Noop for viewfs
+    }
+  }
+}

+ 816 - 0
src/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -0,0 +1,816 @@
+/**
+ * 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.viewfs;
+
+import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_RRR;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.local.LocalConfigKeys;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.viewfs.InodeTree.INode;
+import org.apache.hadoop.fs.viewfs.InodeTree.INodeLink;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * 
+ * ViewFs (extends the AbstractFileSystem interface) implements a client-side
+ * mount table. The viewFs file system is implemented completely in memory on
+ * the client side. The client-side mount table allows a client to provide a 
+ * customized view of a file system namespace that is composed from 
+ * one or more individual file systems (a localFs or Hdfs, S3fs, etc).
+ * For example one could have a mount table that provides links such as
+ * <ul>
+ * <li>  /user          -> hdfs://nnContainingUserDir/user
+ * <li>  /project/foo   -> hdfs://nnProject1/projects/foo
+ * <li>  /project/bar   -> hdfs://nnProject2/projects/bar
+ * <li>  /tmp           -> hdfs://nnTmp/privateTmpForUserXXX
+ * </ul> 
+ * 
+ * ViewFs is specified with the following URI: <b>viewfs:///</b> 
+ * <p>
+ * To use viewfs one would typically set the default file system in the
+ * config  (i.e. fs.defaultFS = viewfs:///) along with the
+ * mount table config variables as described below. 
+ * 
+ * <p>
+ * <b> ** Config variables to specify the mount table entries ** </b>
+ * <p>
+ * 
+ * The file system is initialized from the standard Hadoop config through
+ * config variables.
+ * See {@link FsConstants} for URI and Scheme constants; 
+ * See {@link Constants} for config var constants; 
+ * see {@link ConfigUtil} for convenient lib.
+ * 
+ * <p>
+ * All the mount table config entries for view fs are prefixed by 
+ * <b>fs.viewFs.</b>
+ * For example the above example can be specfied with the following
+ *  config variables:
+ *  <ul>
+ *  <li> fs.viewFs.defaultMT.link./user=hdfs://nnContainingUserDir/user
+ *  <li> fs.viewFs.defaultMT.link./project/foo=hdfs://nnProject1/projects/foo
+ *  <li> fs.viewFs.defaultMT.link./project/bar=hdfs://nnProject2/projects/bar
+ *  <li> fs.viewFs.defaultMT.link./tmp=hdfs://nnTmp/privateTmpForUserXXX
+ *  </ul>
+ *  
+ * The default mount table (when no authority is specified) is 
+ * from config variables prefixed by <b>fs.viewFs.defaultMT </b>
+ * The authority component of a URI can be used to specify a different mount
+ * table. For example,
+ * <ul>
+ * <li>  viewfs://sanjayMountable/
+ * </ul>
+ * is initialized from the fs.viewFs.sanjayMountable.* config variables.
+ * 
+ *  <p> 
+ *  <b> **** Merge Mounts **** </b>(NOTE: merge mounts are not implemented yet.)
+ *  <p>
+ *  
+ *   One can also use "MergeMounts" to merge several directories (this is
+ *   sometimes  called union-mounts or junction-mounts in the literature.
+ *   For example of the home directories are stored on say two file systems
+ *   (because they do not fit on one) then one could specify a mount
+ *   entry such as following merges two dirs:
+ *   <ul>
+ *   <li> /user -> hdfs://nnUser1/user,hdfs://nnUser2/user
+ *   </ul>
+
+ *  Such a mergeLink can be specifed with the following config var where ","
+ *  is used as the seperater for each of links to be merged:
+ *  <ul>
+ *  <li> fs.viewFs.defaultMT.linkMerge./user=
+ *            hdfs://nnUser1/user,hdfs://nnUser1/user
+ *  </ul>
+ *   A special case of the merge mount is where mount table's root is merged
+ *   with the root (slash) of another file system:
+ *   <ul>
+ *   <li>    fs.viewFs.defaultMT.linkMergeSlash=hdfs://nn99/
+ *   </ul>
+ *   In this cases the root of the mount table is merged with the root of
+ *            <b>hdfs://nn99/ </b> 
+ * 
+ */
+/**
+ * ViewFs (extends the AbstractFileSystem interface) implements a client-side
+ * mount table. The viewFs file system is implemented completely in memory on
+ * the client side. The client-side mount table allows a client to provide a 
+ * customized view of a file system namespace that is composed from 
+ * one or more individual file systems (a localFs or Hdfs, S3fs, etc).
+ * For example one could have a mount table that provides links such as
+ * <ul>
+ * <li>  /user          -> hdfs://nnContainingUserDir/user
+ * <li>  /project/foo   -> hdfs://nnProject1/projects/foo
+ * <li>  /project/bar   -> hdfs://nnProject2/projects/bar
+ * <li>  /tmp           -> hdfs://nnTmp/privateTmpForUserXXX
+ * </ul> 
+ * 
+ * ViewFileSystem is specified with the following URI: <b>viewfs:///</b> 
+ * <p>
+ * To use viewfs one would typically set the default file system in the
+ * config  (i.e. fs.default.name< = viewfs:///) along with the
+ * mount table config variables as described below. 
+ * If your core-site.xml does not have the following config value please add it
+ * to your config: fs.viewfs.impl = org.apache.hadoop.fs.viewfs.ViewFileSystem
+ * 
+ * <p>
+ * <b> ** Config variables to specify the mount table entries ** </b>
+ * <p>
+ * 
+ * The file system is initialized from the standard Hadoop config through
+ * config variables.
+ * See {@link FsConstants} for URI and Scheme constants; 
+ * See {@link Constants} for config var constants; 
+ * see {@link ConfigUtil} for convenient lib.
+ * 
+ * <p>
+ * All the mount table config entries for view fs are prefixed by 
+ * <b>fs.viewfs.mounttable.</b>
+ * For example the above example can be specified with the following
+ *  config variables:
+ *  <ul>
+ *  <li> fs.viewfs.mounttable.default.link./user=
+ *  hdfs://nnContainingUserDir/user
+ *  <li> fs.viewfs.mounttable.default.link./project/foo=
+ *  hdfs://nnProject1/projects/foo
+ *  <li> fs.viewfs.mounttable.default.link./project/bar=
+ *  hdfs://nnProject2/projects/bar
+ *  <li> fs.viewfs.mounttable.default.link./tmp=
+ *  hdfs://nnTmp/privateTmpForUserXXX
+ *  </ul>
+ *  
+ * The default mount table (when no authority is specified) is 
+ * from config variables prefixed by <b>fs.viewFs.defaultMT </b>
+ * The authority component of a URI can be used to specify a different mount
+ * table. For example,
+ * <ul>
+ * <li>  viewfs://sanjayMountable/
+ * </ul>
+ * is initialized from the fs.viewFs.sanjayMountable.* config variables.
+ * 
+ *  <p> 
+ *  <b> **** Merge Mounts **** </b>(NOTE: merge mounts are not implemented yet.)
+ *  <p>
+ *  
+ *   One can also use "MergeMounts" to merge several directories (this is
+ *   sometimes  called union-mounts or junction-mounts in the literature.
+ *   For example of the home directories are stored on say two file systems
+ *   (because they do not fit on one) then one could specify a mount
+ *   entry such as following merges two dirs:
+ *   <ul>
+ *   <li> /user -> hdfs://nnUser1/user,hdfs://nnUser2/user
+ *   </ul>
+
+ *  Such a mergeLink can be specified with the following config var where ","
+ *  is used as the separator for each of links to be merged:
+ *  <ul>
+ *  <li> fs.viewfs.mounttable.default.linkMerge./user=
+ *  hdfs://nnUser1/user,hdfs://nnUser1/user
+ *  </ul>
+ *   A special case of the merge mount is where mount table's root is merged
+ *   with the root (slash) of another file system:
+ *   <ul>
+ *   <li>    fs.viewfs.mounttable.default.linkMergeSlash=hdfs://nn99/
+ *   </ul>
+ *   In this cases the root of the mount table is merged with the root of
+ *            <b>hdfs://nn99/ </b> 
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
+public class ViewFs extends AbstractFileSystem {
+  final long creationTime; // of the the mount table
+  final UserGroupInformation ugi; // the user/group of user who created mtable
+  final Configuration config;
+  InodeTree<AbstractFileSystem> fsState;  // the fs state; ie the mount table
+  
+  static final AccessControlException READONLY_MOUNTABLE =
+    new AccessControlException("InternalDir of ViewFs is readonly");
+  public ViewFs(final Configuration conf) throws IOException,
+      URISyntaxException {
+    this(FsConstants.VIEWFS_URI, conf);
+  }
+  
+  /**
+   * This constructor has the signature needed by
+   * {@link AbstractFileSystem#createFileSystem(URI, Configuration)}.
+   * 
+   * @param theUri which must be that of ViewFs
+   * @param conf
+   * @throws IOException
+   * @throws URISyntaxException 
+   */
+  ViewFs(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, FsConstants.VIEWFS_SCHEME, false, -1);
+    creationTime = System.currentTimeMillis();
+    ugi = UserGroupInformation.getCurrentUser();
+    config = conf;
+    // Now build  client side view (i.e. client side mount table) from config.
+    String authority = theUri.getAuthority();
+    fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
+
+      @Override
+      protected
+      AbstractFileSystem getTargetFileSystem(final URI uri)
+        throws URISyntaxException, UnsupportedFileSystemException {
+          return new ChRootedFs(
+              AbstractFileSystem.createFileSystem(uri, config),
+              new Path(uri.getPath()));
+      }
+
+      @Override
+      protected
+      AbstractFileSystem getTargetFileSystem(
+          final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
+        return new InternalDirOfViewFs(dir, creationTime, ugi);
+      }
+
+      @Override
+      protected
+      AbstractFileSystem getTargetFileSystem(URI[] mergeFsURIList)
+          throws URISyntaxException, UnsupportedFileSystemException {
+        throw new UnsupportedFileSystemException("mergefs not implemented yet");
+        // return MergeFs.createMergeFs(mergeFsURIList, config);
+      }
+    };
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return LocalConfigKeys.getServerDefaults(); 
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+ 
+  @Override
+  public FSDataOutputStream createInternal(final Path f,
+      final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress, final int bytesPerChecksum,
+      final boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, UnsupportedFileSystemException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res;
+    try {
+      res = fsState.resolve(getUriPath(f), false);
+    } catch (FileNotFoundException e) {
+      if (createParent) {
+        throw READONLY_MOUNTABLE;
+      } else {
+        throw e;
+      }
+    }
+    assert(res.remainingPath != null);
+    return res.targetFileSystem.createInternal(res.remainingPath, flag,
+        absolutePermission, bufferSize, replication,
+        blockSize, progress, bytesPerChecksum,
+        createParent);
+  }
+
+  @Override
+  public boolean delete(final Path f, final boolean recursive)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    // If internal dir or target is a mount link (ie remainingPath is Slash)
+    if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
+      throw new AccessControlException(
+          "Cannot delete internal mount table directory: " + f);
+    }
+    return res.targetFileSystem.delete(res.remainingPath, recursive);
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final Path f, final long start,
+      final long len) throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return
+      res.targetFileSystem.getFileBlockLocations(res.remainingPath, start, len);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.getFileChecksum(f);
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+
+    //  FileStatus#getPath is a fully qualified path relative to the root of 
+    // target file system.
+    // We need to change it to viewfs URI - relative to root of mount table.
+    
+    // The implementors of RawLocalFileSystem were trying to be very smart.
+    // They implement FileStatus#getOwener lazily -- the object
+    // returned is really a RawLocalFileSystem that expect the
+    // FileStatus#getPath to be unchanged so that it can get owner when needed.
+    // Hence we need to interpose a new ViewFsFileStatus that works around.
+    
+    
+    FileStatus status =  res.targetFileSystem.getFileStatus(res.remainingPath);
+    return new ViewFsFileStatus(status, this.makeQualified(f));
+  }
+
+  @Override
+  public FileStatus getFileLinkStatus(final Path f)
+     throws AccessControlException, FileNotFoundException,
+     UnsupportedFileSystemException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), false); // do not follow mount link
+    return res.targetFileSystem.getFileLinkStatus(res.remainingPath);
+  }
+  
+  @Override
+  public FsStatus getFsStatus() throws AccessControlException,
+      FileNotFoundException, IOException {
+    return new FsStatus(0, 0, 0);
+  }
+
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(final Path f)
+    throws AccessControlException, FileNotFoundException,
+    UnresolvedLinkException, IOException {
+    final InodeTree.ResolveResult<AbstractFileSystem> res =
+      fsState.resolve(getUriPath(f), true);
+    final RemoteIterator<FileStatus> fsIter =
+      res.targetFileSystem.listStatusIterator(res.remainingPath);
+    if (res.isInternalDir()) {
+      return fsIter;
+    }
+    
+    return new RemoteIterator<FileStatus>() {
+      final RemoteIterator<FileStatus> myIter;
+      final ChRootedFs targetFs;
+      { // Init
+          myIter = fsIter;
+          targetFs = (ChRootedFs) res.targetFileSystem;
+      }
+      
+      @Override
+      public boolean hasNext() throws IOException {
+        return myIter.hasNext();
+      }
+      
+      @Override
+      public FileStatus next() throws IOException {
+        FileStatus status =  myIter.next();
+        String suffix = targetFs.stripOutRoot(status.getPath());
+        return new ViewFsFileStatus(status, makeQualified(
+            suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
+      }
+    };
+  }
+  
+  @Override
+  public FileStatus[] listStatus(final Path f) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res =
+      fsState.resolve(getUriPath(f), true);
+    
+    FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath);
+    if (!res.isInternalDir()) {
+      // We need to change the name in the FileStatus as described in
+      // {@link #getFileStatus }
+      ChRootedFs targetFs;
+      targetFs = (ChRootedFs) res.targetFileSystem;
+      int i = 0;
+      for (FileStatus status : statusLst) {
+          String suffix = targetFs.stripOutRoot(status.getPath());
+          statusLst[i++] = new ViewFsFileStatus(status, this.makeQualified(
+              suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix)));
+      }
+    }
+    return statusLst;
+  }
+
+  @Override
+  public void mkdir(final Path dir, final FsPermission permission,
+      final boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(dir), false);
+    res.targetFileSystem.mkdir(res.remainingPath, permission, createParent);
+  }
+
+  @Override
+  public FSDataInputStream open(final Path f, final int bufferSize)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+        fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.open(res.remainingPath, bufferSize);
+  }
+
+  
+  @Override
+  public void renameInternal(final Path src, final Path dst,
+      final boolean overwrite) throws IOException, UnresolvedLinkException {
+    // passing resolveLastComponet as false to catch renaming a mount point 
+    // itself we need to catch this as an internal operation and fail.
+    InodeTree.ResolveResult<AbstractFileSystem> resSrc = 
+      fsState.resolve(getUriPath(src), false); 
+  
+    if (resSrc.isInternalDir()) {
+      throw new AccessControlException(
+          "Cannot Rename within internal dirs of mount table: it is readOnly");
+    }
+      
+    InodeTree.ResolveResult<AbstractFileSystem> resDst = 
+                                fsState.resolve(getUriPath(dst), false);
+    if (resDst.isInternalDir()) {
+      throw new AccessControlException(
+          "Cannot Rename within internal dirs of mount table: it is readOnly");
+    }
+    
+    /**
+    // Alternate 1: renames within same file system - valid but we disallow
+    // Alternate 2: (as described in next para - valid but we have disallowed it
+    //
+    // Note we compare the URIs. the URIs include the link targets. 
+    // hence we allow renames across mount links as long as the mount links
+    // point to the same target.
+    if (!resSrc.targetFileSystem.getUri().equals(
+              resDst.targetFileSystem.getUri())) {
+      throw new IOException("Renames across Mount points not supported");
+    }
+    */
+    
+    //
+    // Alternate 3 : renames ONLY within the the same mount links.
+    //
+
+    if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
+      throw new IOException("Renames across Mount points not supported");
+    }
+    
+    resSrc.targetFileSystem.renameInternal(resSrc.remainingPath,
+      resDst.remainingPath, overwrite);
+  }
+
+  @Override
+  public void renameInternal(final Path src, final Path dst)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, ParentNotDirectoryException,
+      UnresolvedLinkException, IOException {
+    renameInternal(src, dst, false);
+  }
+  
+  @Override
+  public boolean supportsSymlinks() {
+    return true;
+  }
+  
+  @Override
+  public void createSymlink(final Path target, final Path link,
+      final boolean createParent) throws IOException, UnresolvedLinkException {
+    InodeTree.ResolveResult<AbstractFileSystem> res;
+    try {
+      res = fsState.resolve(getUriPath(link), false);
+    } catch (FileNotFoundException e) {
+      if (createParent) {
+        throw READONLY_MOUNTABLE;
+      } else {
+        throw e;
+      }
+    }
+    assert(res.remainingPath != null);
+    res.targetFileSystem.createSymlink(target, res.remainingPath,
+        createParent);  
+  }
+
+  @Override
+  public Path getLinkTarget(final Path f) throws IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), false); // do not follow mount link
+    return res.targetFileSystem.getLinkTarget(res.remainingPath);
+  }
+
+  @Override
+  public void setOwner(final Path f, final String username,
+      final String groupname) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
+  }
+
+  @Override
+  public void setPermission(final Path f, final FsPermission permission)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setPermission(res.remainingPath, permission); 
+    
+  }
+
+  @Override
+  public boolean setReplication(final Path f, final short replication)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    return res.targetFileSystem.setReplication(res.remainingPath, replication);
+  }
+
+  @Override
+  public void setTimes(final Path f, final long mtime, final long atime)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    InodeTree.ResolveResult<AbstractFileSystem> res = 
+      fsState.resolve(getUriPath(f), true);
+    res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
+  }
+
+  @Override
+  public void setVerifyChecksum(final boolean verifyChecksum)
+      throws AccessControlException, IOException {
+    // This is a file system level operations, however ViewFs 
+    // points to many file systems. Noop for ViewFs. 
+  }
+  
+  /*
+   * An instance of this class represents an internal dir of the viewFs 
+   * ie internal dir of the mount table.
+   * It is a ready only mount tbale and create, mkdir or delete operations
+   * are not allowed.
+   * If called on create or mkdir then this target is the parent of the
+   * directory in which one is trying to create or mkdir; hence
+   * in this case the path name passed in is the last component. 
+   * Otherwise this target is the end point of the path and hence
+   * the path name passed in is null. 
+   */
+  static class InternalDirOfViewFs extends AbstractFileSystem {
+    
+    final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
+    final long creationTime; // of the the mount table
+    final UserGroupInformation ugi; // the user/group of user who created mtable
+    
+    public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
+        final long cTime, final UserGroupInformation ugi)
+      throws URISyntaxException {
+      super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
+      theInternalDir = dir;
+      creationTime = cTime;
+      this.ugi = ugi;
+    }
+
+    static private void checkPathIsSlash(final Path f) throws IOException {
+      if (f != InodeTree.SlashPath) {
+        throw new IOException (
+        "Internal implementation error: expected file name to be /" );
+      }
+    }
+
+    @Override
+    public FSDataOutputStream createInternal(final Path f,
+        final EnumSet<CreateFlag> flag, final FsPermission absolutePermission,
+        final int bufferSize, final short replication, final long blockSize,
+        final Progressable progress, final int bytesPerChecksum,
+        final boolean createParent) throws AccessControlException,
+        FileAlreadyExistsException, FileNotFoundException,
+        ParentNotDirectoryException, UnsupportedFileSystemException,
+        UnresolvedLinkException, IOException {
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public boolean delete(final Path f, final boolean recursive)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public BlockLocation[] getFileBlockLocations(final Path f, final long start,
+        final long len) throws FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public FileChecksum getFileChecksum(final Path f)
+        throws FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public FileStatus getFileStatus(final Path f) throws IOException {
+      checkPathIsSlash(f);
+      return new FileStatus(0, true, 0, 0, creationTime, creationTime, 
+          PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+          new Path(theInternalDir.fullPath).makeQualified(
+              FsConstants.VIEWFS_URI, null));
+    }
+    
+    @Override
+    public FileStatus getFileLinkStatus(final Path f)
+        throws FileNotFoundException {
+      // look up i internalDirs children - ignore first Slash
+      INode<AbstractFileSystem> inode =
+        theInternalDir.children.get(f.toUri().toString().substring(1)); 
+      if (inode == null) {
+        throw new FileNotFoundException(
+            "viewFs internal mount table - missing entry:" + f);
+      }
+      FileStatus result;
+      if (inode instanceof INodeLink) {
+        INodeLink<AbstractFileSystem> inodelink = 
+          (INodeLink<AbstractFileSystem>) inode;
+        result = new FileStatus(0, false, 0, 0, creationTime, creationTime,
+            PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+            inodelink.getTargetLink(),
+            new Path(inode.fullPath).makeQualified(
+                FsConstants.VIEWFS_URI, null));
+      } else {
+        result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
+          PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+          new Path(inode.fullPath).makeQualified(
+              FsConstants.VIEWFS_URI, null));
+      }
+      return result;
+    }
+    
+    @Override
+    public FsStatus getFsStatus() {
+      return new FsStatus(0, 0, 0);
+    }
+
+    @Override
+    public FsServerDefaults getServerDefaults() throws IOException {
+      throw new IOException("FsServerDefaults not implemented yet");
+    }
+
+    @Override
+    public int getUriDefaultPort() {
+      return -1;
+    }
+
+    @Override
+    public FileStatus[] listStatus(final Path f) throws AccessControlException,
+        IOException {
+      checkPathIsSlash(f);
+      FileStatus[] result = new FileStatus[theInternalDir.children.size()];
+      int i = 0;
+      for (Entry<String, INode<AbstractFileSystem>> iEntry : 
+                                          theInternalDir.children.entrySet()) {
+        INode<AbstractFileSystem> inode = iEntry.getValue();
+
+        
+        if (inode instanceof INodeLink ) {
+          INodeLink<AbstractFileSystem> link = 
+            (INodeLink<AbstractFileSystem>) inode;
+
+          result[i++] = new FileStatus(0, false, 0, 0,
+            creationTime, creationTime,
+            PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+            link.getTargetLink(),
+            new Path(inode.fullPath).makeQualified(
+                FsConstants.VIEWFS_URI, null));
+        } else {
+          result[i++] = new FileStatus(0, true, 0, 0,
+            creationTime, creationTime,
+            PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
+            new Path(inode.fullPath).makeQualified(
+                FsConstants.VIEWFS_URI, null));
+        }
+      }
+      return result;
+    }
+
+    @Override
+    public void mkdir(final Path dir, final FsPermission permission,
+        final boolean createParent) throws AccessControlException,
+        FileAlreadyExistsException {
+      if (theInternalDir.isRoot & dir == null) {
+        throw new FileAlreadyExistsException("/ already exits");
+      }
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public FSDataInputStream open(final Path f, final int bufferSize)
+        throws FileNotFoundException, IOException {
+      checkPathIsSlash(f);
+      throw new FileNotFoundException("Path points to dir not a file");
+    }
+
+    @Override
+    public void renameInternal(final Path src, final Path dst)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(src);
+      checkPathIsSlash(dst);
+      throw READONLY_MOUNTABLE;     
+    }
+
+    @Override
+    public boolean supportsSymlinks() {
+      return true;
+    }
+    
+    @Override
+    public void createSymlink(final Path target, final Path link,
+        final boolean createParent) throws AccessControlException {
+      throw READONLY_MOUNTABLE;    
+    }
+
+    @Override
+    public Path getLinkTarget(final Path f) throws FileNotFoundException,
+        IOException {
+      return getFileLinkStatus(f).getSymlink();
+    }
+
+    @Override
+    public void setOwner(final Path f, final String username,
+        final String groupname) throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public void setPermission(final Path f, final FsPermission permission)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;    
+    }
+
+    @Override
+    public boolean setReplication(final Path f, final short replication)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;
+    }
+
+    @Override
+    public void setTimes(final Path f, final long mtime, final long atime)
+        throws AccessControlException, IOException {
+      checkPathIsSlash(f);
+      throw READONLY_MOUNTABLE;    
+    }
+
+    @Override
+    public void setVerifyChecksum(final boolean verifyChecksum)
+        throws AccessControlException {
+      throw READONLY_MOUNTABLE;   
+    }
+  }
+}

+ 98 - 0
src/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java

@@ -0,0 +1,98 @@
+package org.apache.hadoop.fs.viewfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+
+/**
+ * This class is needed to address the  problem described in
+ * {@link ViewFileSystem#getFileStatus(org.apache.hadoop.fs.Path)} and
+ * {@link ViewFs#getFileStatus(org.apache.hadoop.fs.Path)}
+ */
+class ViewFsFileStatus extends FileStatus {
+   final FileStatus myFs;
+   Path modifiedPath;
+   ViewFsFileStatus(FileStatus fs, Path newPath) {
+     myFs = fs;
+     modifiedPath = newPath;
+   }
+   
+   @Override
+   public long getLen() {
+     return myFs.getLen();
+   }
+
+   @Override
+   public boolean isFile() {
+     return myFs.isFile();
+   }
+
+   @Override
+   public boolean isDirectory() {
+     return  myFs.isDirectory();
+   }
+   
+   @Override
+   public boolean isDir() {
+     return myFs.isDirectory();
+   }
+   
+   @Override
+   public boolean isSymlink() {
+     return myFs.isSymlink();
+   }
+
+   @Override
+   public long getBlockSize() {
+     return myFs.getBlockSize();
+   }
+
+   @Override
+   public short getReplication() {
+     return myFs.getReplication();
+   }
+
+   @Override
+   public long getModificationTime() {
+     return myFs.getModificationTime();
+   }
+
+   @Override
+   public long getAccessTime() {
+     return myFs.getAccessTime();
+   }
+
+   @Override
+   public FsPermission getPermission() {
+     return myFs.getPermission();
+   }
+   
+   @Override
+   public String getOwner() {
+     return myFs.getOwner();
+   }
+   
+   @Override
+   public String getGroup() {
+     return myFs.getGroup();
+   }
+   
+   @Override
+   public Path getPath() {
+     return modifiedPath;
+   }
+   
+   @Override
+   public void setPath(final Path p) {
+     modifiedPath = p;
+   }
+
+   @Override
+   public Path getSymlink() throws IOException {
+     return myFs.getSymlink();
+   }
+}
+

+ 0 - 1
src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java

@@ -90,7 +90,6 @@ public abstract class FSMainOperationsBaseTest  {
   @After
   public void tearDown() throws Exception {
     fSys.delete(new Path(getAbsoluteTestRootPath(fSys), new Path("test")), true);
-    fSys.delete(new Path(LOCAL_FS_ROOT_URI), true);
   }
   
   

+ 291 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java

@@ -0,0 +1,291 @@
+/**
+ * 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.viewfs;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ChRootedFileSystem;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestChRootedFileSystem {
+  FileSystem fSys; // The ChRoootedFs
+  FileSystem fSysTarget; //
+  Path chrootedTo;
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    Configuration conf = new Configuration();
+    fSysTarget = FileSystem.getLocal(conf);
+    chrootedTo = FileSystemTestHelper.getAbsoluteTestRootPath(fSysTarget);
+    // In case previous test was killed before cleanup
+    fSysTarget.delete(chrootedTo, true);
+    
+    fSysTarget.mkdirs(chrootedTo);
+
+
+    // ChRoot to the root of the testDirectory
+    fSys = new ChRootedFileSystem(fSysTarget, chrootedTo);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fSysTarget.delete(chrootedTo, true);
+  }
+  
+  @Test
+  public void testURI() {
+    URI uri = fSys.getUri();
+    Assert.assertEquals(chrootedTo.toUri(), uri);
+  }
+  
+  @Test
+  public void testBasicPaths() {
+    URI uri = fSys.getUri();
+    Assert.assertEquals(chrootedTo.toUri(), uri);
+    Assert.assertEquals(fSys.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fSys.getWorkingDirectory());
+    Assert.assertEquals(fSys.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fSys.getHomeDirectory());
+    /*
+     * ChRootedFs as its uri like file:///chrootRoot.
+     * This is questionable since path.makequalified(uri, path) ignores
+     * the pathPart of a uri. So our notion of chrooted URI is questionable.
+     * But if we were to fix Path#makeQualified() then  the next test should
+     *  have been:
+
+    Assert.assertEquals(
+        new Path(chrootedTo + "/foo/bar").makeQualified(
+            FsConstants.LOCAL_FS_URI, null),
+        fSys.makeQualified(new Path( "/foo/bar")));
+    */
+    
+    Assert.assertEquals(
+        new Path("/foo/bar").makeQualified(FsConstants.LOCAL_FS_URI, null),
+        fSys.makeQualified(new Path("/foo/bar")));
+  }
+  
+  /** 
+   * Test modify operations (create, mkdir, delete, etc) 
+   * 
+   * Verify the operation via chrootedfs (ie fSys) and *also* via the
+   *  target file system (ie fSysTarget) that has been chrooted.
+   */
+  @Test
+  public void testCreateDelete() throws IOException {
+    
+
+    // Create file 
+    FileSystemTestHelper.createFile(fSys, "/foo");
+    Assert.assertTrue(fSys.isFile(new Path("/foo")));
+    Assert.assertTrue(fSysTarget.isFile(new Path(chrootedTo, "foo")));
+    
+    // Create file with recursive dir
+    FileSystemTestHelper.createFile(fSys, "/newDir/foo");
+    Assert.assertTrue(fSys.isFile(new Path("/newDir/foo")));
+    Assert.assertTrue(fSysTarget.isFile(new Path(chrootedTo,"newDir/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fSys.delete(new Path("/newDir/foo"), false));
+    Assert.assertFalse(fSys.exists(new Path("/newDir/foo")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo, "newDir/foo")));
+    
+    // Create file with a 2 component dirs recursively
+    FileSystemTestHelper.createFile(fSys, "/newDir/newDir2/foo");
+    Assert.assertTrue(fSys.isFile(new Path("/newDir/newDir2/foo")));
+    Assert.assertTrue(fSysTarget.isFile(new Path(chrootedTo,"newDir/newDir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fSys.delete(new Path("/newDir/newDir2/foo"), false));
+    Assert.assertFalse(fSys.exists(new Path("/newDir/newDir2/foo")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo,"newDir/newDir2/foo")));
+  }
+  
+  
+  @Test
+  public void testMkdirDelete() throws IOException {
+    fSys.mkdirs(FileSystemTestHelper.getTestRootPath(fSys, "/dirX"));
+    Assert.assertTrue(fSys.isDirectory(new Path("/dirX")));
+    Assert.assertTrue(fSysTarget.isDirectory(new Path(chrootedTo,"dirX")));
+    
+    fSys.mkdirs(FileSystemTestHelper.getTestRootPath(fSys, "/dirX/dirY"));
+    Assert.assertTrue(fSys.isDirectory(new Path("/dirX/dirY")));
+    Assert.assertTrue(fSysTarget.isDirectory(new Path(chrootedTo,"dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue(fSys.delete(new Path("/dirX/dirY"), false));
+    Assert.assertFalse(fSys.exists(new Path("/dirX/dirY")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo,"dirX/dirY")));
+    
+    Assert.assertTrue(fSys.delete(new Path("/dirX"), false));
+    Assert.assertFalse(fSys.exists(new Path("/dirX")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo,"dirX")));
+    
+  }
+  @Test
+  public void testRename() throws IOException {
+    // Rename a file
+    FileSystemTestHelper.createFile(fSys, "/newDir/foo");
+    fSys.rename(new Path("/newDir/foo"), new Path("/newDir/fooBar"));
+    Assert.assertFalse(fSys.exists(new Path("/newDir/foo")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo,"newDir/foo")));
+    Assert.assertTrue(fSys.isFile(FileSystemTestHelper.getTestRootPath(fSys,"/newDir/fooBar")));
+    Assert.assertTrue(fSysTarget.isFile(new Path(chrootedTo,"newDir/fooBar")));
+    
+    
+    // Rename a dir
+    fSys.mkdirs(new Path("/newDir/dirFoo"));
+    fSys.rename(new Path("/newDir/dirFoo"), new Path("/newDir/dirFooBar"));
+    Assert.assertFalse(fSys.exists(new Path("/newDir/dirFoo")));
+    Assert.assertFalse(fSysTarget.exists(new Path(chrootedTo,"newDir/dirFoo")));
+    Assert.assertTrue(fSys.isDirectory(FileSystemTestHelper.getTestRootPath(fSys,"/newDir/dirFooBar")));
+    Assert.assertTrue(fSysTarget.isDirectory(new Path(chrootedTo,"newDir/dirFooBar")));
+  }
+  
+  
+  /**
+   * We would have liked renames across file system to fail but 
+   * Unfortunately there is not way to distinguish the two file systems 
+   * @throws IOException
+   */
+  @Test
+  public void testRenameAcrossFs() throws IOException {
+    fSys.mkdirs(new Path("/newDir/dirFoo"));
+    fSys.rename(new Path("/newDir/dirFoo"), new Path("file:///tmp/dirFooBar"));
+    FileSystemTestHelper.isDir(fSys, new Path("/tmp/dirFooBar"));
+  }
+ 
+  
+  
+  
+  @Test
+  public void testList() throws IOException {
+    
+    FileStatus fs = fSys.getFileStatus(new Path("/"));
+    Assert.assertTrue(fs.isDirectory());
+    //  should return the full path not the chrooted path
+    Assert.assertEquals(fs.getPath(), chrootedTo);
+    
+    // list on Slash
+    
+    FileStatus[] dirPaths = fSys.listStatus(new Path("/"));
+
+    Assert.assertEquals(0, dirPaths.length);
+    
+    
+
+    FileSystemTestHelper.createFile(fSys, "/foo");
+    FileSystemTestHelper.createFile(fSys, "/bar");
+    fSys.mkdirs(new Path("/dirX"));
+    fSys.mkdirs(FileSystemTestHelper.getTestRootPath(fSys, "/dirY"));
+    fSys.mkdirs(new Path("/dirX/dirXX"));
+    
+    dirPaths = fSys.listStatus(new Path("/"));
+    Assert.assertEquals(4, dirPaths.length); // note 2 crc files
+    
+    // Note the the file status paths are the full paths on target
+    fs = FileSystemTestHelper.containsPath(new Path(chrootedTo, "foo"), dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isFile());
+    fs = FileSystemTestHelper.containsPath(new Path(chrootedTo, "bar"), dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isFile());
+    fs = FileSystemTestHelper.containsPath(new Path(chrootedTo, "dirX"), dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+    fs = FileSystemTestHelper.containsPath(new Path(chrootedTo, "dirY"), dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+  }
+  
+  @Test
+  public void testWorkingDirectory() throws Exception {
+
+    // First we cd to our test root
+    fSys.mkdirs(new Path("/testWd"));
+    Path workDir = new Path("/testWd");
+    fSys.setWorkingDirectory(workDir);
+    Assert.assertEquals(workDir, fSys.getWorkingDirectory());
+
+    fSys.setWorkingDirectory(new Path("."));
+    Assert.assertEquals(workDir, fSys.getWorkingDirectory());
+
+    fSys.setWorkingDirectory(new Path(".."));
+    Assert.assertEquals(workDir.getParent(), fSys.getWorkingDirectory());
+    
+    // cd using a relative path
+
+    // Go back to our test root
+    workDir = new Path("/testWd");
+    fSys.setWorkingDirectory(workDir);
+    Assert.assertEquals(workDir, fSys.getWorkingDirectory());
+    
+    Path relativeDir = new Path("existingDir1");
+    Path absoluteDir = new Path(workDir,"existingDir1");
+    fSys.mkdirs(absoluteDir);
+    fSys.setWorkingDirectory(relativeDir);
+    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
+    // cd using a absolute path
+    absoluteDir = new Path("/test/existingDir2");
+    fSys.mkdirs(absoluteDir);
+    fSys.setWorkingDirectory(absoluteDir);
+    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
+    
+    // Now open a file relative to the wd we just set above.
+    Path absoluteFooPath = new Path(absoluteDir, "foo");
+    fSys.create(absoluteFooPath).close();
+    fSys.open(new Path("foo")).close();
+    
+    // Now mkdir relative to the dir we cd'ed to
+    fSys.mkdirs(new Path("newDir"));
+    Assert.assertTrue(fSys.isDirectory(new Path(absoluteDir, "newDir")));
+
+    /* Filesystem impls (RawLocal and DistributedFileSystem do not check
+     * for existing of working dir
+    absoluteDir = getTestRootPath(fSys, "nonexistingPath");
+    try {
+      fSys.setWorkingDirectory(absoluteDir);
+      Assert.fail("cd to non existing dir should have failed");
+    } catch (Exception e) {
+      // Exception as expected
+    }
+    */
+    
+    // Try a URI
+    final String LOCAL_FS_ROOT_URI = "file:///tmp/test";
+    absoluteDir = new Path(LOCAL_FS_ROOT_URI + "/existingDir");
+    fSys.mkdirs(absoluteDir);
+    fSys.setWorkingDirectory(absoluteDir);
+    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
+
+  }
+ 
+}

+ 292 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestChRootedFs.java

@@ -0,0 +1,292 @@
+/**
+ * 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.viewfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+
+import static org.apache.hadoop.fs.FileContextTestHelper.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ChRootedFs;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestChRootedFs {
+  FileContext fc; // The ChRoootedFs
+  FileContext fcTarget; // 
+  Path chrootedTo;
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    fcTarget = FileContext.getLocalFSFileContext();
+    chrootedTo = FileContextTestHelper.getAbsoluteTestRootPath(fcTarget);
+    // In case previous test was killed before cleanup
+    fcTarget.delete(chrootedTo, true);
+    
+    fcTarget.mkdir(chrootedTo, FileContext.DEFAULT_PERM, true);
+
+    Configuration conf = new Configuration();
+
+    // ChRoot to the root of the testDirectory
+    fc = FileContext.getFileContext(
+        new ChRootedFs(fcTarget.getDefaultFileSystem(), chrootedTo), conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fcTarget.delete(chrootedTo, true);
+  }
+
+  
+  @Test
+  public void testBasicPaths() {
+    URI uri = fc.getDefaultFileSystem().getUri();
+    Assert.assertEquals(chrootedTo.toUri(), uri);
+    Assert.assertEquals(fc.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fc.getWorkingDirectory());
+    Assert.assertEquals(fc.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fc.getHomeDirectory());
+    /*
+     * ChRootedFs as its uri like file:///chrootRoot.
+     * This is questionable since path.makequalified(uri, path) ignores
+     * the pathPart of a uri. So our notion of chrooted URI is questionable.
+     * But if we were to fix Path#makeQualified() then  the next test should
+     *  have been:
+
+    Assert.assertEquals(
+        new Path(chrootedTo + "/foo/bar").makeQualified(
+            FsConstants.LOCAL_FS_URI, null),
+        fc.makeQualified(new Path( "/foo/bar")));
+    */
+    
+    Assert.assertEquals(
+        new Path("/foo/bar").makeQualified(FsConstants.LOCAL_FS_URI, null),
+        fc.makeQualified(new Path("/foo/bar")));
+  }
+  
+  
+  /** 
+   * Test modify operations (create, mkdir, delete, etc) 
+   * 
+   * Verify the operation via chrootedfs (ie fc) and *also* via the
+   *  target file system (ie fclocal) that has been chrooted.
+   */
+  @Test
+  public void testCreateDelete() throws IOException {
+    
+
+    // Create file 
+    FileContextTestHelper.createFileNonRecursive(fc, "/foo");
+    Assert.assertTrue(isFile(fc, new Path("/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(chrootedTo, "foo")));
+    
+    // Create file with recursive dir
+    FileContextTestHelper.createFile(fc, "/newDir/foo");
+    Assert.assertTrue(isFile(fc, new Path("/newDir/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(chrootedTo,"newDir/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fc.delete(new Path("/newDir/foo"), false));
+    Assert.assertFalse(exists(fc, new Path("/newDir/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"newDir/foo")));
+    
+    // Create file with a 2 component dirs recursively
+    FileContextTestHelper.createFile(fc, "/newDir/newDir2/foo");
+    Assert.assertTrue(isFile(fc, new Path("/newDir/newDir2/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(chrootedTo,"newDir/newDir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fc.delete(new Path("/newDir/newDir2/foo"), false));
+    Assert.assertFalse(exists(fc, new Path("/newDir/newDir2/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"newDir/newDir2/foo")));
+  }
+  
+  
+  @Test
+  public void testMkdirDelete() throws IOException {
+    fc.mkdir(FileContextTestHelper.getTestRootPath(fc, "/dirX"), FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue(isDir(fc, new Path("/dirX")));
+    Assert.assertTrue(isDir(fcTarget, new Path(chrootedTo,"dirX")));
+    
+    fc.mkdir(FileContextTestHelper.getTestRootPath(fc, "/dirX/dirY"), FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue(isDir(fc, new Path("/dirX/dirY")));
+    Assert.assertTrue(isDir(fcTarget, new Path(chrootedTo,"dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue(fc.delete(new Path("/dirX/dirY"), false));
+    Assert.assertFalse(exists(fc, new Path("/dirX/dirY")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"dirX/dirY")));
+    
+    Assert.assertTrue(fc.delete(new Path("/dirX"), false));
+    Assert.assertFalse(exists(fc, new Path("/dirX")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"dirX")));
+    
+  }
+  @Test
+  public void testRename() throws IOException {
+    // Rename a file
+    FileContextTestHelper.createFile(fc, "/newDir/foo");
+    fc.rename(new Path("/newDir/foo"), new Path("/newDir/fooBar"));
+    Assert.assertFalse(exists(fc, new Path("/newDir/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"newDir/foo")));
+    Assert.assertTrue(isFile(fc, FileContextTestHelper.getTestRootPath(fc,"/newDir/fooBar")));
+    Assert.assertTrue(isFile(fcTarget, new Path(chrootedTo,"newDir/fooBar")));
+    
+    
+    // Rename a dir
+    fc.mkdir(new Path("/newDir/dirFoo"), FileContext.DEFAULT_PERM, false);
+    fc.rename(new Path("/newDir/dirFoo"), new Path("/newDir/dirFooBar"));
+    Assert.assertFalse(exists(fc, new Path("/newDir/dirFoo")));
+    Assert.assertFalse(exists(fcTarget, new Path(chrootedTo,"newDir/dirFoo")));
+    Assert.assertTrue(isDir(fc, FileContextTestHelper.getTestRootPath(fc,"/newDir/dirFooBar")));
+    Assert.assertTrue(isDir(fcTarget, new Path(chrootedTo,"newDir/dirFooBar")));
+  }
+  
+  
+  /**
+   * We would have liked renames across file system to fail but 
+   * Unfortunately there is not way to distinguish the two file systems 
+   * @throws IOException
+   */
+  @Test
+  public void testRenameAcrossFs() throws IOException {
+    fc.mkdir(new Path("/newDir/dirFoo"), FileContext.DEFAULT_PERM, true);
+    // the root will get interpreted to the root of the chrooted fs.
+    fc.rename(new Path("/newDir/dirFoo"), new Path("file:///dirFooBar"));
+    FileContextTestHelper.isDir(fc, new Path("/dirFooBar"));
+  }
+  
+  @Test
+  public void testList() throws IOException {
+    
+    FileStatus fs = fc.getFileStatus(new Path("/"));
+    Assert.assertTrue(fs.isDirectory());
+    //  should return the full path not the chrooted path
+    Assert.assertEquals(fs.getPath(), chrootedTo);
+    
+    // list on Slash
+    
+    FileStatus[] dirPaths = fc.util().listStatus(new Path("/"));
+
+    Assert.assertEquals(0, dirPaths.length);
+    
+    
+
+    FileContextTestHelper.createFileNonRecursive(fc, "/foo");
+    FileContextTestHelper.createFileNonRecursive(fc, "/bar");
+    fc.mkdir(new Path("/dirX"), FileContext.DEFAULT_PERM, false);
+    fc.mkdir(FileContextTestHelper.getTestRootPath(fc, "/dirY"),
+        FileContext.DEFAULT_PERM, false);
+    fc.mkdir(new Path("/dirX/dirXX"), FileContext.DEFAULT_PERM, false);
+    
+    dirPaths = fc.util().listStatus(new Path("/"));
+    Assert.assertEquals(4, dirPaths.length);
+    
+    // Note the the file status paths are the full paths on target
+    fs = FileContextTestHelper.containsPath(fcTarget, "foo", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isFile());
+    fs = FileContextTestHelper.containsPath(fcTarget, "bar", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isFile());
+    fs = FileContextTestHelper.containsPath(fcTarget, "dirX", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+    fs = FileContextTestHelper.containsPath(fcTarget, "dirY", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+  }
+  
+  @Test
+  public void testWorkingDirectory() throws Exception {
+
+    // First we cd to our test root
+    fc.mkdir(new Path("/testWd"), FileContext.DEFAULT_PERM, false);
+    Path workDir = new Path("/testWd");
+    Path fqWd = fc.makeQualified(workDir);
+    fc.setWorkingDirectory(workDir);
+    Assert.assertEquals(fqWd, fc.getWorkingDirectory());
+
+    fc.setWorkingDirectory(new Path("."));
+    Assert.assertEquals(fqWd, fc.getWorkingDirectory());
+
+    fc.setWorkingDirectory(new Path(".."));
+    Assert.assertEquals(fqWd.getParent(), fc.getWorkingDirectory());
+    
+    // cd using a relative path
+
+    // Go back to our test root
+    workDir = new Path("/testWd");
+    fqWd = fc.makeQualified(workDir);
+    fc.setWorkingDirectory(workDir);
+    Assert.assertEquals(fqWd, fc.getWorkingDirectory());
+    
+    Path relativeDir = new Path("existingDir1");
+    Path absoluteDir = new Path(workDir,"existingDir1");
+    fc.mkdir(absoluteDir, FileContext.DEFAULT_PERM, true);
+    Path fqAbsoluteDir = fc.makeQualified(absoluteDir);
+    fc.setWorkingDirectory(relativeDir);
+    Assert.assertEquals(fqAbsoluteDir, fc.getWorkingDirectory());
+    // cd using a absolute path
+    absoluteDir = new Path("/test/existingDir2");
+    fqAbsoluteDir = fc.makeQualified(absoluteDir);
+    fc.mkdir(absoluteDir, FileContext.DEFAULT_PERM, true);
+    fc.setWorkingDirectory(absoluteDir);
+    Assert.assertEquals(fqAbsoluteDir, fc.getWorkingDirectory());
+    
+    // Now open a file relative to the wd we just set above.
+    Path absolutePath = new Path(absoluteDir, "foo");
+    fc.create(absolutePath, EnumSet.of(CreateFlag.CREATE)).close();
+    fc.open(new Path("foo")).close();
+    
+    // Now mkdir relative to the dir we cd'ed to
+    fc.mkdir(new Path("newDir"), FileContext.DEFAULT_PERM, true);
+    Assert.assertTrue(isDir(fc, new Path(absoluteDir, "newDir")));
+
+    absoluteDir = getTestRootPath(fc, "nonexistingPath");
+    try {
+      fc.setWorkingDirectory(absoluteDir);
+      Assert.fail("cd to non existing dir should have failed");
+    } catch (Exception e) {
+      // Exception as expected
+    }
+    
+    // Try a URI
+    final String LOCAL_FS_ROOT_URI = "file:///tmp/test";
+    absoluteDir = new Path(LOCAL_FS_ROOT_URI + "/existingDir");
+    fc.mkdir(absoluteDir, FileContext.DEFAULT_PERM, true);
+    fc.setWorkingDirectory(absoluteDir);
+    Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());
+
+  }
+ 
+}

+ 60 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestFSMainOperationsLocalFileSystem.java

@@ -0,0 +1,60 @@
+/**
+ * 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.viewfs;
+
+
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTest {
+   static FileSystem fcTarget;
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fcTarget = FileSystem.getLocal(conf);
+    fSys = ViewFileSystemTestSetup.setupForViewFs(fcTarget);
+    super.setUp();
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    ViewFileSystemTestSetup.tearDownForViewFs(fcTarget);
+  }
+  
+  @Test
+  @Override
+  public void testWDAbsolute() throws IOException {
+    Path absoluteDir = FileSystemTestHelper.getTestRootPath(fSys,
+        "test/existingDir");
+    fSys.mkdirs(absoluteDir);
+    fSys.setWorkingDirectory(absoluteDir);
+    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
+
+  }
+}

+ 42 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestFcCreateMkdirLocalFs.java

@@ -0,0 +1,42 @@
+/**
+ * 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.viewfs;
+
+
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+
+import org.junit.After;
+import org.junit.Before;
+
+
+public class TestFcCreateMkdirLocalFs  extends 
+  FileContextCreateMkdirBaseTest {
+
+
+  @Before
+  public void setUp() throws Exception {
+    fc = ViewFsTestSetup.setupForViewFsLocalFs();
+    super.setUp();
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    ViewFsTestSetup.tearDownForViewFsLocalFs();
+  }
+}

+ 87 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestFcMainOperationsLocalFs.java

@@ -0,0 +1,87 @@
+/**
+ * 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.viewfs;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+
+import org.junit.After;
+import org.junit.Before;
+
+
+public class TestFcMainOperationsLocalFs  extends 
+  FileContextMainOperationsBaseTest {
+
+  FileContext fclocal;
+  Path targetOfTests;
+
+  @Before
+  public void setUp() throws Exception {
+    /**
+     * create the test root on local_fs - the  mount table will point here
+     */
+    fclocal = FileContext.getLocalFSFileContext();
+    targetOfTests = FileContextTestHelper.getTestRootPath(fclocal);
+    // In case previous test was killed before cleanup
+    fclocal.delete(targetOfTests, true);
+    
+    fclocal.mkdir(targetOfTests, FileContext.DEFAULT_PERM, true);
+
+    
+    
+    
+    // We create mount table so that the test root on the viewFs points to 
+    // to the test root on the target.
+    // DOing this helps verify the FileStatus.path.
+    //
+    // The test root by default when running eclipse 
+    // is a test dir below the working directory. 
+    // (see FileContextTestHelper).
+    // Since viewFs has no built-in wd, its wd is /user/<username>.
+    // If this test launched via ant (build.xml) the test root is absolute path
+    
+    String srcTestRoot;
+    if (FileContextTestHelper.TEST_ROOT_DIR.startsWith("/")) {
+      srcTestRoot = FileContextTestHelper.TEST_ROOT_DIR;
+    } else {
+      srcTestRoot = "/user/"  + System.getProperty("user.name") + "/" +
+      FileContextTestHelper.TEST_ROOT_DIR;
+    }
+
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, srcTestRoot,
+        targetOfTests.toUri());
+    
+    fc = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
+    //System.out.println("SRCOfTests = "+ FileContextTestHelper.getTestRootPath(fc, "test"));
+    //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
+    super.setUp();
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    fclocal.delete(targetOfTests, true);
+  }
+}

+ 41 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestFcPermissionsLocalFs.java

@@ -0,0 +1,41 @@
+/**
+ * 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.viewfs;
+
+
+import org.apache.hadoop.fs.FileContextPermissionBase;
+
+import org.junit.After;
+import org.junit.Before;
+
+
+public class TestFcPermissionsLocalFs  extends FileContextPermissionBase {
+
+
+  @Before
+  public void setUp() throws Exception {
+    fc = ViewFsTestSetup.setupForViewFsLocalFs();
+    super.setUp();
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    ViewFsTestSetup.tearDownForViewFsLocalFs();
+  }
+}

+ 45 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java

@@ -0,0 +1,45 @@
+/**
+ * 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.viewfs;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+
+import org.junit.After;
+import org.junit.Before;
+
+
+public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
+
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    fsTarget = FileSystem.getLocal(new Configuration());
+    super.setUp();
+    
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(FileSystemTestHelper.getTestRootPath(fsTarget), true);
+    super.tearDown();
+  }
+}

+ 57 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java

@@ -0,0 +1,57 @@
+package org.apache.hadoop.fs.viewfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.fs.viewfs.InodeTree;
+import org.junit.Test;
+
+
+public class TestViewFsConfig {
+  
+  
+  @Test(expected=FileAlreadyExistsException.class)
+  public void testInvalidConfig() throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, "/internalDir/linkToDir2",
+        new Path("file:///dir2").toUri());
+    ConfigUtil.addLink(conf, "/internalDir/linkToDir2/linkToDir3",
+        new Path("file:///dir3").toUri());
+    
+    class Foo { };
+    
+     new InodeTree<Foo>(conf, null) {
+
+      @Override
+      protected
+      Foo getTargetFileSystem(final URI uri)
+        throws URISyntaxException, UnsupportedFileSystemException {
+          return null;
+      }
+
+      @Override
+      protected
+      Foo getTargetFileSystem(
+          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo>
+                                          dir)
+        throws URISyntaxException {
+        return null;
+      }
+
+      @Override
+      protected
+      Foo getTargetFileSystem(URI[] mergeFsURIList)
+          throws URISyntaxException, UnsupportedFileSystemException {
+        return null;
+      }
+    };
+  }
+
+}

+ 43 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsLocalFs.java

@@ -0,0 +1,43 @@
+/**
+ * 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.viewfs;
+
+
+import org.apache.hadoop.fs.FileContext;
+
+import org.junit.After;
+import org.junit.Before;
+
+
+public class TestViewFsLocalFs extends ViewFsBaseTest {
+
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    fcTarget = FileContext.getLocalFSFileContext();
+    super.setUp();
+    
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+ 
+}

+ 479 - 0
src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java

@@ -0,0 +1,479 @@
+/**
+ * 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.viewfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import static org.apache.hadoop.fs.FileSystemTestHelper.*;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+/**
+ * <p>
+ * A collection of tests for the {@link ViewFileSystem}.
+ * This test should be used for testing ViewFileSystem that has mount links to 
+ * a target file system such  localFs or Hdfs etc.
+
+ * </p>
+ * <p>
+ * To test a given target file system create a subclass of this
+ * test and override {@link #setUp()} to initialize the <code>fsTarget</code> 
+ * to point to the file system to which you want the mount targets
+ * 
+ * Since this a junit 4 you can also do a single setup before 
+ * the start of any tests.
+ * E.g.
+ *     @BeforeClass   public static void clusterSetupAtBegining()
+ *     @AfterClass    public static void ClusterShutdownAtEnd()
+ * </p>
+ */
+
+public class ViewFileSystemBaseTest {
+  FileSystem fsView;  // the view file system - the mounts are here
+  FileSystem fsTarget;  // the target file system - the mount will point here
+  Path targetTestRoot;
+
+  @Before
+  public void setUp() throws Exception {
+    targetTestRoot = FileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
+    // In case previous test was killed before cleanup
+    fsTarget.delete(targetTestRoot, true);
+    
+    fsTarget.mkdirs(targetTestRoot);
+    // Make  user and data dirs - we creates links to them in the mount table
+    fsTarget.mkdirs(new Path(targetTestRoot,"user"));
+    fsTarget.mkdirs(new Path(targetTestRoot,"data"));
+    fsTarget.mkdirs(new Path(targetTestRoot,"dir2"));
+    fsTarget.mkdirs(new Path(targetTestRoot,"dir3"));
+    FileSystemTestHelper.createFile(fsTarget, new Path(targetTestRoot,"aFile"));
+    
+    
+    // Now we use the mount fs to set links to user and dir
+    // in the test root
+    
+    // Set up the defaultMT in the config with our mount point links
+    //Configuration conf = new Configuration();
+    Configuration conf = ViewFileSystemTestSetup.configWithViewfsScheme();
+    ConfigUtil.addLink(conf, "/user", new Path(targetTestRoot,"user").toUri());
+    ConfigUtil.addLink(conf, "/user2", new Path(targetTestRoot,"user").toUri());
+    ConfigUtil.addLink(conf, "/data", new Path(targetTestRoot,"data").toUri());
+    ConfigUtil.addLink(conf, "/internalDir/linkToDir2",
+        new Path(targetTestRoot,"dir2").toUri());
+    ConfigUtil.addLink(conf, "/internalDir/internalDir2/linkToDir3",
+        new Path(targetTestRoot,"dir3").toUri());
+    ConfigUtil.addLink(conf, "/danglingLink",
+        new Path(targetTestRoot,"missingTarget").toUri());
+    ConfigUtil.addLink(conf, "/linkToAFile",
+        new Path(targetTestRoot,"aFile").toUri());
+    
+    fsView = FileSystem.get(FsConstants.VIEWFS_URI, conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(FileSystemTestHelper.getTestRootPath(fsTarget), true);
+  }
+  
+  
+  @Test
+  public void testBasicPaths() {
+    Assert.assertEquals(FsConstants.VIEWFS_URI,
+        fsView.getUri());
+    Assert.assertEquals(fsView.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fsView.getWorkingDirectory());
+    Assert.assertEquals(fsView.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fsView.getHomeDirectory());
+    Assert.assertEquals(
+        new Path("/foo/bar").makeQualified(FsConstants.VIEWFS_URI, null),
+        fsView.makeQualified(new Path("/foo/bar")));
+  }
+  
+  /** 
+   * Test modify operations (create, mkdir, delete, etc) 
+   * on the mount file system where the pathname references through
+   * the mount points.  Hence these operation will modify the target
+   * file system.
+   * 
+   * Verify the operation via mountfs (ie fSys) and *also* via the
+   *  target file system (ie fSysLocal) that the mount link points-to.
+   */
+  @Test
+  public void testOperationsThroughMountLinks() throws IOException {
+    // Create file 
+    FileSystemTestHelper.createFile(fsView, "/user/foo");
+    Assert.assertTrue(fsView.isFile(new Path("/user/foo")));
+    Assert.assertTrue(fsTarget.isFile(new Path(targetTestRoot,"user/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fsView.delete(new Path("/user/foo"), false));
+    Assert.assertFalse(fsView.exists(new Path("/user/foo")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"user/foo")));
+    
+    // Create file with a 2 component dirs
+    FileSystemTestHelper.createFile(fsView, "/internalDir/linkToDir2/foo");
+    Assert.assertTrue(fsView.isFile(new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertTrue(fsTarget.isFile(new Path(targetTestRoot,"dir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fsView.delete(new Path("/internalDir/linkToDir2/foo"), false));
+    Assert.assertFalse(fsView.exists(new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"dir2/foo")));
+    
+    
+    // Create file with a 3 component dirs
+    FileSystemTestHelper.createFile(fsView, "/internalDir/internalDir2/linkToDir3/foo");
+    Assert.assertTrue(fsView.isFile(new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertTrue(fsTarget.isFile(new Path(targetTestRoot,"dir3/foo")));
+    
+    // Recursive Create file with missing dirs
+    FileSystemTestHelper.createFile(fsView, "/internalDir/linkToDir2/missingDir/miss2/foo");
+    Assert.assertTrue(fsView.isFile(new Path("/internalDir/linkToDir2/missingDir/miss2/foo")));
+    Assert.assertTrue(fsTarget.isFile(new Path(targetTestRoot,"dir2/missingDir/miss2/foo")));
+
+    
+    // Delete the created file
+    Assert.assertTrue(fsView.delete(new Path("/internalDir/internalDir2/linkToDir3/foo"), false));
+    Assert.assertFalse(fsView.exists(new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"dir3/foo")));
+    
+      
+    // mkdir
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX"));
+    Assert.assertTrue(fsView.isDirectory(new Path("/user/dirX")));
+    Assert.assertTrue(fsTarget.isDirectory(new Path(targetTestRoot,"user/dirX")));
+    
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX/dirY"));
+    Assert.assertTrue(fsView.isDirectory(new Path("/user/dirX/dirY")));
+    Assert.assertTrue(fsTarget.isDirectory(new Path(targetTestRoot,"user/dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue(fsView.delete(new Path("/user/dirX/dirY"), false));
+    Assert.assertFalse(fsView.exists(new Path("/user/dirX/dirY")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"user/dirX/dirY")));
+    
+    Assert.assertTrue(fsView.delete(new Path("/user/dirX"), false));
+    Assert.assertFalse(fsView.exists(new Path("/user/dirX")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"user/dirX")));
+    
+    // Rename a file 
+    FileSystemTestHelper.createFile(fsView, "/user/foo");
+    fsView.rename(new Path("/user/foo"), new Path("/user/fooBar"));
+    Assert.assertFalse(fsView.exists(new Path("/user/foo")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"user/foo")));
+    Assert.assertTrue(fsView.isFile(FileSystemTestHelper.getTestRootPath(fsView,"/user/fooBar")));
+    Assert.assertTrue(fsTarget.isFile(new Path(targetTestRoot,"user/fooBar")));
+    
+    fsView.mkdirs(new Path("/user/dirFoo"));
+    fsView.rename(new Path("/user/dirFoo"), new Path("/user/dirFooBar"));
+    Assert.assertFalse(fsView.exists(new Path("/user/dirFoo")));
+    Assert.assertFalse(fsTarget.exists(new Path(targetTestRoot,"user/dirFoo")));
+    Assert.assertTrue(fsView.isDirectory(FileSystemTestHelper.getTestRootPath(fsView,"/user/dirFooBar")));
+    Assert.assertTrue(fsTarget.isDirectory(new Path(targetTestRoot,"user/dirFooBar")));
+    
+  }
+  
+  // rename across mount points that point to same target also fail 
+  @Test(expected=IOException.class) 
+  public void testRenameAcrossMounts1() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/user/foo");
+    fsView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
+    /* - code if we had wanted this to suceed
+    Assert.assertFalse(fSys.exists(new Path("/user/foo")));
+    Assert.assertFalse(fSysLocal.exists(new Path(targetTestRoot,"user/foo")));
+    Assert.assertTrue(fSys.isFile(FileSystemTestHelper.getTestRootPath(fSys,"/user2/fooBarBar")));
+    Assert.assertTrue(fSysLocal.isFile(new Path(targetTestRoot,"user/fooBarBar")));
+    */
+  }
+  
+  
+  // rename across mount points fail if the mount link targets are different
+  // even if the targets are part of the same target FS
+
+  @Test(expected=IOException.class) 
+  public void testRenameAcrossMounts2() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/user/foo");
+    fsView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+  }
+  
+  static protected boolean SupportsBlocks = false; //  local fs use 1 block
+                                                   // override for HDFS
+  @Test
+  public void testGetBlockLocations() throws IOException {
+    Path targetFilePath = new Path(targetTestRoot,"data/largeFile");
+    FileSystemTestHelper.createFile(fsTarget, 
+        targetFilePath, 10, 1024);
+    Path viewFilePath = new Path("/data/largeFile");
+    Assert.assertTrue(fsView.isFile(viewFilePath));
+    BlockLocation[] viewBL = fsView.getFileBlockLocations(fsView.getFileStatus(viewFilePath), 0, 10240+100);
+    Assert.assertEquals(SupportsBlocks ? 10 : 1, viewBL.length);
+    BlockLocation[] targetBL = fsTarget.getFileBlockLocations(fsTarget.getFileStatus(targetFilePath), 0, 10240+100);
+    compareBLs(viewBL, targetBL);
+    
+    
+    // Same test but now get it via the FileStatus Parameter
+    fsView.getFileBlockLocations(
+        fsView.getFileStatus(viewFilePath), 0, 10240+100);
+    targetBL = fsTarget.getFileBlockLocations(
+        fsTarget.getFileStatus(targetFilePath), 0, 10240+100);
+    compareBLs(viewBL, targetBL);  
+  }
+  
+  void compareBLs(BlockLocation[] viewBL, BlockLocation[] targetBL) {
+    Assert.assertEquals(targetBL.length, viewBL.length);
+    int i = 0;
+    for (BlockLocation vbl : viewBL) {
+      Assert.assertEquals(vbl.toString(), targetBL[i].toString());
+      Assert.assertEquals(targetBL[i].getOffset(), vbl.getOffset());
+      Assert.assertEquals(targetBL[i].getLength(), vbl.getLength());
+      i++;     
+    } 
+  }
+  
+  
+  
+  /**
+   * Test "readOps" (e.g. list, listStatus) 
+   * on internal dirs of mount table
+   * These operations should succeed.
+   */
+  
+  // test list on internal dirs of mount table 
+  @Test
+  public void testListOnInternalDirsOfMountTable() throws IOException {
+    
+    // list on Slash
+    
+    FileStatus[] dirPaths = fsView.listStatus(new Path("/"));
+    FileStatus fs;
+    Assert.assertEquals(6, dirPaths.length);
+    fs = FileSystemTestHelper.containsPath(fsView, "/user", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/data", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/internalDir", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+    fs = FileSystemTestHelper.containsPath(fsView, "/danglingLink", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/linkToAFile", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+      
+      
+      
+      // list on internal dir
+      dirPaths = fsView.listStatus(new Path("/internalDir"));
+      Assert.assertEquals(2, dirPaths.length);
+
+      fs = FileSystemTestHelper.containsPath(fsView, "/internalDir/internalDir2", dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue(fs.isDirectory());
+      fs = FileSystemTestHelper.containsPath(fsView, "/internalDir/linkToDir2", dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue(fs.isSymlink());
+  }
+  
+  @Test
+  public void testListOnMountTargetDirs() throws IOException {
+    FileStatus[] dirPaths = fsView.listStatus(new Path("/data"));
+    FileStatus fs;
+    Assert.assertEquals(0, dirPaths.length);
+    
+    // add a file
+    FileSystemTestHelper.createFile(fsView, "/data/foo");
+    dirPaths = fsView.listStatus(new Path("/data"));
+    Assert.assertEquals(1, dirPaths.length);
+    fs = FileSystemTestHelper.containsPath(fsView, "/data/foo", dirPaths);
+    Assert.assertNotNull(fs);
+    Assert.assertTrue(fs.isFile());
+    
+    // add a dir
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/data/dirX"));
+    dirPaths = fsView.listStatus(new Path("/data"));
+    Assert.assertEquals(2, dirPaths.length);
+    fs = FileSystemTestHelper.containsPath(fsView, "/data/foo", dirPaths);
+    Assert.assertNotNull(fs);
+    Assert.assertTrue(fs.isFile());
+    fs = FileSystemTestHelper.containsPath(fsView, "/data/dirX", dirPaths);
+    Assert.assertNotNull(fs);
+    Assert.assertTrue(fs.isDirectory()); 
+  }
+      
+  @Test
+  public void testFileStatusOnMountLink() throws IOException {
+    Assert.assertTrue(fsView.getFileStatus(new Path("/")).isDirectory());
+    checkFileStatus(fsView, "/", fileType.isDir);
+    checkFileStatus(fsView, "/user", fileType.isDir); // link followed => dir
+    checkFileStatus(fsView, "/data", fileType.isDir);
+    checkFileStatus(fsView, "/internalDir", fileType.isDir);
+    checkFileStatus(fsView, "/internalDir/linkToDir2", fileType.isDir);
+    checkFileStatus(fsView, "/internalDir/internalDir2/linkToDir3",
+        fileType.isDir);
+    checkFileStatus(fsView, "/linkToAFile", fileType.isFile);
+  }
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testgetFSonDanglingLink() throws IOException {
+    fsView.getFileStatus(new Path("/danglingLink"));
+  }
+  
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testgetFSonNonExistingInternalDir() throws IOException {
+    fsView.getFileStatus(new Path("/internalDir/nonExisting"));
+  }
+
+  
+  
+  
+  
+  /**
+   * Test modify operations (create, mkdir, rename, etc) 
+   * on internal dirs of mount table
+   * These operations should fail since the mount table is read-only or
+   * because the internal dir that it is trying to create already
+   * exits.
+   */
+ 
+ 
+  // Mkdir on internal mount table should fail
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirSlash() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/"));
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirExisting1() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/internalDir"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirExisting2() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/internalDir/linkToDir2"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirNew() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/dirNew"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirNew2() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/internalDir/dirNew"));
+  }
+  
+  // Create on internal mount table should fail
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreate1() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/foo"); // 1 component
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreate2() throws IOException {  // 2 component
+    FileSystemTestHelper.createFile(fsView, "/internalDir/foo");
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/missingDir/foo");
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir2() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/missingDir/miss2/foo");
+  }
+  
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir3() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/internalDir/miss2/foo");
+  }
+  
+  // Delete on internal mount table should fail
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testInternalDeleteNonExisting() throws IOException {
+      fsView.delete(new Path("/NonExisting"), false);
+  }
+  @Test(expected=FileNotFoundException.class) 
+  public void testInternalDeleteNonExisting2() throws IOException {
+      fsView.delete(new Path("/internalDir/NonExisting"), false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalDeleteExisting() throws IOException {
+      fsView.delete(new Path("/internalDir"), false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalDeleteExisting2() throws IOException {
+    Assert.assertTrue(
+        fsView.getFileStatus(
+            new Path("/internalDir/linkToDir2")).isDirectory());
+    fsView.delete(new Path("/internalDir/linkToDir2"), false);
+  } 
+  
+  
+  // Rename on internal mount table should fail
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename1() throws IOException {
+    fsView.rename(new Path("/internalDir"), new Path("/newDir"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename2() throws IOException {
+    Assert.assertTrue(
+        fsView.getFileStatus(new Path("/internalDir/linkToDir2")).isDirectory());
+    fsView.rename(new Path("/internalDir/linkToDir2"),
+        new Path("/internalDir/dir1"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename3() throws IOException {
+    fsView.rename(new Path("/user"), new Path("/internalDir/linkToDir2"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRenameToSlash() throws IOException {
+    fsView.rename(new Path("/internalDir/linkToDir2/foo"), new Path("/"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRenameFromSlash() throws IOException {
+    fsView.rename(new Path("/"), new Path("/bar"));
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalSetOwner() throws IOException {
+    fsView.setOwner(new Path("/internalDir"), "foo", "bar");
+  }
+}

+ 78 - 0
src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java

@@ -0,0 +1,78 @@
+package org.apache.hadoop.fs.viewfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+
+
+/**
+ * This class is for  setup and teardown for viewFileSystem so that
+ * it can be tested via the standard FileSystem tests.
+ * 
+ * If tests launched via ant (build.xml) the test root is absolute path
+ * If tests launched via eclipse, the test root is 
+ * is a test dir below the working directory. (see FileSystemTestHelper).
+ * Since viewFs has no built-in wd, its wd is /user/<username>.
+ * 
+ * We set a viewFileSystems with mount point for 
+ * /<firstComponent>" pointing to the target fs's  testdir 
+ */
+public class ViewFileSystemTestSetup {
+
+  /**
+   * 
+   * @param fsTarget - the target fs of the view fs.
+   * @return return the ViewFS File context to be used for tests
+   * @throws Exception
+   */
+  static public FileSystem setupForViewFs(FileSystem fsTarget) throws Exception {
+    /**
+     * create the test root on local_fs - the  mount table will point here
+     */
+    Configuration conf = configWithViewfsScheme();
+    Path targetOfTests = FileSystemTestHelper.getTestRootPath(fsTarget);
+    // In case previous test was killed before cleanup
+    fsTarget.delete(targetOfTests, true);
+    
+    fsTarget.mkdirs(targetOfTests);
+  
+    String srcTestFirstDir;
+    if (FileSystemTestHelper.TEST_ROOT_DIR.startsWith("/")) {
+      int indexOf2ndSlash = FileSystemTestHelper.TEST_ROOT_DIR.indexOf('/', 1);
+      srcTestFirstDir = FileSystemTestHelper.TEST_ROOT_DIR.substring(0, indexOf2ndSlash);
+    } else {
+      srcTestFirstDir = "/user"; 
+  
+    }
+    //System.out.println("srcTestFirstDir=" + srcTestFirstDir);
+  
+    // Set up the defaultMT in the config with mount point links
+    // The test dir is root is below  /user/<userid>
+
+    ConfigUtil.addLink(conf, srcTestFirstDir,
+        targetOfTests.toUri());
+    
+    FileSystem fcView = FileSystem.get(FsConstants.VIEWFS_URI, conf);
+    //System.out.println("SRCOfTests = "+ getTestRootPath(fc, "test"));
+    //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
+    return fcView;
+  }
+
+  /**
+   * 
+   * delete the test directory in the target  fs
+   */
+  static public void tearDownForViewFs(FileSystem fsTarget) throws Exception {
+    Path targetOfTests = FileSystemTestHelper.getTestRootPath(fsTarget);
+    fsTarget.delete(targetOfTests, true);
+  }
+  
+  public static Configuration configWithViewfsScheme() {
+    Configuration conf = new Configuration();
+    conf.set("fs.viewfs.impl", ViewFileSystem.class.getName());
+    return conf; 
+  }
+}

+ 528 - 0
src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java

@@ -0,0 +1,528 @@
+/**
+ * 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.viewfs;
+
+import static org.apache.hadoop.fs.FileContextTestHelper.checkFileLinkStatus;
+import static org.apache.hadoop.fs.FileContextTestHelper.checkFileStatus;
+import static org.apache.hadoop.fs.FileContextTestHelper.exists;
+import static org.apache.hadoop.fs.FileContextTestHelper.isDir;
+import static org.apache.hadoop.fs.FileContextTestHelper.isFile;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileContextTestHelper.fileType;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+/**
+ * <p>
+ * A collection of tests for the {@link ViewFs}.
+ * This test should be used for testing ViewFs that has mount links to 
+ * a target file system such  localFs or Hdfs etc.
+
+ * </p>
+ * <p>
+ * To test a given target file system create a subclass of this
+ * test and override {@link #setUp()} to initialize the <code>fcTarget</code> 
+ * to point to the file system to which you want the mount targets
+ * 
+ * Since this a junit 4 you can also do a single setup before 
+ * the start of any tests.
+ * E.g.
+ *     @BeforeClass   public static void clusterSetupAtBegining()
+ *     @AfterClass    public static void ClusterShutdownAtEnd()
+ * </p>
+ */
+public class ViewFsBaseTest {
+  FileContext fcView; // the view file system - the mounts are here
+  FileContext fcTarget; // the target file system - the mount will point here
+  Path targetTestRoot;
+
+  @Before
+  public void setUp() throws Exception {
+
+    targetTestRoot = FileContextTestHelper.getAbsoluteTestRootPath(fcTarget);
+    // In case previous test was killed before cleanup
+    fcTarget.delete(targetTestRoot, true);
+    
+    fcTarget.mkdir(targetTestRoot, FileContext.DEFAULT_PERM, true);
+    // Make  user and data dirs - we creates links to them in the mount table
+    fcTarget.mkdir(new Path(targetTestRoot,"user"),
+        FileContext.DEFAULT_PERM, true);
+    fcTarget.mkdir(new Path(targetTestRoot,"data"),
+        FileContext.DEFAULT_PERM, true);
+    fcTarget.mkdir(new Path(targetTestRoot,"dir2"),
+        FileContext.DEFAULT_PERM, true);
+    fcTarget.mkdir(new Path(targetTestRoot,"dir3"),
+        FileContext.DEFAULT_PERM, true);
+    FileContextTestHelper.createFile(fcTarget, new Path(targetTestRoot,"aFile"));
+    
+    
+    // Now we use the mount fs to set links to user and dir
+    // in the test root
+    
+    // Set up the defaultMT in the config with our mount point links
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, "/user",
+        new Path(targetTestRoot,"user").toUri());
+    ConfigUtil.addLink(conf, "/user2",
+        new Path(targetTestRoot,"user").toUri());
+    ConfigUtil.addLink(conf, "/data",
+        new Path(targetTestRoot,"data").toUri());
+    ConfigUtil.addLink(conf, "/internalDir/linkToDir2",
+        new Path(targetTestRoot,"dir2").toUri());
+    ConfigUtil.addLink(conf, "/internalDir/internalDir2/linkToDir3",
+        new Path(targetTestRoot,"dir3").toUri());
+    ConfigUtil.addLink(conf, "/danglingLink",
+        new Path(targetTestRoot,"missingTarget").toUri());
+    ConfigUtil.addLink(conf, "/linkToAFile",
+        new Path(targetTestRoot,"aFile").toUri());
+    
+    fcView = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fcTarget.delete(FileContextTestHelper.getTestRootPath(fcTarget), true);
+  }
+  
+  @Test
+  public void testBasicPaths() {
+    Assert.assertEquals(FsConstants.VIEWFS_URI,
+        fcView.getDefaultFileSystem().getUri());
+    Assert.assertEquals(fcView.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fcView.getWorkingDirectory());
+    Assert.assertEquals(fcView.makeQualified(
+        new Path("/user/" + System.getProperty("user.name"))),
+        fcView.getHomeDirectory());
+    Assert.assertEquals(
+        new Path("/foo/bar").makeQualified(FsConstants.VIEWFS_URI, null),
+        fcView.makeQualified(new Path("/foo/bar")));
+  }
+  
+  /** 
+   * Test modify operations (create, mkdir, delete, etc) 
+   * on the mount file system where the pathname references through
+   * the mount points.  Hence these operation will modify the target
+   * file system.
+   * 
+   * Verify the operation via mountfs (ie fc) and *also* via the
+   *  target file system (ie fclocal) that the mount link points-to.
+   */
+  @Test
+  public void testOperationsThroughMountLinks() throws IOException {
+    // Create file 
+    FileContextTestHelper.createFileNonRecursive(fcView, "/user/foo");
+    Assert.assertTrue(isFile(fcView, new Path("/user/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(targetTestRoot,"user/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fcView.delete(new Path("/user/foo"), false));
+    Assert.assertFalse(exists(fcView, new Path("/user/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"user/foo")));
+    
+    // Create file with a 2 component dirs
+    FileContextTestHelper.createFileNonRecursive(fcView,
+        "/internalDir/linkToDir2/foo");
+    Assert.assertTrue(isFile(fcView, new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(targetTestRoot,"dir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue(fcView.delete(new Path("/internalDir/linkToDir2/foo"),false));
+    Assert.assertFalse(exists(fcView, new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"dir2/foo")));
+    
+    
+    // Create file with a 3 component dirs
+    FileContextTestHelper.createFileNonRecursive(fcView,
+        "/internalDir/internalDir2/linkToDir3/foo");
+    Assert.assertTrue(isFile(fcView,
+        new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertTrue(isFile(fcTarget, new Path(targetTestRoot,"dir3/foo")));
+    
+    // Recursive Create file with missing dirs
+    FileContextTestHelper.createFile(fcView,
+        "/internalDir/linkToDir2/missingDir/miss2/foo");
+    Assert.assertTrue(isFile(fcView,
+        new Path("/internalDir/linkToDir2/missingDir/miss2/foo")));
+    Assert.assertTrue(isFile(fcTarget,
+        new Path(targetTestRoot,"dir2/missingDir/miss2/foo")));
+
+    
+    // Delete the created file
+    Assert.assertTrue(fcView.delete(
+        new Path("/internalDir/internalDir2/linkToDir3/foo"), false));
+    Assert.assertFalse(exists(fcView,
+        new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"dir3/foo")));
+    
+      
+    // mkdir
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/user/dirX"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue(isDir(fcView, new Path("/user/dirX")));
+    Assert.assertTrue(isDir(fcTarget, new Path(targetTestRoot,"user/dirX")));
+    
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/user/dirX/dirY"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue(isDir(fcView, new Path("/user/dirX/dirY")));
+    Assert.assertTrue(isDir(fcTarget,new Path(targetTestRoot,"user/dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue(fcView.delete(new Path("/user/dirX/dirY"), false));
+    Assert.assertFalse(exists(fcView, new Path("/user/dirX/dirY")));
+    Assert.assertFalse(exists(fcTarget,
+        new Path(targetTestRoot,"user/dirX/dirY")));
+    
+    Assert.assertTrue(fcView.delete(new Path("/user/dirX"), false));
+    Assert.assertFalse(exists(fcView, new Path("/user/dirX")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"user/dirX")));
+    
+    // Rename a file 
+    FileContextTestHelper.createFile(fcView, "/user/foo");
+    fcView.rename(new Path("/user/foo"), new Path("/user/fooBar"));
+    Assert.assertFalse(exists(fcView, new Path("/user/foo")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"user/foo")));
+    Assert.assertTrue(isFile(fcView,
+        FileContextTestHelper.getTestRootPath(fcView,"/user/fooBar")));
+    Assert.assertTrue(isFile(fcTarget, new Path(targetTestRoot,"user/fooBar")));
+    
+    fcView.mkdir(new Path("/user/dirFoo"), FileContext.DEFAULT_PERM, false);
+    fcView.rename(new Path("/user/dirFoo"), new Path("/user/dirFooBar"));
+    Assert.assertFalse(exists(fcView, new Path("/user/dirFoo")));
+    Assert.assertFalse(exists(fcTarget, new Path(targetTestRoot,"user/dirFoo")));
+    Assert.assertTrue(isDir(fcView,
+        FileContextTestHelper.getTestRootPath(fcView,"/user/dirFooBar")));
+    Assert.assertTrue(isDir(fcTarget,new Path(targetTestRoot,"user/dirFooBar")));
+    
+  }
+  
+  // rename across mount points that point to same target also fail 
+  @Test(expected=IOException.class) 
+  public void testRenameAcrossMounts1() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/user/foo");
+    fcView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
+    /* - code if we had wanted this to suceed
+    Assert.assertFalse(exists(fc, new Path("/user/foo")));
+    Assert.assertFalse(exists(fclocal, new Path(targetTestRoot,"user/foo")));
+    Assert.assertTrue(isFile(fc,
+       FileContextTestHelper.getTestRootPath(fc,"/user2/fooBarBar")));
+    Assert.assertTrue(isFile(fclocal,
+        new Path(targetTestRoot,"user/fooBarBar")));
+    */
+  }
+  
+  
+  // rename across mount points fail if the mount link targets are different
+  // even if the targets are part of the same target FS
+
+  @Test(expected=IOException.class) 
+  public void testRenameAcrossMounts2() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/user/foo");
+    fcView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+  }
+  
+  
+  
+  
+  static protected boolean SupportsBlocks = false; //  local fs use 1 block
+                                                   // override for HDFS
+  @Test
+  public void testGetBlockLocations() throws IOException {
+    Path targetFilePath = new Path(targetTestRoot,"data/largeFile");
+    FileContextTestHelper.createFile(fcTarget, targetFilePath, 10, 1024);
+    Path viewFilePath = new Path("/data/largeFile");
+    checkFileStatus(fcView, viewFilePath.toString(), fileType.isFile);
+    BlockLocation[] viewBL = fcView.getFileBlockLocations(viewFilePath,
+        0, 10240+100);
+    Assert.assertEquals(SupportsBlocks ? 10 : 1, viewBL.length);
+    BlockLocation[] targetBL = fcTarget.getFileBlockLocations(targetFilePath, 0, 10240+100);
+    compareBLs(viewBL, targetBL);
+    
+    
+    // Same test but now get it via the FileStatus Parameter
+    fcView.getFileBlockLocations(viewFilePath, 0, 10240+100);
+    targetBL = fcTarget.getFileBlockLocations(targetFilePath, 0, 10240+100);
+    compareBLs(viewBL, targetBL);  
+  }
+  
+  void compareBLs(BlockLocation[] viewBL, BlockLocation[] targetBL) {
+    Assert.assertEquals(targetBL.length, viewBL.length);
+    int i = 0;
+    for (BlockLocation vbl : viewBL) {
+      Assert.assertEquals(vbl.toString(), targetBL[i].toString());
+      Assert.assertEquals(targetBL[i].getOffset(), vbl.getOffset());
+      Assert.assertEquals(targetBL[i].getLength(), vbl.getLength());
+      i++;     
+    } 
+  }
+   
+  
+  /**
+   * Test "readOps" (e.g. list, listStatus) 
+   * on internal dirs of mount table
+   * These operations should succeed.
+   */
+  
+  // test list on internal dirs of mount table 
+  @Test
+  public void testListOnInternalDirsOfMountTable() throws IOException {
+    
+    // list on Slash
+    
+    FileStatus[] dirPaths = fcView.util().listStatus(new Path("/"));
+    FileStatus fs;
+    Assert.assertEquals(6, dirPaths.length);
+    fs = FileContextTestHelper.containsPath(fcView, "/user", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/data", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/internalDir", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isDirectory());
+    fs = FileContextTestHelper.containsPath(fcView, "/danglingLink", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/linkToAFile", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue(fs.isSymlink());
+      
+      
+      
+      // list on internal dir
+      dirPaths = fcView.util().listStatus(new Path("/internalDir"));
+      Assert.assertEquals(2, dirPaths.length);
+
+      fs = FileContextTestHelper.containsPath(fcView,
+          "/internalDir/internalDir2", dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue(fs.isDirectory());
+      fs = FileContextTestHelper.containsPath(fcView,
+          "/internalDir/linkToDir2", dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue(fs.isSymlink());
+  }
+      
+  @Test
+  public void testFileStatusOnMountLink() throws IOException {
+    Assert.assertTrue(fcView.getFileStatus(new Path("/")).isDirectory());
+    checkFileStatus(fcView, "/", fileType.isDir);
+    checkFileStatus(fcView, "/user", fileType.isDir);
+    checkFileStatus(fcView, "/data", fileType.isDir);
+    checkFileStatus(fcView, "/internalDir", fileType.isDir);
+    checkFileStatus(fcView, "/internalDir/linkToDir2", fileType.isDir);
+    checkFileStatus(fcView, "/internalDir/internalDir2/linkToDir3", fileType.isDir);
+    checkFileStatus(fcView, "/linkToAFile", fileType.isFile);
+
+    try {
+      fcView.getFileStatus(new Path("/danglingLink"));
+      Assert.fail("Excepted a not found exception here");
+    } catch ( FileNotFoundException e) {
+      // as excepted
+    }
+  }
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testgetFSonDanglingLink() throws IOException {
+    fcView.getFileStatus(new Path("/danglingLink"));
+  }
+  
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testgetFSonNonExistingInternalDir() throws IOException {
+    fcView.getFileStatus(new Path("/internalDir/nonExisting"));
+  }
+  
+  @Test
+  public void testgetFileLinkStatus() throws IOException {
+    checkFileLinkStatus(fcView, "/user", fileType.isSymlink);
+    checkFileLinkStatus(fcView, "/data", fileType.isSymlink);
+    checkFileLinkStatus(fcView, "/internalDir/linkToDir2", fileType.isSymlink);
+    checkFileLinkStatus(fcView, "/internalDir/internalDir2/linkToDir3",
+        fileType.isSymlink);
+    checkFileLinkStatus(fcView, "/linkToAFile", fileType.isSymlink);
+    checkFileLinkStatus(fcView, "/internalDir", fileType.isDir);
+    checkFileLinkStatus(fcView, "/internalDir/internalDir2", fileType.isDir);
+  }
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testgetFileLinkStatusonNonExistingInternalDir()
+    throws IOException {
+    fcView.getFileLinkStatus(new Path("/internalDir/nonExisting"));
+  }
+  
+  @Test
+  public void testSymlinkTarget() throws IOException {
+
+    // get link target`
+    Assert.assertTrue(fcView.getLinkTarget(new Path("/user"
+      )).equals(new Path(targetTestRoot,"user")));
+    Assert.assertTrue(fcView.getLinkTarget(new Path("/data"
+      )).equals(new Path(targetTestRoot,"data")));
+    Assert.assertTrue(fcView.getLinkTarget(new Path("/internalDir/linkToDir2"
+      )).equals(new Path(targetTestRoot,"dir2")));
+    Assert.assertTrue(fcView.getLinkTarget(new Path(
+     "/internalDir/internalDir2/linkToDir3"
+      )).equals(new Path(targetTestRoot,"dir3")));
+    Assert.assertTrue(fcView.getLinkTarget(new Path("/linkToAFile"
+      )).equals(new Path(targetTestRoot,"aFile")));
+  }
+  
+  @Test(expected=IOException.class) 
+  public void testgetLinkTargetOnNonLink() throws IOException {
+    fcView.getLinkTarget(new Path("/internalDir/internalDir2"));
+  }
+  
+  
+  
+  
+  /**
+   * Test modify operations (create, mkdir, rename, etc) 
+   * on internal dirs of mount table
+   * These operations should fail since the mount table is read-only or
+   * because the internal dir that it is trying to create already
+   * exits.
+   */
+ 
+ 
+  // Mkdir on internal mount table should fail
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirSlash() throws IOException {
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/"),
+        FileContext.DEFAULT_PERM, false);
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirExisting1() throws IOException {
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/internalDir"),
+        FileContext.DEFAULT_PERM, false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirExisting2() throws IOException {
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView,
+        "/internalDir/linkToDir2"),
+        FileContext.DEFAULT_PERM, false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirNew() throws IOException {
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/dirNew"),
+        FileContext.DEFAULT_PERM, false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirNew2() throws IOException {
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/internalDir/dirNew"),
+        FileContext.DEFAULT_PERM, false);
+  }
+  
+  // Create on internal mount table should fail
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreate1() throws IOException {
+    FileContextTestHelper.createFileNonRecursive(fcView, "/foo"); // 1 component
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreate2() throws IOException {  // 2 component
+    FileContextTestHelper.createFileNonRecursive(fcView, "/internalDir/foo");
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/missingDir/foo");
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir2() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/missingDir/miss2/foo");
+  }
+  
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalCreateMissingDir3() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/internalDir/miss2/foo");
+  }
+  
+  // Delete on internal mount table should fail
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testInternalDeleteNonExisting() throws IOException {
+      fcView.delete(new Path("/NonExisting"), false);
+  }
+  @Test(expected=FileNotFoundException.class) 
+  public void testInternalDeleteNonExisting2() throws IOException {
+      fcView.delete(new Path("/internalDir/NonExisting"), false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalDeleteExisting() throws IOException {
+      fcView.delete(new Path("/internalDir"), false);
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalDeleteExisting2() throws IOException {
+    Assert.assertTrue(
+        fcView.getFileStatus(new Path("/internalDir/linkToDir2")).isDirectory());
+    fcView.delete(new Path("/internalDir/linkToDir2"), false);
+  } 
+  
+  
+  // Rename on internal mount table should fail
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename1() throws IOException {
+    fcView.rename(new Path("/internalDir"), new Path("/newDir"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename2() throws IOException {
+    Assert.assertTrue(
+        fcView.getFileStatus(new Path("/internalDir/linkToDir2")).isDirectory());
+    fcView.rename(new Path("/internalDir/linkToDir2"),
+        new Path("/internalDir/dir1"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRename3() throws IOException {
+    fcView.rename(new Path("/user"), new Path("/internalDir/linkToDir2"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRenameToSlash() throws IOException {
+    fcView.rename(new Path("/internalDir/linkToDir2/foo"), new Path("/"));
+  }
+  @Test(expected=AccessControlException.class) 
+  public void testInternalRenameFromSlash() throws IOException {
+    fcView.rename(new Path("/"), new Path("/bar"));
+  }
+  
+  @Test(expected=AccessControlException.class) 
+  public void testInternalSetOwner() throws IOException {
+    fcView.setOwner(new Path("/internalDir"), "foo", "bar");
+  }
+}

+ 72 - 0
src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java

@@ -0,0 +1,72 @@
+package org.apache.hadoop.fs.viewfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+
+
+/**
+ * This class is for  setup and teardown for viewFs so that
+ * it can be tested via the standard FileContext tests.
+ * 
+ * If tests launched via ant (build.xml) the test root is absolute path
+ * If tests launched via eclipse, the test root is 
+ * is a test dir below the working directory. (see FileContextTestHelper).
+ * Since viewFs has no built-in wd, its wd is /user/<username>.
+ * 
+ * We set up fc to be the viewFs with mount point for 
+ * /<firstComponent>" pointing to the local file system's testdir 
+ */
+public class ViewFsTestSetup {
+
+
+   /* 
+   * return the ViewFS File context to be used for tests
+   */
+  static public FileContext setupForViewFsLocalFs() throws Exception {
+    /**
+     * create the test root on local_fs - the  mount table will point here
+     */
+    FileContext fclocal = FileContext.getLocalFSFileContext();
+    Path targetOfTests = FileContextTestHelper.getTestRootPath(fclocal);
+    // In case previous test was killed before cleanup
+    fclocal.delete(targetOfTests, true);
+    
+    fclocal.mkdir(targetOfTests, FileContext.DEFAULT_PERM, true);
+  
+    String srcTestFirstDir;
+    if (FileContextTestHelper.TEST_ROOT_DIR.startsWith("/")) {
+      int indexOf2ndSlash = FileContextTestHelper.TEST_ROOT_DIR.indexOf('/', 1);
+      srcTestFirstDir = FileContextTestHelper.TEST_ROOT_DIR.substring(0, indexOf2ndSlash);
+    } else {
+      srcTestFirstDir = "/user"; 
+  
+    }
+    //System.out.println("srcTestFirstDir=" + srcTestFirstDir);
+  
+    // Set up the defaultMT in the config with mount point links
+    // The test dir is root is below  /user/<userid>
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, srcTestFirstDir,
+        targetOfTests.toUri());
+    
+    FileContext fc = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
+    //System.out.println("SRCOfTests = "+ getTestRootPath(fc, "test"));
+    //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
+    return fc;
+  }
+
+  /**
+   * 
+   * delete the test directory in the target local fs
+   */
+  static public void tearDownForViewFsLocalFs() throws Exception {
+    FileContext fclocal = FileContext.getLocalFSFileContext();
+    Path targetOfTests = FileContextTestHelper.getTestRootPath(fclocal);
+    fclocal.delete(targetOfTests, true);
+  }
+
+}