瀏覽代碼

HADOOP-635. In DFSShell, permit specification of multiple files as the source for copy and move commands. Contributed by Dhruba.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@468108 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 年之前
父節點
當前提交
2461859e2f
共有 2 個文件被更改,包括 132 次插入4 次删除
  1. 4 0
      CHANGES.txt
  2. 128 4
      src/java/org/apache/hadoop/dfs/DFSShell.java

+ 4 - 0
CHANGES.txt

@@ -58,6 +58,10 @@ Trunk (unreleased changes)
     for up to an hour, barring some other process trying to create the
     same file.  (Dhruba Borthakur via cutting)
 
+16. HADOOP-635.  In DFSShell, permit specification of multiple files
+    as the source for file copy and move commands.
+    (Dhruba Borthakur via cutting)
+
 
 Release 0.7.2 - 2006-10-18
 

+ 128 - 4
src/java/org/apache/hadoop/dfs/DFSShell.java

@@ -336,6 +336,66 @@ public class DFSShell extends ToolBase {
       }
     }
 
+    /**
+     * Move/rename DFS file(s) to a destination dfs file. Multiple source
+     * files can be specified. The destination is the last element of
+     * the argvp[] array.
+     * If multiple source files are specified, then the destination 
+     * must be a directory. Otherwise, IOException is thrown.
+     * @exception: IOException  
+     */
+    private int rename(String argv[], Configuration conf) throws IOException {
+      int i = 0;
+      int exitCode = 0;
+      String cmd = argv[i++];  
+      String dest = argv[argv.length-1];
+      //
+      // If the user has specified multiple source files, then
+      // the destination has to be a directory
+      //
+      if (argv.length > 3) {
+        Path dst = new Path(dest);
+        if (!fs.isDirectory(dst)) {
+          throw new IOException( "When moving multiple files, " 
+            + "destination " + dest + " should be a directory." );
+        }
+      }
+      //
+      // for each source file, issue the rename
+      //
+      for (; i < argv.length - 1; i++) {
+        try {
+          //
+          // issue the rename to the remote dfs server
+          //
+          rename(argv[i], dest);
+        } catch (RemoteException e) {
+          //
+          // This is a error returned by hadoop server. Print
+          // out the first line of the error mesage.
+          //
+          exitCode = -1;
+          try {
+            String[] content;
+            content = e.getLocalizedMessage().split("\n");
+            System.err.println(cmd.substring(1) + ": " +
+                               content[0]);
+          } catch (Exception ex) {
+            System.err.println(cmd.substring(1) + ": " +
+                               ex.getLocalizedMessage());
+          }
+        } catch (IOException e) {
+          //
+          // IO exception encountered locally.
+          //
+          exitCode = -1;
+          System.err.println(cmd.substring(1) + ": " +
+                             e.getLocalizedMessage());
+        }
+      }
+      return exitCode;
+    }
+
     /**
      * Copy DFS files that match the file pattern <i>srcf</i>
      * to a destination dfs file.
@@ -358,6 +418,66 @@ public class DFSShell extends ToolBase {
       }
     }
 
+    /**
+     * Copy DFS file(s) to a destination dfs file. Multiple source
+     * files can be specified. The destination is the last element of
+     * the argvp[] array.
+     * If multiple source files are specified, then the destination 
+     * must be a directory. Otherwise, IOException is thrown.
+     * @exception: IOException  
+     */
+    private int copy(String argv[], Configuration conf) throws IOException {
+      int i = 0;
+      int exitCode = 0;
+      String cmd = argv[i++];  
+      String dest = argv[argv.length-1];
+      //
+      // If the user has specified multiple source files, then
+      // the destination has to be a directory
+      //
+      if (argv.length > 3) {
+        Path dst = new Path(dest);
+        if (!fs.isDirectory(dst)) {
+          throw new IOException( "When copying multiple files, " 
+            + "destination " + dest + " should be a directory." );
+        }
+      }
+      //
+      // for each source file, issue the copy
+      //
+      for (; i < argv.length - 1; i++) {
+        try {
+          //
+          // issue the copy to the remote dfs server
+          //
+          copy(argv[i], dest, conf);
+        } catch (RemoteException e) {
+          //
+          // This is a error returned by hadoop server. Print
+          // out the first line of the error mesage.
+          //
+          exitCode = -1;
+          try {
+            String[] content;
+            content = e.getLocalizedMessage().split("\n");
+            System.err.println(cmd.substring(1) + ": " +
+                               content[0]);
+          } catch (Exception ex) {
+            System.err.println(cmd.substring(1) + ": " +
+                               ex.getLocalizedMessage());
+          }
+        } catch (IOException e) {
+          //
+          // IO exception encountered locally.
+          //
+          exitCode = -1;
+          System.err.println(cmd.substring(1) + ": " +
+                             e.getLocalizedMessage());
+        }
+      }
+      return exitCode;
+    }
+
     /**
      * Delete all files in DFS that match the file pattern <i>srcf</i>
      * @param srcf: a file pattern specifying source files
@@ -568,8 +688,7 @@ public class DFSShell extends ToolBase {
         //
         if ("-put".equals(cmd) || "-get".equals(cmd) || 
             "-copyFromLocal".equals(cmd) || "-moveFromLocal".equals(cmd) || 
-            "-copyToLocal".equals(cmd) || "-moveToLocal".equals(cmd) || 
-            "-mv".equals(cmd) || "-cp".equals(cmd)) {
+            "-copyToLocal".equals(cmd) || "-moveToLocal".equals(cmd)) {
                 if (argv.length != 3) {
                   printUsage(cmd);
                   return exitCode;
@@ -586,6 +705,11 @@ public class DFSShell extends ToolBase {
                   printUsage(cmd);
                   return exitCode;
                 }
+        } else if ("-mv".equals(cmd) || "-cp".equals(cmd)) {
+                if (argv.length < 3) {
+                  printUsage(cmd);
+                  return exitCode;
+                }
         }
 
         // initialize DFSShell
@@ -626,9 +750,9 @@ public class DFSShell extends ToolBase {
                 String arg = i < argv.length ? argv[i++] : "";
                 ls(arg, true);
             } else if ("-mv".equals(cmd)) {
-                rename(argv[i++], argv[i++]);
+                exitCode = rename(argv, conf);
             } else if ("-cp".equals(cmd)) {
-                copy(argv[i++], argv[i++], conf);
+                exitCode = copy(argv, conf);
             } else if ("-rm".equals(cmd)) {
                 delete(argv[i++], false);
             } else if ("-rmr".equals(cmd)) {