You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/10/26 21:59:43 UTC
svn commit: r468108 - in /lucene/hadoop/trunk: CHANGES.txt
src/java/org/apache/hadoop/dfs/DFSShell.java
Author: cutting
Date: Thu Oct 26 12:59:43 2006
New Revision: 468108
URL: http://svn.apache.org/viewvc?view=rev&rev=468108
Log:
HADOOP-635. In DFSShell, permit specification of multiple files as the source for copy and move commands. Contributed by Dhruba.
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=468108&r1=468107&r2=468108
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Thu Oct 26 12:59:43 2006
@@ -58,6 +58,10 @@
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
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java?view=diff&rev=468108&r1=468107&r2=468108
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java Thu Oct 26 12:59:43 2006
@@ -337,6 +337,66 @@
}
/**
+ * 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.
* When copying mutiple files, the destination must be a directory.
@@ -359,6 +419,66 @@
}
/**
+ * 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
* @param recursive: if need to delete subdirs
@@ -568,8 +688,7 @@
//
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 @@
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 @@
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)) {