ソースを参照

svn merge -c 1102093 from trunk for HADOOP-7267.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/yahoo-merge@1127289 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 14 年 前
コミット
1c3f13440b

+ 5 - 2
CHANGES.txt

@@ -62,12 +62,15 @@ Trunk (unreleased changes)
     HADOOP-7271. Standardize shell command error messages.  (Daryn Sharp
     via szetszwo)
 
-    HADOOP-7275. Refactor the stat commands to conform to new FsCommand
+    HADOOP-7275. Refactor the stat command to conform to new FsCommand
     class.  (Daryn Sharp via szetszwo)
 
-    HADOOP-7237. Refactor the touchz commands to conform to new FsCommand
+    HADOOP-7237. Refactor the touchz command to conform to new FsCommand
     class.  (Daryn Sharp via szetszwo)
 
+    HADOOP-7267. Refactor the rm/rmr/expunge commands to conform to new
+    FsCommand class.  (Daryn Sharp via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 3 - 123
src/java/org/apache/hadoop/fs/FsShell.java

@@ -636,77 +636,6 @@ public class FsShell extends Configured implements Tool {
     return exitCode;
   }
 
-  /**
-   * Delete all files that match the file pattern <i>srcf</i>.
-   * @param srcf a file pattern specifying source files
-   * @param recursive if need to delete subdirs
-   * @param skipTrash Should we skip the trash, if it's enabled?
-   * @throws IOException  
-   * @see org.apache.hadoop.fs.FileSystem#globStatus(Path)
-   */
-  void delete(String srcf, final boolean recursive, final boolean skipTrash) 
-                                                            throws IOException {
-    //rm behavior in Linux
-    //  [~/1207]$ ls ?.txt
-    //  x.txt  z.txt
-    //  [~/1207]$ rm x.txt y.txt z.txt 
-    //  rm: cannot remove `y.txt': No such file or directory
-
-    Path srcPattern = new Path(srcf);
-    new DelayedExceptionThrowing() {
-      @Override
-      void process(Path p, FileSystem srcFs) throws IOException {
-        delete(p, srcFs, recursive, skipTrash);
-      }
-    }.globAndProcess(srcPattern, srcPattern.getFileSystem(getConf()));
-  }
-    
-  /* delete a file */
-  private void delete(Path src, FileSystem srcFs, boolean recursive, 
-                      boolean skipTrash) throws IOException {
-    FileStatus fs = null;
-    try {
-      fs = srcFs.getFileStatus(src);
-    } catch (FileNotFoundException fnfe) {
-      // Have to re-throw so that console output is as expected
-      throw new PathNotFoundException(src.toString());
-    }
-    
-    if (fs.isDirectory() && !recursive) {
-      throw new IOException("Cannot remove directory \"" + src +
-                            "\", use -rmr instead");
-    }
-    
-    if(!skipTrash) {
-      try {
-        if (Trash.moveToAppropriateTrash(srcFs, src, getConf())) {
-          System.out.println("Moved to trash: " + src);
-          return;
-        }
-      } catch (IOException e) {
-        LOG.debug("Error with trash", e);
-        Exception cause = (Exception) e.getCause();
-        String msg = "";
-        if(cause != null) {
-          msg = cause.getLocalizedMessage();
-        }
-        System.err.println("Problem with Trash." + msg +". Consider using -skipTrash option");        
-        throw e;
-      }
-    }
-    
-    if (srcFs.delete(src, true)) {
-      System.out.println("Deleted " + src);
-    } else {
-      throw new IOException("Delete failed " + src);
-    }
-  }
-
-  private void expunge() throws IOException {
-    getTrash().expunge();
-    getTrash().checkpoint();
-  }
-
   /**
    * Returns the Trash object associated with this shell.
    */
@@ -736,8 +665,8 @@ public class FsShell extends Configured implements Tool {
       "The full syntax is: \n\n" +
       "hadoop fs [-fs <local | file system URI>] [-conf <configuration file>]\n\t" +
       "[-D <property=value>] [-df [<path>]] [-du [-s] [-h] <path>]\n\t" +
-      "[-dus <path>] [-mv <src> <dst>] [-cp <src> <dst>] [-rm [-skipTrash] <src>]\n\t" + 
-      "[-rmr [-skipTrash] <src>] [-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
+      "[-dus <path>] [-mv <src> <dst>] [-cp <src> <dst>]\n\t" + 
+      "[-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
       "[-moveFromLocal <localsrc> ... <dst>] [" + 
       GET_SHORT_USAGE + "\n\t" +
       "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
@@ -786,16 +715,6 @@ public class FsShell extends Configured implements Tool {
       "\t\tdestination.  When copying multiple files, the destination\n" +
       "\t\tmust be a directory. \n";
 
-    String rm = "-rm [-skipTrash] <src>: \tDelete all files that match the specified file pattern.\n" +
-      "\t\tEquivalent to the Unix command \"rm <src>\"\n" +
-      "\t\t-skipTrash option bypasses trash, if enabled, and immediately\n" +
-      "deletes <src>";
-
-    String rmr = "-rmr [-skipTrash] <src>: \tRemove all directories which match the specified file \n" +
-      "\t\tpattern. Equivalent to the Unix command \"rm -rf <src>\"\n" +
-      "\t\t-skipTrash option bypasses trash, if enabled, and immediately\n" +
-      "deletes <src>";
-
     String put = "-put <localsrc> ... <dst>: \tCopy files " + 
     "from the local file system \n\t\tinto fs. \n";
 
@@ -818,8 +737,6 @@ public class FsShell extends Configured implements Tool {
     String test = "-test -[ezd] <path>: If file { exists, has zero length, is a directory\n" +
       "\t\tthen return 0, else return 1.\n";
 
-    String expunge = "-expunge: Empty the Trash.\n";
-    
     String help = "-help [cmd]: \tDisplays help for given command or all commands if none\n" +
       "\t\tis specified.\n";
 
@@ -838,12 +755,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println(du);
     } else if ("dus".equals(cmd)) {
       System.out.println(dus);
-    } else if ("rm".equals(cmd)) {
-      System.out.println(rm);
-    } else if ("expunge".equals(cmd)) {
-      System.out.println(expunge);
-    } else if ("rmr".equals(cmd)) {
-      System.out.println(rmr);
     } else if ("mv".equals(cmd)) {
       System.out.println(mv);
     } else if ("cp".equals(cmd)) {
@@ -880,8 +791,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println(dus);
       System.out.println(mv);
       System.out.println(cp);
-      System.out.println(rm);
-      System.out.println(rmr);
       System.out.println(put);
       System.out.println(copyFromLocal);
       System.out.println(moveFromLocal);
@@ -921,14 +830,6 @@ public class FsShell extends Configured implements Tool {
   private int doall(String cmd, String argv[], int startindex) {
     int exitCode = 0;
     int i = startindex;
-    boolean rmSkipTrash = false;
-    
-    // Check for -skipTrash option in rm/rmr
-    if(("-rm".equals(cmd) || "-rmr".equals(cmd)) 
-        && "-skipTrash".equals(argv[i])) {
-      rmSkipTrash = true;
-      i++;
-    }
     
     //
     // for each source file, issue the command
@@ -938,11 +839,7 @@ public class FsShell extends Configured implements Tool {
         //
         // issue the command to the fs
         //
-        if ("-rm".equals(cmd)) {
-          delete(argv[i], false, rmSkipTrash);
-        } else if ("-rmr".equals(cmd)) {
-          delete(argv[i], true, rmSkipTrash);
-        } else if ("-df".equals(cmd)) {
+        if ("-df".equals(cmd)) {
           df(argv[i]);
         }
       } catch (IOException e) {
@@ -979,9 +876,6 @@ public class FsShell extends Configured implements Tool {
     } else if ("-df".equals(cmd) ) {
       System.err.println("Usage: java FsShell" +
                          " [" + cmd + " [<path>]]");
-    } else if ("-rm".equals(cmd) || "-rmr".equals(cmd)) {
-      System.err.println("Usage: java FsShell [" + cmd + 
-                           " [-skipTrash] <src>]");
     } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
       System.err.println("Usage: java FsShell" + 
                          " [" + cmd + " <src> <dst>]");
@@ -1006,9 +900,6 @@ public class FsShell extends Configured implements Tool {
       System.err.println("           [-dus <path>]");
       System.err.println("           [-mv <src> <dst>]");
       System.err.println("           [-cp <src> <dst>]");
-      System.err.println("           [-rm [-skipTrash] <path>]");
-      System.err.println("           [-rmr [-skipTrash] <path>]");
-      System.err.println("           [-expunge]");
       System.err.println("           [-put <localsrc> ... <dst>]");
       System.err.println("           [-copyFromLocal <localsrc> ... <dst>]");
       System.err.println("           [-moveFromLocal <localsrc> ... <dst>]");
@@ -1064,11 +955,6 @@ public class FsShell extends Configured implements Tool {
         printUsage(cmd);
         return exitCode;
       }
-    } else if ("-rm".equals(cmd) || "-rmr".equals(cmd)) {
-      if (argv.length < 2) {
-        printUsage(cmd);
-        return exitCode;
-      }
     }
     // initialize FsShell
     try {
@@ -1117,12 +1003,6 @@ public class FsShell extends Configured implements Tool {
         exitCode = rename(argv, getConf());
       } else if ("-cp".equals(cmd)) {
         exitCode = copy(argv, getConf());
-      } else if ("-rm".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
-      } else if ("-rmr".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
-      } else if ("-expunge".equals(cmd)) {
-        expunge();
       } else if ("-df".equals(cmd)) {
         if (argv.length-1 > 0) {
           exitCode = doall(cmd, argv, i);

+ 132 - 0
src/java/org/apache/hadoop/fs/shell/Delete.java

@@ -0,0 +1,132 @@
+/**
+ * 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.shell;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Trash;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsDirectoryException;
+
+/**
+ * Classes that delete paths
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+class Delete extends FsCommand {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Rm.class, "-rm");
+    factory.addClass(Rmr.class, "-rmr");
+    factory.addClass(Expunge.class, "-expunge");
+  }
+
+  /** remove non-directory paths */
+  public static class Rm extends FsCommand {
+    public static final String NAME = "rm";
+    public static final String USAGE = "[-skipTrash] <src> ...";
+    public static final String DESCRIPTION =
+      "Delete all files that match the specified file pattern.\n" +
+      "Equivalent to the Unix command \"rm <src>\"\n" +
+      "-skipTrash option bypasses trash, if enabled, and immediately\n" +
+      "deletes <src>";
+
+    private boolean skipTrash = false;
+    private boolean deleteDirs = false;
+    
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(
+          null, 1, Integer.MAX_VALUE, "R", "skipTrash");
+      cf.parse(args);
+      deleteDirs = cf.getOpt("R");
+      skipTrash = cf.getOpt("skipTrash");
+    }
+    
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (item.stat.isDirectory() && !deleteDirs) {
+        throw new PathIsDirectoryException(item.toString());
+      }
+
+      // TODO: if the user wants the trash to be used but there is any
+      // problem (ie. creating the trash dir, moving the item to be deleted,
+      // 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)) {
+        throw new PathIOException(item.toString());
+      }
+      out.println("Deleted " + item);
+    }
+
+    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));
+      }
+      return success;
+    }
+  }
+  
+  /** remove any path */
+  static class Rmr extends Rm {
+    public static final String NAME = "rmr";
+    public static final String USAGE = Rm.USAGE;
+    public static final String DESCRIPTION =
+      "Remove all directories which match the specified file\n" +
+      "pattern. Equivalent to the Unix command \"rm -rf <src>\"\n" +
+      "-skipTrash option bypasses trash, if enabled, and immediately\n" +
+      "deletes <src>";
+    
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      args.addFirst("-R");
+      super.processOptions(args);
+    }
+  }
+  
+  /** empty the trash */
+  static class Expunge extends FsCommand {
+    public static final String NAME = "expunge";
+    public static final String USAGE = "";
+    public static final String DESCRIPTION = "Empty the Trash";
+
+    // TODO: should probably allow path arguments for the filesystems
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 0, 0);
+      cf.parse(args);
+    }
+
+    @Override
+    protected void processArguments(LinkedList<PathData> args)
+    throws IOException {
+      Trash trash = new Trash(getConf());
+      trash.expunge();
+      trash.checkpoint();    
+    }
+  }
+}

+ 1 - 0
src/java/org/apache/hadoop/fs/shell/FsCommand.java

@@ -45,6 +45,7 @@ abstract public class FsCommand extends Command {
   public static void registerCommands(CommandFactory factory) {
     factory.registerCommands(Copy.class);
     factory.registerCommands(Count.class);
+    factory.registerCommands(Delete.class);
     factory.registerCommands(Display.class);
     factory.registerCommands(FsShellPermissions.class);
     factory.registerCommands(Ls.class);

+ 2 - 2
src/test/core/org/apache/hadoop/cli/testConf.xml

@@ -299,7 +299,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rm \[-skipTrash\] &lt;src&gt;:( |\t)*Delete all files that match the specified file pattern.( )*</expected-output>
+          <expected-output>^-rm \[-skipTrash\] &lt;src&gt; \.\.\.:( |\t)*Delete all files that match the specified file pattern.( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -326,7 +326,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rmr \[-skipTrash\] &lt;src&gt;:( |\t)*Remove all directories which match the specified file( )*</expected-output>
+          <expected-output>^-rmr \[-skipTrash\] &lt;src&gt; \.\.\.:( |\t)*Remove all directories which match the specified file( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

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

@@ -280,7 +280,7 @@ public class TestTrash extends TestCase {
         System.err.println("Exception raised from Trash.run " +
                            e.getLocalizedMessage());
       }
-      assertTrue(val == -1);
+      assertEquals("exit code", 1, val);
       assertTrue(fs.exists(trashRoot));
     }