Forráskód Böngészése

commit 56c6784b8a7589f5cbccba09d393ea83c03fc715
Author: Sanjay Radia <sradia@yahoo-inc.com>
Date: Mon Dec 20 18:12:53 2010 -0800

Bugzilla 4219520 - fixed bug in viewfs to allow non-null authority.
Hence URIs like viewfs://clusterXMountTableName/ work.
Typical usage is where each cluster has its own mount table named after the cluster.
That is authority in this URI will be set to the cluster name.
This feature was always there. But there was a bug.
- Note as part of this the AbstractFileSystem#getUri was fixed to allow optional authorites. (was needed for the
main fix)

+++ b/YAHOO-CHANGES.txt
+
+ - fix bug in viewfs to allow authority in viewfs uri as in viewfs://mountTableName/
+ This simplifies the usage sceanario where each cluster has its own mount table where the mount-table is
+ named by say the cluster name. This allows the convenience of accessing a particular cluster
+ via viewfs://clusterName/path
+ Note this also allows one to access a private view via viewfs://fooViewMountTableName/
+ Note this feature was in there; this is merely a bug fix.


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

Owen O'Malley 14 éve
szülő
commit
345a76ede3

+ 10 - 9
src/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -255,18 +255,19 @@ public abstract class AbstractFileSystem {
               + " is not valid");
               + " is not valid");
     }
     }
     String authority = uri.getAuthority();
     String authority = uri.getAuthority();
-    if (!authorityNeeded) {
-      if (authority != null) {
-        throw new HadoopIllegalArgumentException("Scheme with non-null authority: "
-            + uri);
-      }
-      return new URI(supportedScheme + ":///");
-    }
     if (authority == null) {
     if (authority == null) {
-      throw new HadoopIllegalArgumentException("Uri without authority: " + uri);
+       if (authorityNeeded) {
+         throw new HadoopIllegalArgumentException("Uri without authority: " + uri);
+       } else {
+         return new URI(supportedScheme + ":///");
+       }   
     }
     }
+    // authority is non null  - AuthorityNeeded may be true or false.
     int port = uri.getPort();
     int port = uri.getPort();
-    port = port == -1 ? defaultPort : port;
+    port = (port == -1 ? defaultPort : port);
+    if (port == -1) { // no port supplied and default port is not specified
+      return new URI(supportedScheme, authority, "/", null);
+    }
     return new URI(supportedScheme + "://" + uri.getHost() + ":" + port);
     return new URI(supportedScheme + "://" + uri.getHost() + ":" + port);
   }
   }
   
   

+ 5 - 2
src/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java

@@ -107,8 +107,11 @@ class ChRootedFileSystem extends FileSystem {
      * this issue but if we were to make it external then this needs
      * this issue but if we were to make it external then this needs
      * to be resolved.
      * to be resolved.
      */
      */
-    myUri = new URI(myFs.getUri().toString() //+ Path.SEPARATOR
-        + chRootPathPart.toString().substring(1));
+    // Handle the two cases:
+    //              scheme:/// and scheme://authority/
+    myUri = new URI(myFs.getUri().toString() + 
+        (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
+          chRootPathPart.toString().substring(1));
 
 
     workingDir = getHomeDirectory();
     workingDir = getHomeDirectory();
     // We don't use the wd of the myFs bu set it to root.
     // We don't use the wd of the myFs bu set it to root.

+ 5 - 1
src/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java

@@ -94,7 +94,11 @@ class ChRootedFs extends AbstractFileSystem {
      * this issue but if we were to make it external then this needs
      * this issue but if we were to make it external then this needs
      * to be resolved.
      * to be resolved.
      */
      */
-    myUri = new URI(myFs.getUri().toString() + chRootPathPart.toString().substring(1));
+    // Handle the two cases:
+    //              scheme:/// and scheme://authority/
+    myUri = new URI(myFs.getUri().toString() + 
+        (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
+          chRootPathPart.toString().substring(1));
     super.checkPath(theRoot);
     super.checkPath(theRoot);
   }
   }
   
   

+ 15 - 11
src/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -60,6 +60,7 @@ public class ViewFileSystem extends FileSystem {
   
   
   final long creationTime; // of the the mount table
   final long creationTime; // of the the mount table
   final UserGroupInformation ugi; // the user/group of user who created mtable
   final UserGroupInformation ugi; // the user/group of user who created mtable
+  URI myUri;
   private Path workingDir;
   private Path workingDir;
   Configuration config;
   Configuration config;
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
@@ -125,12 +126,13 @@ public class ViewFileSystem extends FileSystem {
       throws IOException {
       throws IOException {
     super.initialize(theUri, conf);
     super.initialize(theUri, conf);
     setConf(conf);
     setConf(conf);
-    workingDir =
-      this.makeQualified(new Path("/user/" + ugi.getShortUserName()));
     config = conf;
     config = conf;
     // Now build  client side view (i.e. client side mount table) from config.
     // Now build  client side view (i.e. client side mount table) from config.
     final String authority = theUri.getAuthority();
     final String authority = theUri.getAuthority();
     try {
     try {
+      myUri = new URI(FsConstants.VIEWFS_SCHEME, authority, "/", null, null);
+      workingDir =
+        this.makeQualified(new Path("/user/" + ugi.getShortUserName()));
       fsState = new InodeTree<FileSystem>(conf, authority) {
       fsState = new InodeTree<FileSystem>(conf, authority) {
 
 
         @Override
         @Override
@@ -145,7 +147,7 @@ public class ViewFileSystem extends FileSystem {
         protected
         protected
         FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
         FileSystem getTargetFileSystem(final INodeDir<FileSystem> dir)
           throws URISyntaxException {
           throws URISyntaxException {
-          return new InternalDirOfViewFs(dir, creationTime, ugi);
+          return new InternalDirOfViewFs(dir, creationTime, ugi, myUri);
         }
         }
 
 
         @Override
         @Override
@@ -172,7 +174,7 @@ public class ViewFileSystem extends FileSystem {
   ViewFileSystem(final URI theUri, final Configuration conf)
   ViewFileSystem(final URI theUri, final Configuration conf)
     throws IOException {
     throws IOException {
     this();
     this();
-    initialize(FsConstants.VIEWFS_URI, conf);
+    initialize(theUri, conf);
   }
   }
   
   
   /**
   /**
@@ -192,7 +194,7 @@ public class ViewFileSystem extends FileSystem {
 
 
   @Override
   @Override
   public URI getUri() {
   public URI getUri() {
-    return FsConstants.VIEWFS_URI;
+    return myUri;
   }
   }
   
   
   /**
   /**
@@ -446,12 +448,14 @@ public class ViewFileSystem extends FileSystem {
     final InodeTree.INodeDir<FileSystem>  theInternalDir;
     final InodeTree.INodeDir<FileSystem>  theInternalDir;
     final long creationTime; // of the the mount table
     final long creationTime; // of the the mount table
     final UserGroupInformation ugi; // the user/group of user who created mtable
     final UserGroupInformation ugi; // the user/group of user who created mtable
+    final URI myUri;
     
     
     public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
     public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
-        final long cTime, final UserGroupInformation ugi)
+        final long cTime, final UserGroupInformation ugi, URI uri)
       throws URISyntaxException {
       throws URISyntaxException {
+      myUri = uri;
       try {
       try {
-        initialize(FsConstants.VIEWFS_URI, new Configuration());
+        initialize(myUri, new Configuration());
       } catch (IOException e) {
       } catch (IOException e) {
         throw new RuntimeException("Cannot occur");
         throw new RuntimeException("Cannot occur");
       }
       }
@@ -469,7 +473,7 @@ public class ViewFileSystem extends FileSystem {
     
     
     @Override
     @Override
     public URI getUri() {
     public URI getUri() {
-      return FsConstants.VIEWFS_URI;
+      return myUri;
     }
     }
 
 
     @Override
     @Override
@@ -533,7 +537,7 @@ public class ViewFileSystem extends FileSystem {
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
 
 
           new Path(theInternalDir.fullPath).makeQualified(
           new Path(theInternalDir.fullPath).makeQualified(
-              FsConstants.VIEWFS_URI, null));
+              myUri, null));
     }
     }
     
     
 
 
@@ -554,13 +558,13 @@ public class ViewFileSystem extends FileSystem {
             ugi.getUserName(), ugi.getGroupNames()[0],
             ugi.getUserName(), ugi.getGroupNames()[0],
             link.getTargetLink(),
             link.getTargetLink(),
             new Path(inode.fullPath).makeQualified(
             new Path(inode.fullPath).makeQualified(
-                FsConstants.VIEWFS_URI, null));
+                myUri, null));
         } else {
         } else {
           result[i++] = new FileStatus(0, true, 0, 0,
           result[i++] = new FileStatus(0, true, 0, 0,
             creationTime, creationTime, PERMISSION_RRR,
             creationTime, creationTime, PERMISSION_RRR,
             ugi.getUserName(), ugi.getGroupNames()[0],
             ugi.getUserName(), ugi.getGroupNames()[0],
             new Path(inode.fullPath).makeQualified(
             new Path(inode.fullPath).makeQualified(
-                FsConstants.VIEWFS_URI, null));
+                myUri, null));
         }
         }
       }
       }
       return result;
       return result;

+ 9 - 7
src/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -265,7 +265,7 @@ public class ViewFs extends AbstractFileSystem {
       protected
       protected
       AbstractFileSystem getTargetFileSystem(
       AbstractFileSystem getTargetFileSystem(
           final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
           final INodeDir<AbstractFileSystem> dir) throws URISyntaxException {
-        return new InternalDirOfViewFs(dir, creationTime, ugi);
+        return new InternalDirOfViewFs(dir, creationTime, ugi, getUri());
       }
       }
 
 
       @Override
       @Override
@@ -603,14 +603,16 @@ public class ViewFs extends AbstractFileSystem {
     final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
     final InodeTree.INodeDir<AbstractFileSystem>  theInternalDir;
     final long creationTime; // of the the mount table
     final long creationTime; // of the the mount table
     final UserGroupInformation ugi; // the user/group of user who created mtable
     final UserGroupInformation ugi; // the user/group of user who created mtable
+    final URI myUri; // the URI of the outer ViewFs
     
     
     public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
     public InternalDirOfViewFs(final InodeTree.INodeDir<AbstractFileSystem> dir,
-        final long cTime, final UserGroupInformation ugi)
+        final long cTime, final UserGroupInformation ugi, final URI uri)
       throws URISyntaxException {
       throws URISyntaxException {
       super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
       super(FsConstants.VIEWFS_URI, FsConstants.VIEWFS_SCHEME, false, -1);
       theInternalDir = dir;
       theInternalDir = dir;
       creationTime = cTime;
       creationTime = cTime;
       this.ugi = ugi;
       this.ugi = ugi;
+      myUri = uri;
     }
     }
 
 
     static private void checkPathIsSlash(final Path f) throws IOException {
     static private void checkPathIsSlash(final Path f) throws IOException {
@@ -659,7 +661,7 @@ public class ViewFs extends AbstractFileSystem {
       return new FileStatus(0, true, 0, 0, creationTime, creationTime, 
       return new FileStatus(0, true, 0, 0, creationTime, creationTime, 
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           new Path(theInternalDir.fullPath).makeQualified(
           new Path(theInternalDir.fullPath).makeQualified(
-              FsConstants.VIEWFS_URI, null));
+              myUri, null));
     }
     }
     
     
     @Override
     @Override
@@ -680,12 +682,12 @@ public class ViewFs extends AbstractFileSystem {
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             inodelink.getTargetLink(),
             inodelink.getTargetLink(),
             new Path(inode.fullPath).makeQualified(
             new Path(inode.fullPath).makeQualified(
-                FsConstants.VIEWFS_URI, null));
+                myUri, null));
       } else {
       } else {
         result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
         result = new FileStatus(0, true, 0, 0, creationTime, creationTime,
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           new Path(inode.fullPath).makeQualified(
           new Path(inode.fullPath).makeQualified(
-              FsConstants.VIEWFS_URI, null));
+              myUri, null));
       }
       }
       return result;
       return result;
     }
     }
@@ -725,13 +727,13 @@ public class ViewFs extends AbstractFileSystem {
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             link.getTargetLink(),
             link.getTargetLink(),
             new Path(inode.fullPath).makeQualified(
             new Path(inode.fullPath).makeQualified(
-                FsConstants.VIEWFS_URI, null));
+                myUri, null));
         } else {
         } else {
           result[i++] = new FileStatus(0, true, 0, 0,
           result[i++] = new FileStatus(0, true, 0, 0,
             creationTime, creationTime,
             creationTime, creationTime,
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
             new Path(inode.fullPath).makeQualified(
             new Path(inode.fullPath).makeQualified(
-                FsConstants.VIEWFS_URI, null));
+                myUri, null));
         }
         }
       }
       }
       return result;
       return result;

+ 11 - 17
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java

@@ -1,20 +1,4 @@
-/**
- * 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;
 package org.apache.hadoop.fs.viewfs;
 
 
 
 
@@ -26,6 +10,16 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 
 
 
 
+
+/**
+ * 
+ * Test the ViewFileSystemBaseTest 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 TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
 public class TestViewFileSystemLocalFileSystem extends ViewFileSystemBaseTest {
 
 
 
 

+ 62 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java

@@ -0,0 +1,62 @@
+
+package org.apache.hadoop.fs.viewfs;
+
+
+import java.net.URI;
+
+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.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 TestViewFileSystemWithAuthorityLocalFileSystem extends ViewFileSystemBaseTest {
+  URI schemeWithAuthority;
+
+  @Before
+  public void setUp() throws Exception {
+    // create the test root on local_fs
+    fsTarget = FileSystem.getLocal(new Configuration());
+    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);
+    fsView = FileSystem.get(schemeWithAuthority, conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(FileSystemTestHelper.getTestRootPath(fsTarget), true);
+    super.tearDown();
+  }
+ 
+  @Test
+  public void testBasicPaths() {
+    Assert.assertEquals(schemeWithAuthority,
+        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(schemeWithAuthority, null),
+        fsView.makeQualified(new Path("/foo/bar")));
+  }
+}

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

@@ -0,0 +1,60 @@
+
+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")));
+  }
+}

+ 4 - 1
src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java

@@ -62,6 +62,8 @@ public class ViewFileSystemBaseTest {
   FileSystem fsView;  // the view file system - the mounts are here
   FileSystem fsView;  // the view file system - the mounts are here
   FileSystem fsTarget;  // the target file system - the mount will point here
   FileSystem fsTarget;  // the target file system - the mount will point here
   Path targetTestRoot;
   Path targetTestRoot;
+  Configuration conf;
+
 
 
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
@@ -83,7 +85,7 @@ public class ViewFileSystemBaseTest {
     
     
     // Set up the defaultMT in the config with our mount point links
     // Set up the defaultMT in the config with our mount point links
     //Configuration conf = new Configuration();
     //Configuration conf = new Configuration();
-    Configuration conf = ViewFileSystemTestSetup.configWithViewfsScheme();
+    conf = ViewFileSystemTestSetup.configWithViewfsScheme();
     ConfigUtil.addLink(conf, "/user", new Path(targetTestRoot,"user").toUri());
     ConfigUtil.addLink(conf, "/user", new Path(targetTestRoot,"user").toUri());
     ConfigUtil.addLink(conf, "/user2", 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, "/data", new Path(targetTestRoot,"data").toUri());
@@ -119,6 +121,7 @@ public class ViewFileSystemBaseTest {
         new Path("/foo/bar").makeQualified(FsConstants.VIEWFS_URI, null),
         new Path("/foo/bar").makeQualified(FsConstants.VIEWFS_URI, null),
         fsView.makeQualified(new Path("/foo/bar")));
         fsView.makeQualified(new Path("/foo/bar")));
   }
   }
+
   
   
   /** 
   /** 
    * Test modify operations (create, mkdir, delete, etc) 
    * Test modify operations (create, mkdir, delete, etc) 

+ 6 - 1
src/test/core/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java

@@ -25,6 +25,7 @@ import static org.apache.hadoop.fs.FileContextTestHelper.isFile;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.URI;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
@@ -64,6 +65,9 @@ public class ViewFsBaseTest {
   FileContext fcView; // the view file system - the mounts are here
   FileContext fcView; // the view file system - the mounts are here
   FileContext fcTarget; // the target file system - the mount will point here
   FileContext fcTarget; // the target file system - the mount will point here
   Path targetTestRoot;
   Path targetTestRoot;
+  Configuration conf;
+  FileContext xfcViewWithAuthority; // same as fsView but with authority
+  URI schemeWithAuthority;
 
 
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
@@ -89,7 +93,7 @@ public class ViewFsBaseTest {
     // in the test root
     // in the test root
     
     
     // Set up the defaultMT in the config with our mount point links
     // Set up the defaultMT in the config with our mount point links
-    Configuration conf = new Configuration();
+    conf = new Configuration();
     ConfigUtil.addLink(conf, "/user",
     ConfigUtil.addLink(conf, "/user",
         new Path(targetTestRoot,"user").toUri());
         new Path(targetTestRoot,"user").toUri());
     ConfigUtil.addLink(conf, "/user2",
     ConfigUtil.addLink(conf, "/user2",
@@ -106,6 +110,7 @@ public class ViewFsBaseTest {
         new Path(targetTestRoot,"aFile").toUri());
         new Path(targetTestRoot,"aFile").toUri());
     
     
     fcView = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
     fcView = FileContext.getFileContext(FsConstants.VIEWFS_URI, conf);
+    // Also try viewfs://default/    - note authority is name of mount table
   }
   }
 
 
   @After
   @After