Преглед на файлове

svn merge -c 1100369 from trunk for HADOOP-7251.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/yahoo-merge@1127271 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze преди 14 години
родител
ревизия
95644048cc

+ 3 - 0
CHANGES.txt

@@ -50,6 +50,9 @@ Trunk (unreleased changes)
     HADOOP-7249. Refactor the chmod/chown/chgrp command to conform to new
     FsCommand class.  (Daryn Sharp via szetszwo)
 
+    HADOOP-7251. Refactor the getmerge command to conform to new FsCommand
+    class.  (Daryn Sharp via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -307,49 +307,6 @@ public class FsShell extends Configured implements Tool {
     }
   }
 
-  /**
-   * Get all the files in the directories that match the source file 
-   * pattern and merge and sort them to only one file on local fs 
-   * srcf is kept.
-   * @param srcf a file pattern specifying source files
-   * @param dstf a destination local file/directory 
-   * @exception IOException  
-   * @see org.apache.hadoop.fs.FileSystem.globStatus 
-   */
-  void copyMergeToLocal(String srcf, Path dst) throws IOException {
-    copyMergeToLocal(srcf, dst, false);
-  }    
-    
-
-  /**
-   * Get all the files in the directories that match the source file pattern
-   * and merge and sort them to only one file on local fs 
-   * srcf is kept.
-   * 
-   * Also adds a string between the files (useful for adding \n
-   * to a text file)
-   * @param srcf a file pattern specifying source files
-   * @param dstf a destination local file/directory
-   * @param endline if an end of line character is added to a text file 
-   * @throws IOException on error
-   * @see org.apache.hadoop.fs.FileSystem.globStatus 
-   */
-  void copyMergeToLocal(String srcf, Path dst, boolean endline) throws IOException {
-    Path srcPath = new Path(srcf);
-    FileSystem srcFs = srcPath.getFileSystem(getConf());
-    Path [] srcs = FileUtil.stat2Paths(srcFs.globStatus(srcPath), 
-                                       srcPath);
-    for(int i=0; i<srcs.length; i++) {
-      if (endline) {
-        FileUtil.copyMerge(srcFs, srcs[i], 
-                           FileSystem.getLocal(getConf()), dst, false, getConf(), "\n");
-      } else {
-        FileUtil.copyMerge(srcFs, srcs[i], 
-                           FileSystem.getLocal(getConf()), dst, false, getConf(), null);
-      }
-    }
-  }      
-
   /**
    * Obtain the indicated file and copy to the local name.
    * srcf is removed.
@@ -994,7 +951,7 @@ public class FsShell extends Configured implements Tool {
       "[-rmr [-skipTrash] <src>] [-put <localsrc> ... <dst>] [-copyFromLocal <localsrc> ... <dst>]\n\t" +
       "[-moveFromLocal <localsrc> ... <dst>] [" + 
       GET_SHORT_USAGE + "\n\t" +
-      "[-getmerge <src> <localdst> [addnl]] [-cat <src>]\n\t" +
+      "[-cat <src>]\n\t" +
       "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
       "[-report]\n\t" +
       "[-touchz <path>] [-test -[ezd] <path>] [-stat [format] <path>]\n\t" +
@@ -1066,10 +1023,6 @@ public class FsShell extends Configured implements Tool {
       "\t\tto the local name.  <src> is kept.  When copying mutiple, \n" +
       "\t\tfiles, the destination must be a directory. \n";
 
-    String getmerge = "-getmerge <src> <localdst>:  Get all the files in the directories that \n" +
-      "\t\tmatch the source file pattern and merge and sort them to only\n" +
-      "\t\tone file on local fs. <src> is kept.\n";
-
     String cat = "-cat <src>: \tFetch all files that match the file pattern <src> \n" +
       "\t\tand display their content on stdout.\n";
 
@@ -1132,8 +1085,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println(moveFromLocal);
     } else if ("get".equals(cmd)) {
       System.out.println(get);
-    } else if ("getmerge".equals(cmd)) {
-      System.out.println(getmerge);
     } else if ("copyToLocal".equals(cmd)) {
       System.out.println(copyToLocal);
     } else if ("moveToLocal".equals(cmd)) {
@@ -1172,7 +1123,6 @@ public class FsShell extends Configured implements Tool {
       System.out.println(copyFromLocal);
       System.out.println(moveFromLocal);
       System.out.println(get);
-      System.out.println(getmerge);
       System.out.println(cat);
       System.out.println(copyToLocal);
       System.out.println(moveToLocal);
@@ -1342,7 +1292,6 @@ public class FsShell extends Configured implements Tool {
       System.err.println("           [-copyFromLocal <localsrc> ... <dst>]");
       System.err.println("           [-moveFromLocal <localsrc> ... <dst>]");
       System.err.println("           [" + GET_SHORT_USAGE + "]");
-      System.err.println("           [-getmerge <src> <localdst> [addnl]]");
       System.err.println("           [-cat <src>]");
       System.err.println("           [-text <src>]");
       System.err.println("           [" + COPYTOLOCAL_SHORT_USAGE + "]");
@@ -1448,11 +1397,6 @@ public class FsShell extends Configured implements Tool {
         moveFromLocal(srcs, argv[i++]);
       } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
         copyToLocal(argv, i);
-      } else if ("-getmerge".equals(cmd)) {
-        if (argv.length>i+2)
-          copyMergeToLocal(argv[i++], new Path(argv[i++]), Boolean.parseBoolean(argv[i++]));
-        else
-          copyMergeToLocal(argv[i++], new Path(argv[i++]));
       } else if ("-cat".equals(cmd)) {
         exitCode = doall(cmd, argv, i);
       } else if ("-text".equals(cmd)) {

+ 72 - 0
src/java/org/apache/hadoop/fs/shell/Copy.java

@@ -0,0 +1,72 @@
+/**
+ * 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.FileUtil;
+import org.apache.hadoop.fs.Path;
+
+/** Various commands for copy files */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+class Copy extends FsCommand {  
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Merge.class, "-getmerge");
+  }
+
+  /** merge multiple files together */
+  public static class Merge extends Copy {
+    public static final String NAME = "MergeToLocal";    
+    public static final String USAGE = "<src> <localdst> [addnl]";
+    public static final String DESCRIPTION =
+      "Get all the files in the directories that\n" +
+      "match the source file pattern and merge and sort them to only\n" +
+      "one file on local fs. <src> is kept.\n";
+
+    protected PathData dst = null;
+    protected String delimiter = null;
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      CommandFormat cf = new CommandFormat(null, 2, 3);
+      cf.parse(args);
+
+      // TODO: this really should be a -nl option
+      if ((args.size() > 2) && Boolean.parseBoolean(args.removeLast())) {
+        delimiter = "\n";
+      } else {
+        delimiter = null;
+      }
+      
+      Path path = new Path(args.removeLast());
+      dst = new PathData(path.getFileSystem(getConf()), path);
+    }
+
+    @Override
+    protected void processPath(PathData src) throws IOException {
+      FileUtil.copyMerge(src.fs, src.path,
+          dst.fs, dst.path, false, getConf(), delimiter);
+    }
+  }
+}

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

@@ -43,6 +43,7 @@ abstract public class FsCommand extends Command {
    * @param factory where to register the class
    */
   public static void registerCommands(CommandFactory factory) {
+    factory.registerCommands(Copy.class);
     factory.registerCommands(Count.class);
     factory.registerCommands(FsShellPermissions.class);
     factory.registerCommands(Ls.class);

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

@@ -430,7 +430,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getmerge &lt;src&gt; &lt;localdst&gt;:  Get all the files in the directories that( )*</expected-output>
+          <expected-output>^-getmerge &lt;src&gt; &lt;localdst&gt; \[addnl\]:( |\t)*Get all the files in the directories that( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>