You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sr...@apache.org on 2011/05/06 04:11:32 UTC

svn commit: r1100026 [3/3] - in /hadoop/common/trunk: ./ src/java/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/viewfs/ src/test/core/org/apache/hadoop/fs/ src/test/core/org/apache/hadoop/fs/viewfs/

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsWithAuthorityLocalFs.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsWithAuthorityLocalFs.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsWithAuthorityLocalFs.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsWithAuthorityLocalFs.java Fri May  6 02:11:31 2011
@@ -0,0 +1,76 @@
+/**
+ * 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.fs.FileContext;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * 
+ * Test the ViewFsBaseTest using a viewfs with authority: 
+ *    viewfs://mountTableName/
+ *    ie the authority is used to load a mount table.
+ *    The authority name used is "default"
+ *
+ */
+
+public class TestViewFsWithAuthorityLocalFs extends ViewFsBaseTest {
+  URI schemeWithAuthority;
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    fcTarget = FileContext.getLocalFSFileContext();
+    super.setUp(); // this sets up conf (and fcView which we replace)
+    
+    // Now create a viewfs using a mount table called "default"
+    // hence viewfs://default/
+    schemeWithAuthority = 
+      new URI(FsConstants.VIEWFS_SCHEME, "default", "/", null, null);
+    fcView = FileContext.getFileContext(schemeWithAuthority, conf);  
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+  
+  @Test
+  public void testBasicPaths() {
+      Assert.assertEquals(schemeWithAuthority,
+          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(schemeWithAuthority, null),
+          fcView.makeQualified(new Path("/foo/bar")));
+  }
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java Fri May  6 02:11:31 2011
@@ -0,0 +1,89 @@
+/**
+ * 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.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * The FileStatus is being serialized in MR as jobs are submitted.
+ * Since viewfs has overlayed ViewFsFileStatus, we ran into
+ * serialization problems. THis test is test the fix.
+ */
+public class TestViewfsFileStatus {
+
+  private static final File TEST_DIR =
+    new File(System.getProperty("test.build.data", "/tmp"),
+          TestViewfsFileStatus.class.getSimpleName());
+
+  @Test
+  public void testFileStatusSerialziation()
+      throws IOException, URISyntaxException {
+    String testfilename = "testFileStatusSerialziation";
+    TEST_DIR.mkdirs();
+    File infile = new File(TEST_DIR, testfilename);
+    final byte[] content = "dingos".getBytes();
+
+    FileOutputStream fos = null;
+    try {
+      fos = new FileOutputStream(infile);
+      fos.write(content);
+    } finally {
+      if (fos != null) {
+        fos.close();
+      }
+    }
+    assertEquals((long)content.length, infile.length());
+
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, "/foo/bar/baz", TEST_DIR.toURI());
+    FileSystem vfs = FileSystem.get(FsConstants.VIEWFS_URI, conf);
+    assertEquals(ViewFileSystem.class, vfs.getClass());
+    FileStatus stat = vfs.getFileStatus(new Path("/foo/bar/baz", testfilename));
+    assertEquals(content.length, stat.getLen());
+    // check serialization/deserialization
+    DataOutputBuffer dob = new DataOutputBuffer();
+    stat.write(dob);
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(dob.getData(), 0, dob.getLength());
+    FileStatus deSer = new FileStatus();
+    deSer.readFields(dib);
+    assertEquals(content.length, deSer.getLen());
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    FileUtil.fullyDelete(TEST_DIR);
+  }
+
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java Fri May  6 02:11:31 2011
@@ -0,0 +1,608 @@
+/**
+ * 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.util.List;
+
+
+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.fs.viewfs.ViewFileSystem.MountPoint;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+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;
+  Configuration conf;
+
+
+  @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();
+    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 testGetMountPoints() {
+    ViewFileSystem viewfs = (ViewFileSystem) fsView;
+    MountPoint[] mountPoints = viewfs.getMountPoints();
+    Assert.assertEquals(7, mountPoints.length); 
+  }
+  
+  /**
+   * This default implementation is when viewfs has mount points
+   * into file systems, such as LocalFs that do no have delegation tokens.
+   * It should be overridden for when mount points into hdfs.
+   */
+  @Test
+  public void testGetDelegationTokens() throws IOException {
+    List<Token<?>> delTokens = 
+        fsView.getDelegationTokens("sanjay");
+    Assert.assertEquals(0, delTokens.size()); 
+  }
+  
+  @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("Created file should be type file",
+        fsView.isFile(new Path("/user/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        fsTarget.isFile(new Path(targetTestRoot,"user/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should suceed",
+        fsView.delete(new Path("/user/foo"), false));
+    Assert.assertFalse("File should not exist after delete",
+        fsView.exists(new Path("/user/foo")));
+    Assert.assertFalse("Target File should not exist after delete",
+        fsTarget.exists(new Path(targetTestRoot,"user/foo")));
+    
+    // Create file with a 2 component dirs
+    FileSystemTestHelper.createFile(fsView, "/internalDir/linkToDir2/foo");
+    Assert.assertTrue("Created file should be type file",
+        fsView.isFile(new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        fsTarget.isFile(new Path(targetTestRoot,"dir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should suceed",
+        fsView.delete(new Path("/internalDir/linkToDir2/foo"), false));
+    Assert.assertFalse("File should not exist after delete",
+        fsView.exists(new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertFalse("Target File should not exist after delete",
+        fsTarget.exists(new Path(targetTestRoot,"dir2/foo")));
+    
+    
+    // Create file with a 3 component dirs
+    FileSystemTestHelper.createFile(fsView, "/internalDir/internalDir2/linkToDir3/foo");
+    Assert.assertTrue("Created file should be type file",
+        fsView.isFile(new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        fsTarget.isFile(new Path(targetTestRoot,"dir3/foo")));
+    
+    // Recursive Create file with missing dirs
+    FileSystemTestHelper.createFile(fsView,
+        "/internalDir/linkToDir2/missingDir/miss2/foo");
+    Assert.assertTrue("Created file should be type file",
+        fsView.isFile(new Path("/internalDir/linkToDir2/missingDir/miss2/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        fsTarget.isFile(new Path(targetTestRoot,"dir2/missingDir/miss2/foo")));
+
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should succeed",
+        fsView.delete(
+            new Path("/internalDir/internalDir2/linkToDir3/foo"), false));
+    Assert.assertFalse("File should not exist after delete",
+        fsView.exists(new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertFalse("Target File should not exist after delete",
+        fsTarget.exists(new Path(targetTestRoot,"dir3/foo")));
+    
+      
+    // mkdir
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX"));
+    Assert.assertTrue("New dir should be type dir", 
+        fsView.isDirectory(new Path("/user/dirX")));
+    Assert.assertTrue("Target of new dir should be of type dir",
+        fsTarget.isDirectory(new Path(targetTestRoot,"user/dirX")));
+    
+    fsView.mkdirs(
+        FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX/dirY"));
+    Assert.assertTrue("New dir should be type dir", 
+        fsView.isDirectory(new Path("/user/dirX/dirY")));
+    Assert.assertTrue("Target of new dir should be of type dir",
+        fsTarget.isDirectory(new Path(targetTestRoot,"user/dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue("Delete should succeed",
+        fsView.delete(new Path("/user/dirX/dirY"), false));
+    Assert.assertFalse("File should not exist after delete",
+        fsView.exists(new Path("/user/dirX/dirY")));
+    Assert.assertFalse("Target File should not exist after delete",
+        fsTarget.exists(new Path(targetTestRoot,"user/dirX/dirY")));
+    
+    Assert.assertTrue("Delete should succeed",
+        fsView.delete(new Path("/user/dirX"), false));
+    Assert.assertFalse("File should not exist after delete",
+        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("Renamed src should not exist", 
+        fsView.exists(new Path("/user/foo")));
+    Assert.assertFalse("Renamed src should not exist in target",
+        fsTarget.exists(new Path(targetTestRoot,"user/foo")));
+    Assert.assertTrue("Renamed dest should  exist as file",
+        fsView.isFile(FileSystemTestHelper.getTestRootPath(fsView,"/user/fooBar")));
+    Assert.assertTrue("Renamed dest should  exist as file in target",
+        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("Renamed src should not exist", 
+        fsView.exists(new Path("/user/dirFoo")));
+    Assert.assertFalse("Renamed src should not exist in target",
+        fsTarget.exists(new Path(targetTestRoot,"user/dirFoo")));
+    Assert.assertTrue("Renamed dest should  exist as dir",
+        fsView.isDirectory(FileSystemTestHelper.getTestRootPath(fsView,"/user/dirFooBar")));
+    Assert.assertTrue("Renamed dest should  exist as dir in target",
+        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("Created File should be type File",
+        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("A mount should appear as symlink", fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/data", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/internalDir", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", fs.isDirectory());
+    fs = FileSystemTestHelper.containsPath(fsView, "/danglingLink", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", fs.isSymlink());
+    fs = FileSystemTestHelper.containsPath(fsView, "/linkToAFile", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", 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("A mount should appear as symlink", fs.isDirectory());
+      fs = FileSystemTestHelper.containsPath(fsView, "/internalDir/linkToDir2",
+          dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue("A mount should appear as symlink", 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
+    long len = 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("Created file shoudl appear as a file", fs.isFile());
+    Assert.assertEquals(len, fs.getLen());
+    
+    // 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("Created file shoudl appear as a file", fs.isFile());
+    fs = FileSystemTestHelper.containsPath(fsView, "/data/dirX", dirPaths);
+    Assert.assertNotNull(fs);
+    Assert.assertTrue("Created dir should appear as a dir", 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 resolvePath(p) 
+   */
+  
+  @Test
+  public void testResolvePathInternalPaths() throws IOException {
+    Assert.assertEquals(new Path("/"), fsView.resolvePath(new Path("/")));
+    Assert.assertEquals(new Path("/internalDir"),
+                          fsView.resolvePath(new Path("/internalDir")));
+  }
+  @Test
+  public void testResolvePathMountPoints() throws IOException {
+    Assert.assertEquals(new Path(targetTestRoot,"user"),
+                          fsView.resolvePath(new Path("/user")));
+    Assert.assertEquals(new Path(targetTestRoot,"data"),
+        fsView.resolvePath(new Path("/data")));
+    Assert.assertEquals(new Path(targetTestRoot,"dir2"),
+        fsView.resolvePath(new Path("/internalDir/linkToDir2")));
+    Assert.assertEquals(new Path(targetTestRoot,"dir3"),
+        fsView.resolvePath(new Path("/internalDir/internalDir2/linkToDir3")));
+
+  }
+  
+  @Test
+  public void testResolvePathThroughMountPoints() throws IOException {
+    FileSystemTestHelper.createFile(fsView, "/user/foo");
+    Assert.assertEquals(new Path(targetTestRoot,"user/foo"),
+                          fsView.resolvePath(new Path("/user/foo")));
+    
+    fsView.mkdirs(
+        FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX"));
+    Assert.assertEquals(new Path(targetTestRoot,"user/dirX"),
+        fsView.resolvePath(new Path("/user/dirX")));
+
+    
+    fsView.mkdirs(
+        FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX/dirY"));
+    Assert.assertEquals(new Path(targetTestRoot,"user/dirX/dirY"),
+        fsView.resolvePath(new Path("/user/dirX/dirY")));
+  }
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathDanglingLink() throws IOException {
+      fsView.resolvePath(new Path("/danglingLink"));
+  }
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathMissingThroughMountPoints() throws IOException {
+    fsView.resolvePath(new Path("/user/nonExisting"));
+  }
+  
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathMissingThroughMountPoints2() throws IOException {
+    fsView.mkdirs(
+        FileSystemTestHelper.getTestRootPath(fsView, "/user/dirX"));
+    fsView.resolvePath(new Path("/user/dirX/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 existing internal mount table succeed except for /
+  @Test(expected=AccessControlException.class) 
+  public void testInternalMkdirSlash() throws IOException {
+    fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView, "/"));
+  }
+  
+  public void testInternalMkdirExisting1() throws IOException {
+    Assert.assertTrue("mkdir of existing dir should succeed", 
+        fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView,
+        "/internalDir")));
+  }
+
+  public void testInternalMkdirExisting2() throws IOException {
+    Assert.assertTrue("mkdir of existing dir should succeed", 
+        fsView.mkdirs(FileSystemTestHelper.getTestRootPath(fsView,
+        "/internalDir/linkToDir2")));
+  }
+  
+  // Mkdir for new internal mount table should fail
+  @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 File 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 {
+        fsView.getFileStatus(
+            new Path("/internalDir/linkToDir2")).isDirectory();
+    fsView.delete(new Path("/internalDir/linkToDir2"), false);
+  } 
+  
+  @Test
+  public void testMkdirOfMountLink() throws IOException {
+    // data exists - mkdirs returns true even though no permission in internal
+    // mount table
+    Assert.assertTrue("mkdir of existing mount link should succeed", 
+        fsView.mkdirs(new Path("/data")));
+  }
+  
+  
+  // 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 {
+       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");
+  }
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java Fri May  6 02:11:31 2011
@@ -0,0 +1,95 @@
+/**
+ * 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.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 fsView = FileSystem.get(FsConstants.VIEWFS_URI, conf);
+    //System.out.println("SRCOfTests = "+ getTestRootPath(fs, "test"));
+    //System.out.println("TargetOfTests = "+ targetOfTests.toUri());
+    return fsView;
+  }
+
+  /**
+   * 
+   * 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; 
+  }
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java Fri May  6 02:11:31 2011
@@ -0,0 +1,646 @@
+/**
+ * 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 java.net.URI;
+import java.util.List;
+
+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.fs.viewfs.ViewFs.MountPoint;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+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;
+  Configuration conf;
+  FileContext xfcViewWithAuthority; // same as fsView but with authority
+  URI schemeWithAuthority;
+
+  @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
+    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);
+    // Also try viewfs://default/    - note authority is name of mount table
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fcTarget.delete(FileContextTestHelper.getTestRootPath(fcTarget), true);
+  }
+  
+  @Test
+  public void testGetMountPoints() {
+    ViewFs viewfs = (ViewFs) fcView.getDefaultFileSystem();
+    MountPoint[] mountPoints = viewfs.getMountPoints();
+    Assert.assertEquals(7, mountPoints.length); 
+  }
+  
+  /**
+   * This default implementation is when viewfs has mount points
+   * into file systems, such as LocalFs that do no have delegation tokens.
+   * It should be overridden for when mount points into hdfs.
+   */
+  @Test
+  public void testGetDelegationTokens() throws IOException {
+    List<Token<?>> delTokens = 
+        fcView.getDelegationTokens(new Path("/"), "sanjay");
+    Assert.assertEquals(0, delTokens.size()); 
+  }
+
+  
+  @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("Create file should be file",
+		isFile(fcView, new Path("/user/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        isFile(fcTarget, new Path(targetTestRoot,"user/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should succeed",
+        fcView.delete(new Path("/user/foo"), false));
+    Assert.assertFalse("File should not exist after delete",
+        exists(fcView, new Path("/user/foo")));
+    Assert.assertFalse("Target File should not exist after delete",
+        exists(fcTarget, new Path(targetTestRoot,"user/foo")));
+    
+    // Create file with a 2 component dirs
+    FileContextTestHelper.createFileNonRecursive(fcView,
+        "/internalDir/linkToDir2/foo");
+    Assert.assertTrue("Created file should be type file",
+        isFile(fcView, new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertTrue("Target of created file should be type file",
+        isFile(fcTarget, new Path(targetTestRoot,"dir2/foo")));
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should suceed",
+        fcView.delete(new Path("/internalDir/linkToDir2/foo"),false));
+    Assert.assertFalse("File should not exist after deletion",
+        exists(fcView, new Path("/internalDir/linkToDir2/foo")));
+    Assert.assertFalse("Target should not exist after deletion",
+        exists(fcTarget, new Path(targetTestRoot,"dir2/foo")));
+    
+    
+    // Create file with a 3 component dirs
+    FileContextTestHelper.createFileNonRecursive(fcView,
+        "/internalDir/internalDir2/linkToDir3/foo");
+    Assert.assertTrue("Created file should be of type file", 
+        isFile(fcView, new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertTrue("Target of created file should also be type file",
+        isFile(fcTarget, new Path(targetTestRoot,"dir3/foo")));
+    
+    // Recursive Create file with missing dirs
+    FileContextTestHelper.createFile(fcView,
+        "/internalDir/linkToDir2/missingDir/miss2/foo");
+    Assert.assertTrue("Created file should be of type file",
+      isFile(fcView, new Path("/internalDir/linkToDir2/missingDir/miss2/foo")));
+    Assert.assertTrue("Target of created file should also be type file",
+        isFile(fcTarget, new Path(targetTestRoot,"dir2/missingDir/miss2/foo")));
+
+    
+    // Delete the created file
+    Assert.assertTrue("Delete should succeed",  fcView.delete(
+        new Path("/internalDir/internalDir2/linkToDir3/foo"), false));
+    Assert.assertFalse("Deleted File should not exist", 
+        exists(fcView, new Path("/internalDir/internalDir2/linkToDir3/foo")));
+    Assert.assertFalse("Target of deleted file should not exist", 
+        exists(fcTarget, new Path(targetTestRoot,"dir3/foo")));
+    
+      
+    // mkdir
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/user/dirX"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue("New dir should be type dir", 
+        isDir(fcView, new Path("/user/dirX")));
+    Assert.assertTrue("Target of new dir should be of type dir",
+        isDir(fcTarget, new Path(targetTestRoot,"user/dirX")));
+    
+    fcView.mkdir(FileContextTestHelper.getTestRootPath(fcView, "/user/dirX/dirY"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertTrue("New dir should be type dir", 
+        isDir(fcView, new Path("/user/dirX/dirY")));
+    Assert.assertTrue("Target of new dir should be of type dir",
+        isDir(fcTarget,new Path(targetTestRoot,"user/dirX/dirY")));
+    
+
+    // Delete the created dir
+    Assert.assertTrue("Delete should succeed",
+        fcView.delete(new Path("/user/dirX/dirY"), false));
+    Assert.assertFalse("Deleted File should not exist",
+        exists(fcView, new Path("/user/dirX/dirY")));
+    Assert.assertFalse("Deleted Target should not exist", 
+        exists(fcTarget, new Path(targetTestRoot,"user/dirX/dirY")));
+    
+    Assert.assertTrue("Delete should succeed",
+        fcView.delete(new Path("/user/dirX"), false));
+    Assert.assertFalse("Deleted File should not exist",
+        exists(fcView, new Path("/user/dirX")));
+    Assert.assertFalse("Deleted Target should not exist",
+        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("Renamed src should not exist", 
+        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("Renamed src should not exist",
+        exists(fcView, new Path("/user/dirFoo")));
+    Assert.assertFalse("Renamed src should not exist in target",
+        exists(fcTarget, new Path(targetTestRoot,"user/dirFoo")));
+    Assert.assertTrue("Renamed dest should  exist as dir",
+        isDir(fcView,
+        FileContextTestHelper.getTestRootPath(fcView,"/user/dirFooBar")));
+    Assert.assertTrue("Renamed dest should  exist as dir in target",
+        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 succeed
+    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("A mount should appear as symlink", fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/data", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/internalDir", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("InternalDirs should appear as dir", fs.isDirectory());
+    fs = FileContextTestHelper.containsPath(fcView, "/danglingLink", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", fs.isSymlink());
+    fs = FileContextTestHelper.containsPath(fcView, "/linkToAFile", dirPaths);
+      Assert.assertNotNull(fs);
+      Assert.assertTrue("A mount should appear as symlink", 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("InternalDirs should appear as dir",fs.isDirectory());
+      fs = FileContextTestHelper.containsPath(fcView,
+          "/internalDir/linkToDir2", dirPaths);
+        Assert.assertNotNull(fs);
+        Assert.assertTrue("A mount should appear as symlink", fs.isSymlink());
+  }
+      
+  @Test
+  public void testFileStatusOnMountLink() throws IOException {
+    Assert.assertTrue("Slash should appear as dir", 
+        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.assertEquals(fcView.getLinkTarget(new Path("/user")),
+        (new Path(targetTestRoot,"user")));
+    Assert.assertEquals(fcView.getLinkTarget(new Path("/data")),
+        (new Path(targetTestRoot,"data")));
+    Assert.assertEquals(
+        fcView.getLinkTarget(new Path("/internalDir/linkToDir2")),
+        (new Path(targetTestRoot,"dir2")));
+    Assert.assertEquals(
+        fcView.getLinkTarget(new Path("/internalDir/internalDir2/linkToDir3")),
+        (new Path(targetTestRoot,"dir3")));
+    Assert.assertEquals(fcView.getLinkTarget(new Path("/linkToAFile")),
+        (new Path(targetTestRoot,"aFile")));
+  }
+  
+  @Test(expected=IOException.class) 
+  public void testgetLinkTargetOnNonLink() throws IOException {
+    fcView.getLinkTarget(new Path("/internalDir/internalDir2"));
+  }
+  
+  /*
+   * Test resolvePath(p) 
+   * TODO In the tests below replace 
+   * fcView.getDefaultFileSystem().resolvePath() fcView.resolvePath()
+   */
+  
+  @Test
+  public void testResolvePathInternalPaths() throws IOException {
+    Assert.assertEquals(new Path("/"), fcView.getDefaultFileSystem().resolvePath(new Path("/")));
+    Assert.assertEquals(new Path("/internalDir"),
+                          fcView.getDefaultFileSystem().resolvePath(new Path("/internalDir")));
+  }
+  @Test
+  public void testResolvePathMountPoints() throws IOException {
+    Assert.assertEquals(new Path(targetTestRoot,"user"),
+                          fcView.getDefaultFileSystem().resolvePath(new Path("/user")));
+    Assert.assertEquals(new Path(targetTestRoot,"data"),
+        fcView.getDefaultFileSystem().resolvePath(new Path("/data")));
+    Assert.assertEquals(new Path(targetTestRoot,"dir2"),
+        fcView.getDefaultFileSystem().resolvePath(new Path("/internalDir/linkToDir2")));
+    Assert.assertEquals(new Path(targetTestRoot,"dir3"),
+        fcView.getDefaultFileSystem().resolvePath(new Path("/internalDir/internalDir2/linkToDir3")));
+
+  }
+  
+  @Test
+  public void testResolvePathThroughMountPoints() throws IOException {
+    FileContextTestHelper.createFile(fcView, "/user/foo");
+    Assert.assertEquals(new Path(targetTestRoot,"user/foo"),
+                          fcView.getDefaultFileSystem().resolvePath(new Path("/user/foo")));
+    
+    fcView.mkdir(
+        FileContextTestHelper.getTestRootPath(fcView, "/user/dirX"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertEquals(new Path(targetTestRoot,"user/dirX"),
+        fcView.getDefaultFileSystem().resolvePath(new Path("/user/dirX")));
+
+    
+    fcView.mkdir(
+        FileContextTestHelper.getTestRootPath(fcView, "/user/dirX/dirY"),
+        FileContext.DEFAULT_PERM, false);
+    Assert.assertEquals(new Path(targetTestRoot,"user/dirX/dirY"),
+        fcView.getDefaultFileSystem().resolvePath(new Path("/user/dirX/dirY")));
+  }
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathDanglingLink() throws IOException {
+      fcView.getDefaultFileSystem().resolvePath(new Path("/danglingLink"));
+  }
+  
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathMissingThroughMountPoints() throws IOException {
+    fcView.getDefaultFileSystem().resolvePath(new Path("/user/nonExisting"));
+  }
+  
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathMissingThroughMountPoints2() throws IOException {
+    fcView.mkdir(
+        FileContextTestHelper.getTestRootPath(fcView, "/user/dirX"),
+        FileContext.DEFAULT_PERM, false);
+    fcView.getDefaultFileSystem().resolvePath(new Path("/user/dirX/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 {
+    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("Delete of link to dir should succeed",
+        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("linkTODir2 should be a dir", 
+        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");
+  }
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java?rev=1100026&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java Fri May  6 02:11:31 2011
@@ -0,0 +1,89 @@
+/**
+ * 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.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);
+  }
+
+}