Browse Source

HADOOP-5086. Use the appropriate FileSystem for trash URIs.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.19@737222 13f79535-47bb-0310-9956-ffa450edef68
Christopher Douglas 16 years ago
parent
commit
d0037d95f4

+ 2 - 0
CHANGES.txt

@@ -69,6 +69,8 @@ Release 0.19.1 - Unreleased
     HADOOP-5009. DataNode#shutdown sometimes leaves data block scanner
     verification log unclosed. (hairong)
 
+    HADOOP-5086. Use the appropriate FileSystem for trash URIs. (cdouglas)
+
 Release 0.19.0 - 2008-11-18
 
   INCOMPATIBLE CHANGES

+ 1 - 1
src/core/org/apache/hadoop/fs/FsShell.java

@@ -1043,7 +1043,7 @@ public class FsShell extends Configured implements Tool {
       throw new IOException("Cannot remove directory \"" + src +
                             "\", use -rmr instead");
     }
-    Trash trashTmp = new Trash(srcFs.getConf());
+    Trash trashTmp = new Trash(srcFs, getConf());
     if (trashTmp.moveToTrash(src)) {
       System.out.println("Moved to trash: " + src);
       return;

+ 16 - 6
src/core/org/apache/hadoop/fs/Trash.java

@@ -50,21 +50,31 @@ public class Trash extends Configured {
   private static final DateFormat CHECKPOINT = new SimpleDateFormat("yyMMddHHmm");
   private static final int MSECS_PER_MINUTE = 60*1000;
 
-  private FileSystem fs;
-  private Path trash;
-  private Path current;
-  private long interval;
+  private final FileSystem fs;
+  private final Path trash;
+  private final Path current;
+  private final long interval;
 
   /** Construct a trash can accessor.
    * @param conf a Configuration
    */
   public Trash(Configuration conf) throws IOException {
-    this(FileSystem.get(conf).getHomeDirectory(), conf);
+    this(FileSystem.get(conf), conf);
   }
 
-  private Trash(Path home, Configuration conf) throws IOException {
+  /**
+   * Construct a trash can accessor for the FileSystem provided.
+   */
+  public Trash(FileSystem fs, Configuration conf) throws IOException {
     super(conf);
+    this.fs = fs;
+    this.trash = new Path(fs.getHomeDirectory(), TRASH);
+    this.current = new Path(trash, CURRENT);
+    this.interval = conf.getLong("fs.trash.interval", 60) * MSECS_PER_MINUTE;
+  }
 
+  private Trash(Path home, Configuration conf) throws IOException {
+    super(conf);
     this.fs = home.getFileSystem(conf);
     this.trash = new Path(home, TRASH);
     this.current = new Path(trash, CURRENT);

+ 313 - 0
src/test/org/apache/hadoop/fs/TestTrash.java

@@ -0,0 +1,313 @@
+/**
+ * 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;
+
+
+import junit.framework.TestCase;
+import java.io.File;
+import java.io.IOException;
+import java.io.DataOutputStream;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Trash;
+import org.apache.hadoop.fs.LocalFileSystem;
+
+/**
+ * This class tests commands from Trash.
+ */
+public class TestTrash extends TestCase {
+
+  private final static Path TEST_DIR =
+    new Path(new File(System.getProperty("test.build.data","/tmp")
+          ).toURI().toString().replace(' ', '+'), "testTrash");
+
+  protected static Path writeFile(FileSystem fs, Path f) throws IOException {
+    DataOutputStream out = fs.create(f);
+    out.writeBytes("dhruba: " + f);
+    out.close();
+    assertTrue(fs.exists(f));
+    return f;
+  }
+
+  protected static Path mkdir(FileSystem fs, Path p) throws IOException {
+    assertTrue(fs.mkdirs(p));
+    assertTrue(fs.exists(p));
+    assertTrue(fs.getFileStatus(p).isDir());
+    return p;
+  }
+
+  // check that the specified file is in Trash
+  protected static void checkTrash(FileSystem fs, Path trashRoot,
+      String pathname) throws IOException {
+    Path p = new Path(trashRoot+"/"+new Path(pathname).getName());
+    assertTrue(fs.exists(p));
+  }
+
+  // check that the specified file is not in Trash
+  static void checkNotInTrash(FileSystem fs, Path trashRoot, String pathname)
+                              throws IOException {
+    Path p = new Path(trashRoot+"/"+ new Path(pathname).getName());
+    assertTrue(!fs.exists(p));
+  }
+
+  protected static void trashShell(final FileSystem fs, final Path base)
+      throws IOException {
+    Configuration conf = new Configuration();
+    conf.set("fs.trash.interval", "10"); // 10 minute
+    conf.set("fs.default.name", fs.getUri().toString());
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    Path trashRoot = null;
+
+    // First create a new directory with mkdirs
+    Path myPath = new Path(base, "test/mkdirs");
+    mkdir(fs, myPath);
+
+    // Second, create a file in that directory.
+    Path myFile = new Path(base, "test/mkdirs/myFile");
+    writeFile(fs, myFile);
+
+    // Verify that expunge without Trash directory
+    // won't throw Exception
+    {
+      String[] args = new String[1];
+      args[0] = "-expunge";
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+    }
+
+    // Verify that we succeed in removing the file we created.
+    // This should go into Trash.
+    {
+      String[] args = new String[2];
+      args[0] = "-rm";
+      args[1] = new Path(base, "test/mkdirs/myFile").toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+
+      trashRoot = shell.getCurrentTrashDir();
+      checkTrash(fs, trashRoot, args[1]);
+    }
+
+    // Verify that we can recreate the file
+    writeFile(fs, myFile);
+
+    // Verify that we succeed in removing the file we re-created
+    {
+      String[] args = new String[2];
+      args[0] = "-rm";
+      args[1] = new Path(base, "test/mkdirs/myFile").toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+    }
+
+    // Verify that we can recreate the file
+    writeFile(fs, myFile);
+
+    // Verify that we succeed in removing the whole directory
+    // along with the file inside it.
+    {
+      String[] args = new String[2];
+      args[0] = "-rmr";
+      args[1] = new Path(base, "test/mkdirs").toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+    }
+
+    // recreate directory
+    mkdir(fs, myPath);
+
+    // Verify that we succeed in removing the whole directory
+    {
+      String[] args = new String[2];
+      args[0] = "-rmr";
+      args[1] = new Path(base, "test/mkdirs").toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+    }
+
+    // Check that we can delete a file from the trash
+    {
+        Path toErase = new Path(trashRoot, "toErase");
+        int retVal = -1;
+        writeFile(fs, toErase);
+        try {
+          retVal = shell.run(new String[] {"-rm", toErase.toString()});
+        } catch (Exception e) {
+          System.err.println("Exception raised from Trash.run " +
+                             e.getLocalizedMessage());
+        }
+        assertTrue(retVal == 0);
+        checkNotInTrash (fs, trashRoot, toErase.toString());
+        checkNotInTrash (fs, trashRoot, toErase.toString()+".1");
+    }
+
+    // simulate Trash removal
+    {
+      String[] args = new String[1];
+      args[0] = "-expunge";
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+    }
+
+    // verify that after expunging the Trash, it really goes away
+    checkNotInTrash(fs, trashRoot, new Path(base, "test/mkdirs/myFile").toString());
+
+    // recreate directory and file
+    mkdir(fs, myPath);
+    writeFile(fs, myFile);
+
+    // remove file first, then remove directory
+    {
+      String[] args = new String[2];
+      args[0] = "-rm";
+      args[1] = new Path(base, "test/mkdirs/myFile").toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+      checkTrash(fs, trashRoot, args[1]);
+
+      args = new String[2];
+      args[0] = "-rmr";
+      args[1] = new Path(base, "test/mkdirs").toString();
+      val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == 0);
+      checkTrash(fs, trashRoot, args[1]);
+    }
+
+    // attempt to remove parent of trash
+    {
+      String[] args = new String[2];
+      args[0] = "-rmr";
+      args[1] = trashRoot.getParent().getParent().toString();
+      int val = -1;
+      try {
+        val = shell.run(args);
+      } catch (Exception e) {
+        System.err.println("Exception raised from Trash.run " +
+                           e.getLocalizedMessage());
+      }
+      assertTrue(val == -1);
+      assertTrue(fs.exists(trashRoot));
+    }
+  }
+
+  public static void trashNonDefaultFS(Configuration conf) throws IOException {
+    conf.set("fs.trash.interval", "10"); // 10 minute
+    // attempt non-default FileSystem trash
+    {
+      final FileSystem lfs = FileSystem.getLocal(conf);
+      Path p = TEST_DIR;
+      Path f = new Path(p, "foo/bar");
+      if (lfs.exists(p)) {
+        lfs.delete(p, true);
+      }
+      try {
+        f = writeFile(lfs, f);
+
+        FileSystem.closeAll();
+        Trash lTrash =
+          new Trash(FileSystem.get(URI.create("file:///"), conf), conf);
+        lTrash.moveToTrash(f.getParent());
+        assertTrue(lfs.exists(new Path(lTrash.getCurrentTrashDir(), "foo/bar")));
+      } finally {
+        if (lfs.exists(p)) {
+          lfs.delete(p, true);
+        }
+      }
+    }
+  }
+
+  public void testTrash() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
+    trashShell(FileSystem.getLocal(conf), TEST_DIR);
+  }
+
+  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");
+    trashNonDefaultFS(conf);
+  }
+
+  static class TestLFS extends LocalFileSystem {
+    Path home;
+    TestLFS() {
+      this(TEST_DIR);
+    }
+    TestLFS(Path home) {
+      super();
+      this.home = home;
+    }
+    public Path getHomeDirectory() {
+      return home;
+    }
+  }
+}

+ 65 - 0
src/test/org/apache/hadoop/hdfs/TestHDFSTrash.java

@@ -0,0 +1,65 @@
+/**
+ * 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.hdfs;
+
+import java.io.IOException;
+
+import junit.framework.Test;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import junit.extensions.TestSetup;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TestTrash;
+
+/**
+ * This class tests commands from Trash.
+ */
+public class TestHDFSTrash extends TestTrash {
+
+  private static MiniDFSCluster cluster = null;
+  public static Test suite() {
+    TestSetup setup = new TestSetup(new TestSuite(TestHDFSTrash.class)) {
+      protected void setUp() throws Exception {
+        Configuration conf = new Configuration();
+        cluster = new MiniDFSCluster(conf, 2, true, null);
+      }
+      protected void tearDown() throws Exception {
+        if (cluster != null) { cluster.shutdown(); }
+      }
+    };
+    return setup;
+  }
+
+  /**
+   * Tests Trash on HDFS
+   */
+  public void testTrash() throws IOException {
+    trashShell(cluster.getFileSystem(), new Path("/"));
+  }
+
+  public void testNonDefaultFS() throws IOException {
+    FileSystem fs = cluster.getFileSystem();
+    Configuration conf = fs.getConf();
+    conf.set("fs.default.name", fs.getUri().toString());
+    trashNonDefaultFS(conf);
+  }
+
+}

+ 0 - 266
src/test/org/apache/hadoop/hdfs/TestTrash.java

@@ -1,266 +0,0 @@
-/**
- * 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.hdfs;
-
-import junit.framework.TestCase;
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * This class tests commands from Trash.
- */
-public class TestTrash extends TestCase {
-
-  static private Path writeFile(FileSystem fs, Path f) throws IOException {
-    DataOutputStream out = fs.create(f);
-    out.writeBytes("dhruba: " + f);
-    out.close();
-    assertTrue(fs.exists(f));
-    return f;
-  }
-
-  static private Path mkdir(FileSystem fs, Path p) throws IOException {
-    assertTrue(fs.mkdirs(p));
-    assertTrue(fs.exists(p));
-    assertTrue(fs.getFileStatus(p).isDir());
-    return p;
-  }
-
-  // check that the specified file is in Trash
-  static void checkTrash(FileSystem fs, Path trashRoot, String pathname)
-                         throws IOException {
-    Path p = new Path(trashRoot+"/"+new Path(pathname).getName());
-    assertTrue(fs.exists(p));
-  }
-
-  // check that the specified file is not in Trash
-  static void checkNotInTrash(FileSystem fs, Path trashRoot, String pathname)
-                              throws IOException {
-    Path p = new Path(trashRoot+"/"+ new Path(pathname).getName());
-    assertTrue(!fs.exists(p));
-  }
-
-  static void show(String s) {
-    System.out.println(Thread.currentThread().getStackTrace()[2] + " " + s);
-  }
-
-  /**
-   * Tests Trash
-   */
-  public void testTrash() throws IOException {
-    Configuration conf = new Configuration();
-    conf.set("fs.trash.interval", "10"); // 10 minute
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
-    FileSystem fs = cluster.getFileSystem();
-    DistributedFileSystem fileSys = (DistributedFileSystem) fs;
-    FsShell shell = new FsShell();
-    shell.setConf(conf);
-    Path trashRoot = null;
-
-    try {
-      // First create a new directory with mkdirs
-      Path myPath = new Path("/test/mkdirs");
-      mkdir(fs, myPath);
-
-      // Second, create a file in that directory.
-      Path myFile = new Path("/test/mkdirs/myFile");
-      writeFile(fs, myFile);
-
-      // Verify that expunge without Trash directory 
-      // won't throw Exception
-      {
-        String[] args = new String[1];
-        args[0] = "-expunge";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-      }
-
-      // Verify that we succeed in removing the file we created.
-      // This should go into Trash.
-      {
-        String[] args = new String[2];
-        args[0] = "-rm";
-        args[1] = "/test/mkdirs/myFile";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-
-        trashRoot = shell.getCurrentTrashDir();
-        checkTrash(fs, trashRoot, args[1]);
-      }
-
-      // Verify that we can recreate the file
-      writeFile(fs, myFile);
-
-      // Verify that we succeed in removing the file we re-created
-      {
-        String[] args = new String[2];
-        args[0] = "-rm";
-        args[1] = "/test/mkdirs/myFile";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-      }
-
-      // Verify that we can recreate the file
-      writeFile(fs, myFile);
-
-      // Verify that we succeed in removing the whole directory
-      // along with the file inside it.
-      {
-        String[] args = new String[2];
-        args[0] = "-rmr";
-        args[1] = "/test/mkdirs";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-      }
-
-      // recreate directory 
-      mkdir(fs, myPath);
-
-      // Verify that we succeed in removing the whole directory
-      {
-        String[] args = new String[2];
-        args[0] = "-rmr";
-        args[1] = "/test/mkdirs";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-      }
-
-      // Check that we can delete a file from the trash
-      {
-          Path toErase = new Path(trashRoot, "toErase");
-          int retVal = -1;
-          writeFile(fs, toErase);
-          try {
-            retVal = shell.run(new String[] {"-rm", toErase.toString()});
-          } catch (Exception e) {
-            System.err.println("Exception raised from Trash.run " +
-                               e.getLocalizedMessage()); 
-          }
-          assertTrue(retVal == 0);
-          checkNotInTrash (fs, trashRoot, toErase.toString());
-          checkNotInTrash (fs, trashRoot, toErase.toString()+".1");
-      }
-
-      // simulate Trash removal
-      {
-        String[] args = new String[1];
-        args[0] = "-expunge";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-      }
-
-      // verify that after expunging the Trash, it really goes away
-      checkNotInTrash(fs, trashRoot, "/test/mkdirs/myFile");
-
-      // recreate directory and file
-      mkdir(fs, myPath);
-      writeFile(fs, myFile);
-
-      // remove file first, then remove directory
-      {
-        String[] args = new String[2];
-        args[0] = "-rm";
-        args[1] = "/test/mkdirs/myFile";
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-        checkTrash(fs, trashRoot, args[1]);
-
-        args = new String[2];
-        args[0] = "-rmr";
-        args[1] = "/test/mkdirs";
-        val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == 0);
-        checkTrash(fs, trashRoot, args[1]);
-      }
-
-      // attempt to remove parent of trash
-      {
-        String[] args = new String[2];
-        args[0] = "-rmr";
-        args[1] = trashRoot.getParent().getParent().toString();
-        int val = -1;
-        try {
-          val = shell.run(args);
-        } catch (Exception e) {
-          System.err.println("Exception raised from Trash.run " +
-                             e.getLocalizedMessage()); 
-        }
-        assertTrue(val == -1);
-        assertTrue(fs.exists(trashRoot));
-      }
-    } finally {
-      try {
-        fs.close();
-      } catch (Exception e) {
-      }
-      cluster.shutdown();
-    }
-  }
-}