Browse Source

Merging r1129989 for HADOOP-7284 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/yahoo-merge@1131288 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 14 years ago
parent
commit
58fd867eb0

+ 2 - 0
CHANGES.txt

@@ -124,6 +124,8 @@ Trunk (unreleased changes)
     HADOOP-7336. TestFileContextResolveAfs will fail with default 
     test.build.data property. (jitendra)
 
+    HADOOP-7284 Trash and shell's rm does not work for viewfs (Sanjay Radia)
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 12 - 3
src/java/org/apache/hadoop/fs/Trash.java

@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.fs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_CHECKPOINT_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.text.DateFormat;
@@ -30,7 +35,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.*;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.StringUtils;
@@ -103,7 +107,12 @@ public class Trash extends Configured {
       Configuration conf) throws IOException {
     Path fullyResolvedPath = fs.resolvePath(p);
     Trash trash = new Trash(FileSystem.get(fullyResolvedPath.toUri(), conf), conf);
-    return trash.moveToTrash(fullyResolvedPath);
+    boolean success =  trash.moveToTrash(fullyResolvedPath);
+    if (success) {
+      System.out.println("Moved: '" + p + "' to trash at: " +
+          trash.getCurrentTrashDir() );
+    }
+    return success;
   }
   
   private Trash(Path home, Configuration conf) throws IOException {
@@ -141,7 +150,7 @@ public class Trash extends Configured {
     if (!fs.exists(path))                         // check that path exists
       throw new FileNotFoundException(path.toString());
 
-    String qpath = path.makeQualified(fs).toString();
+    String qpath = fs.makeQualified(path).toString();
 
     if (qpath.startsWith(trash.toString())) {
       return false;                               // already in trash

+ 1 - 3
src/java/org/apache/hadoop/fs/shell/Delete.java

@@ -74,7 +74,6 @@ class Delete extends FsCommand {
       // etc), then the path will just be deleted because moveToTrash returns
       // false and it falls thru to fs.delete.  this doesn't seem right
       if (moveToTrash(item)) {
-        out.println("Moved to trash: " + item);
         return;
       }
       if (!item.fs.delete(item.path, deleteDirs)) {
@@ -86,8 +85,7 @@ class Delete extends FsCommand {
     private boolean moveToTrash(PathData item) throws IOException {
       boolean success = false;
       if (!skipTrash) {
-        Trash trash = new Trash(item.fs, getConf());
-        success = (trash.isEnabled() && trash.moveToTrash(item.path));
+        success = Trash.moveToAppropriateTrash(item.fs, item.path, getConf());
       }
       return success;
     }

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

@@ -116,8 +116,7 @@ class ChRootedFileSystem extends FileSystem {
           chRootPathPart.toString().substring(1));
 
     workingDir = getHomeDirectory();
-    // We don't use the wd of the myFs,  (lets set it to root anyway)
-    myFs.setWorkingDirectory(chRootPathPart);
+    // We don't use the wd of the myFs
   }
   
   /** 

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

@@ -67,4 +67,51 @@ public class ConfigUtil {
     addLink( conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, 
         src, target);   
   }
+  
+  /**
+   * Add config variable for homedir for default mount table
+   * @param conf - add to this conf
+   * @param homedir - the home dir path starting with slash
+   */
+  public static void setHomeDirConf(final Configuration conf,
+      final String homedir) {
+    setHomeDirConf(  conf,
+        Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE,   homedir);
+  }
+  
+  /**
+   * Add config variable for homedir the specified mount table
+   * @param conf - add to this conf
+   * @param homedir - the home dir path starting with slash
+   */
+  public static void setHomeDirConf(final Configuration conf,
+              final String mountTableName, final String homedir) {
+    if (!homedir.startsWith("/")) {
+      throw new IllegalArgumentException("Home dir should start with /:"
+          + homedir);
+    }
+    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
+        Constants.CONFIG_VIEWFS_HOMEDIR, homedir);
+  }
+  
+  /**
+   * Get the value of the home dir conf value for default mount table
+   * @param conf - from this conf
+   * @return home dir value, null if variable is not in conf
+   */
+  public static String getHomeDirValue(final Configuration conf) {
+    return getHomeDirValue(conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE);
+  }
+  
+  /**
+   * Get the value of the home dir conf value for specfied mount table
+   * @param conf - from this conf
+   * @param mountTableName - the mount table
+   * @return home dir value, null if variable is not in conf
+   */
+  public static String getHomeDirValue(final Configuration conf, 
+      final String mountTableName) {
+    return conf.get(getConfigViewFsPrefix(mountTableName) + "." +
+        Constants.CONFIG_VIEWFS_HOMEDIR);
+  }
 }

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

@@ -30,6 +30,12 @@ public interface Constants {
    * Prefix for the config variable prefix for the ViewFs mount-table
    */
   public static final String CONFIG_VIEWFS_PREFIX = "fs.viewfs.mounttable";
+ 
+  /**
+   * Prefix for the home dir for the mount table - if not specified
+   * then the hadoop default value (/user) is used.
+   */
+  public static final String CONFIG_VIEWFS_HOMEDIR = "homedir";
   
   /**
    * Config variable name for the default mount table.

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

@@ -62,6 +62,8 @@ abstract class InodeTree<T> {
   
   final INodeDir<T> root; // the root of the mount table
   
+  final String homedirPrefix; // the homedir config value for this mount table
+  
   List<MountPoint<T>> mountPoints = new ArrayList<MountPoint<T>>();
   
   
@@ -298,6 +300,7 @@ abstract class InodeTree<T> {
     if (vName == null) {
       vName = Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE;
     }
+    homedirPrefix = ConfigUtil.getHomeDirValue(config, vName);
     root = new INodeDir<T>("/", UserGroupInformation.getCurrentUser());
     root.InodeDirFs = getTargetFileSystem(root);
     root.isRoot = true;
@@ -319,6 +322,9 @@ abstract class InodeTree<T> {
         } else if (src.startsWith(linkMergePrefix)) { // A merge link
           isMergeLink = true;
           src = src.substring(linkMergePrefix.length());
+        } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) {
+          // ignore - we set home dir from config
+          continue;
         } else {
           throw new IOException(
           "ViewFs: Cannot initialize: Invalid entry in Mount table in config: "+ 
@@ -442,4 +448,13 @@ abstract class InodeTree<T> {
   List<MountPoint<T>> getMountPoints() { 
     return mountPoints;
   }
+  
+  /**
+   * 
+   * @return home dir value from mount table; null if no config value
+   * was found.
+   */
+  String getHomeDirPrefixValue() {
+    return homedirPrefix;
+  }
 }

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

@@ -90,6 +90,7 @@ public class ViewFileSystem extends FileSystem {
   private Path workingDir;
   Configuration config;
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
+  Path homeDir = null;
   
   /**
    * Prohibits names which contain a ".", "..", ":" or "/" 
@@ -157,8 +158,6 @@ public class ViewFileSystem extends FileSystem {
     final String authority = theUri.getAuthority();
     try {
       myUri = new URI(FsConstants.VIEWFS_SCHEME, authority, "/", null, null);
-      workingDir =
-        this.makeQualified(new Path("/user/" + ugi.getShortUserName()));
       fsState = new InodeTree<FileSystem>(conf, authority) {
 
         @Override
@@ -184,6 +183,7 @@ public class ViewFileSystem extends FileSystem {
           // return MergeFs.createMergeFs(mergeFsURIList, config);
         }
       };
+      workingDir = this.getHomeDirectory();
     } catch (URISyntaxException e) {
       throw new IOException("URISyntax exception: " + theUri);
     }
@@ -234,6 +234,19 @@ public class ViewFileSystem extends FileSystem {
     return res.targetFileSystem.resolvePath(res.remainingPath);
   }
   
+  @Override
+  public Path getHomeDirectory() {
+    if (homeDir == null) {
+      String base = fsState.getHomeDirPrefixValue();
+      if (base == null) {
+        base = "/user";
+      }
+      homeDir = 
+        this.makeQualified(new Path(base + "/" + ugi.getShortUserName()));
+    }
+    return homeDir;
+  }
+  
   @Override
   public Path getWorkingDirectory() {
     return workingDir;

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

@@ -146,6 +146,7 @@ public class ViewFs extends AbstractFileSystem {
   final UserGroupInformation ugi; // the user/group of user who created mtable
   final Configuration config;
   InodeTree<AbstractFileSystem> fsState;  // the fs state; ie the mount table
+  Path homeDir = null;
   
   static AccessControlException readOnlyMountTable(final String operation,
       final String p) {
@@ -234,6 +235,19 @@ public class ViewFs extends AbstractFileSystem {
     return -1;
   }
  
+  @Override
+  public Path getHomeDirectory() {
+    if (homeDir == null) {
+      String base = fsState.getHomeDirPrefixValue();
+      if (base == null) {
+        base = "/user";
+      }
+      homeDir = 
+        this.makeQualified(new Path(base + "/" + ugi.getShortUserName()));
+    }
+    return homeDir;
+  }
+  
   @Override
   public Path resolvePath(final Path f) throws FileNotFoundException,
           AccessControlException, UnresolvedLinkException, IOException {

+ 52 - 22
src/test/core/org/apache/hadoop/fs/TestTrash.java

@@ -24,7 +24,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
-import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -57,10 +56,10 @@ public class TestTrash extends TestCase {
   }
 
   // check that the specified file is in Trash
-  protected static void checkTrash(FileSystem fs, Path trashRoot,
+  protected static void checkTrash(FileSystem trashFs, Path trashRoot,
       Path path) throws IOException {
     Path p = new Path(trashRoot+"/"+ path.toUri().getPath());
-    assertTrue(fs.exists(p));
+    assertTrue("Could not find file in trash: "+ p , trashFs.exists(p));
   }
   
   // counts how many instances of the file are in the Trash
@@ -89,11 +88,33 @@ public class TestTrash extends TestCase {
     Path p = new Path(trashRoot+"/"+ new Path(pathname).getName());
     assertTrue(!fs.exists(p));
   }
-
+  
+  /**
+   * Test trash for the shell's delete command for the file system fs
+   * @param fs
+   * @param base - the base path where files are created
+   * @throws IOException
+   */
   protected static void trashShell(final FileSystem fs, final Path base)
-      throws IOException {
+  throws IOException {
     Configuration conf = new Configuration();
-    conf.set("fs.default.name", fs.getUri().toString());
+    conf.set("fs.defaultFS", fs.getUri().toString());
+    trashShell(conf, base, null, null);
+  }
+
+  /**
+   * 
+   * Test trash for the shell's delete command for the default file system
+   * specified in the paramter conf
+   * @param conf 
+   * @param base - the base path where files are created
+   * @param trashRoot - the expected place where the trashbin resides
+   * @throws IOException
+   */
+  public static void trashShell(final Configuration conf, final Path base,
+      FileSystem trashRootFs, Path trashRoot)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
 
     conf.set(FS_TRASH_INTERVAL_KEY, "0"); // disabled
     assertFalse(new Trash(conf).isEnabled());
@@ -103,7 +124,12 @@ public class TestTrash extends TestCase {
 
     FsShell shell = new FsShell();
     shell.setConf(conf);
-    Path trashRoot = null;
+    if (trashRoot == null) {
+      trashRoot = shell.getCurrentTrashDir();
+    }
+    if (trashRootFs == null) {
+      trashRootFs = fs;
+    }
 
     // First create a new directory with mkdirs
     Path myPath = new Path(base, "test/mkdirs");
@@ -143,8 +169,8 @@ public class TestTrash extends TestCase {
       }
       assertTrue(val == 0);
 
-      trashRoot = shell.getCurrentTrashDir();
-      checkTrash(fs, trashRoot, myFile);
+ 
+      checkTrash(trashRootFs, trashRoot, fs.makeQualified(myFile));
     }
 
     // Verify that we can recreate the file
@@ -206,7 +232,7 @@ public class TestTrash extends TestCase {
     {
         Path toErase = new Path(trashRoot, "toErase");
         int retVal = -1;
-        writeFile(fs, toErase);
+        writeFile(trashRootFs, toErase);
         try {
           retVal = shell.run(new String[] {"-rm", toErase.toString()});
         } catch (Exception e) {
@@ -214,8 +240,8 @@ public class TestTrash extends TestCase {
                              e.getLocalizedMessage());
         }
         assertTrue(retVal == 0);
-        checkNotInTrash (fs, trashRoot, toErase.toString());
-        checkNotInTrash (fs, trashRoot, toErase.toString()+".1");
+        checkNotInTrash (trashRootFs, trashRoot, toErase.toString());
+        checkNotInTrash (trashRootFs, trashRoot, toErase.toString()+".1");
     }
 
     // simulate Trash removal
@@ -233,7 +259,7 @@ public class TestTrash extends TestCase {
     }
 
     // verify that after expunging the Trash, it really goes away
-    checkNotInTrash(fs, trashRoot, new Path(base, "test/mkdirs/myFile").toString());
+    checkNotInTrash(trashRootFs, trashRoot, new Path(base, "test/mkdirs/myFile").toString());
 
     // recreate directory and file
     mkdir(fs, myPath);
@@ -252,7 +278,7 @@ public class TestTrash extends TestCase {
                            e.getLocalizedMessage());
       }
       assertTrue(val == 0);
-      checkTrash(fs, trashRoot, myFile);
+      checkTrash(trashRootFs, trashRoot, myFile);
 
       args = new String[2];
       args[0] = "-rmr";
@@ -265,7 +291,7 @@ public class TestTrash extends TestCase {
                            e.getLocalizedMessage());
       }
       assertTrue(val == 0);
-      checkTrash(fs, trashRoot, myPath);
+      checkTrash(trashRootFs, trashRoot, myPath);
     }
 
     // attempt to remove parent of trash
@@ -281,7 +307,7 @@ public class TestTrash extends TestCase {
                            e.getLocalizedMessage());
       }
       assertEquals("exit code", 1, val);
-      assertTrue(fs.exists(trashRoot));
+      assertTrue(trashRootFs.exists(trashRoot));
     }
     
     // Verify skip trash option really works
@@ -299,7 +325,8 @@ public class TestTrash extends TestCase {
       int val = -1;
       try {
         // Clear out trash
-        assertEquals(0, shell.run(new String [] { "-expunge" } ));
+        assertEquals("-expunge failed", 
+            0, shell.run(new String [] { "-expunge" } ));
         
         val = shell.run(args);
         
@@ -307,7 +334,10 @@ public class TestTrash extends TestCase {
         System.err.println("Exception raised from Trash.run " +
             e.getLocalizedMessage());
       }
-      assertFalse(fs.exists(trashRoot)); // No new Current should be created
+      assertFalse("Expected TrashRoot (" + trashRoot + 
+          ") to exist in file system:"
+          + trashRootFs.getUri(), 
+          trashRootFs.exists(trashRoot)); // No new Current should be created
       assertFalse(fs.exists(myFile));
       assertTrue(val == 0);
     }
@@ -335,7 +365,7 @@ public class TestTrash extends TestCase {
             e.getLocalizedMessage());
       }
 
-      assertFalse(fs.exists(trashRoot)); // No new Current should be created
+      assertFalse(trashRootFs.exists(trashRoot)); // No new Current should be created
       assertFalse(fs.exists(myPath));
       assertFalse(fs.exists(myFile));
       assertTrue(val == 0);
@@ -420,7 +450,7 @@ public class TestTrash extends TestCase {
   public void testNonDefaultFS() throws IOException {
     Configuration conf = new Configuration();
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
-    conf.set("fs.default.name", "invalid://host/bar/foo");
+    conf.set("fs.defaultFS", "invalid://host/bar/foo");
     trashNonDefaultFS(conf);
   }
   
@@ -438,7 +468,7 @@ public class TestTrash extends TestCase {
     emptierThread.start();
 
     FileSystem fs = FileSystem.getLocal(conf);
-    conf.set("fs.default.name", fs.getUri().toString());
+    conf.set("fs.defaultFS", fs.getUri().toString());
     FsShell shell = new FsShell();
     shell.setConf(conf);
     shell.init();
@@ -521,7 +551,7 @@ public class TestTrash extends TestCase {
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
     FileSystem fs = FileSystem.getLocal(conf);
     
-    conf.set("fs.default.name", fs.getUri().toString());
+    conf.set("fs.defaultFS", fs.getUri().toString());
     conf.set(FS_TRASH_INTERVAL_KEY, "10"); //minutes..
     FsShell shell = new FsShell();
     shell.setConf(conf);

+ 18 - 1
src/test/core/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 
@@ -287,5 +288,21 @@ public class TestChRootedFileSystem {
     Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
 
   }
- 
+  
+  /*
+   * Test resolvePath(p) 
+   */
+  
+  @Test
+  public void testResolvePath() throws IOException {
+    Assert.assertEquals(chrootedTo, fSys.resolvePath(new Path("/"))); 
+    FileSystemTestHelper.createFile(fSys, "/foo");
+    Assert.assertEquals(new Path(chrootedTo, "foo"),
+        fSys.resolvePath(new Path("/foo"))); 
+  }
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathNonExisting() throws IOException {
+      fSys.resolvePath(new Path("/nonExisting"));
+  }
 }

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.EnumSet;
@@ -288,5 +289,22 @@ public class TestChRootedFs {
     Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());
 
   }
+  
+  /*
+   * Test resolvePath(p) 
+   */
+  
+  @Test
+  public void testResolvePath() throws IOException {
+    Assert.assertEquals(chrootedTo, fc.getDefaultFileSystem().resolvePath(new Path("/"))); 
+    FileContextTestHelper.createFile(fc, "/foo");
+    Assert.assertEquals(new Path(chrootedTo, "foo"),
+        fc.getDefaultFileSystem().resolvePath(new Path("/foo"))); 
+  }
+
+  @Test(expected=FileNotFoundException.class) 
+  public void testResolvePathNonExisting() throws IOException {
+      fc.getDefaultFileSystem().resolvePath(new Path("/nonExisting"));
+  }
  
 }

+ 2 - 1
src/test/core/org/apache/hadoop/fs/viewfs/TestFSMainOperationsLocalFileSystem.java

@@ -37,7 +37,8 @@ public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTes
   public void setUp() throws Exception {
     Configuration conf = new Configuration();
     fcTarget = FileSystem.getLocal(conf);
-    fSys = ViewFileSystemTestSetup.setupForViewFs(fcTarget);
+    fSys = ViewFileSystemTestSetup.setupForViewFs(
+        ViewFileSystemTestSetup.configWithViewfsScheme(), fcTarget);
     super.setUp();
   }
   

+ 112 - 0
src/test/core/org/apache/hadoop/fs/viewfs/TestViewFsTrash.java

@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.viewfs;
+
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TestTrash;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mortbay.log.Log;
+
+public class TestViewFsTrash {
+  FileSystem fsTarget;  // the target file system - the mount will point here
+  FileSystem fsView;
+  Path targetTestRoot;
+  Configuration conf;
+
+  static class TestLFS extends LocalFileSystem {
+    Path home;
+    TestLFS() throws IOException {
+      this(new Path(FileSystemTestHelper.TEST_ROOT_DIR));
+    }
+    TestLFS(Path home) throws IOException {
+      super();
+      this.home = home;
+    }
+    public Path getHomeDirectory() {
+      return home;
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fsTarget = FileSystem.getLocal(new Configuration());
+    targetTestRoot = FileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
+    // In case previous test was killed before cleanup
+    fsTarget.delete(targetTestRoot, true);
+    // cleanup trash from previous run if it stuck around
+    fsTarget.delete(new Path(fsTarget.getHomeDirectory(), ".Trash/Current"),
+        true);
+    
+    fsTarget.mkdirs(targetTestRoot);
+    fsTarget.mkdirs(new Path(targetTestRoot,"dir1"));
+    
+    
+    // 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 = ViewFileSystemTestSetup.configWithViewfsScheme();
+    
+    // create a link for home directory so that trash path works
+    // set up viewfs's home dir root to point to home dir root on target
+    // But home dir is different on linux, mac etc.
+    // Figure it out by calling home dir on target
+    
+   String homeDir = fsTarget.getHomeDirectory().toUri().getPath();
+   int indexOf2ndSlash = homeDir.indexOf('/', 1);
+   String homeDirRoot = homeDir.substring(0, indexOf2ndSlash);
+   ConfigUtil.addLink(conf, homeDirRoot,
+       fsTarget.makeQualified(new Path(homeDirRoot)).toUri()); 
+   ConfigUtil.setHomeDirConf(conf, homeDirRoot);
+   Log.info("Home dir base " + homeDirRoot);
+    
+    fsView = ViewFileSystemTestSetup.setupForViewFs(conf, fsTarget);
+    
+    // set working dir so that relative paths
+    //fsView.setWorkingDirectory(new Path(fsTarget.getWorkingDirectory().toUri().getPath()));
+    conf.set("fs.defaultFS", FsConstants.VIEWFS_URI.toString());
+  }
+ 
+
+  @After
+  public void tearDown() throws Exception {
+    fsTarget.delete(targetTestRoot, true);
+    fsTarget.delete(new Path(fsTarget.getHomeDirectory(), ".Trash/Current"),
+        true);
+  }
+
+  
+  @Test
+  public void testTrash() throws IOException {
+    TestTrash.trashShell(conf, FileSystemTestHelper.getTestRootPath(fsView),
+        fsTarget, new Path(fsTarget.getHomeDirectory(), ".Trash/Current"));
+  }
+  
+}

+ 14 - 17
src/test/core/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java

@@ -32,7 +32,8 @@ import org.apache.hadoop.fs.viewfs.ConfigUtil;
  * 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>.
+ * Since viewFs has no built-in wd, its wd is /user/<username> 
+ *          (or /User/<username> on mac)
  * 
  * We set a viewFileSystems with mount point for 
  * /<firstComponent>" pointing to the target fs's  testdir 
@@ -45,32 +46,28 @@ public class ViewFileSystemTestSetup {
    * @return return the ViewFS File context to be used for tests
    * @throws Exception
    */
-  static public FileSystem setupForViewFs(FileSystem fsTarget) throws Exception {
+  static public FileSystem setupForViewFs(Configuration conf, 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());
+    // Now set up a link from viewfs to targetfs for the first component of
+    // path of testdir. For example, if testdir is /user/<userid>/xx then
+    // a link from /user to targetfs://user.
+    
+    String testDir = FileSystemTestHelper.getTestRootPath(fsTarget).toUri().getPath();
+    int indexOf2ndSlash = testDir.indexOf('/', 1);
+    String testDirFirstComponent = testDir.substring(0, indexOf2ndSlash);
+    
+    
+    ConfigUtil.addLink(conf, testDirFirstComponent,
+        fsTarget.makeQualified(new Path(testDirFirstComponent)).toUri()); 
     
     FileSystem fsView = FileSystem.get(FsConstants.VIEWFS_URI, conf);
     //System.out.println("SRCOfTests = "+ getTestRootPath(fs, "test"));