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/06/28 16:14:59 UTC

svn commit: r417761 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/util/CopyFiles.java

Author: cutting
Date: Wed Jun 28 07:14:59 2006
New Revision: 417761

URL: http://svn.apache.org/viewvc?rev=417761&view=rev
Log:
HADOOP-328.  Add an option to the distcp command to ignore read errors while copying. Contributed by Owen.

Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=417761&r1=417760&r2=417761&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Jun 28 07:14:59 2006
@@ -74,6 +74,9 @@
     remove the workaround used to initialize classes.
     (omalley via cutting)
 
+18. HADOOP-328.  Add an option to the "distcp" command to ignore read
+    errors while copying.  (omalley via cutting)
+
 
 Release 0.3.2 - 2006-06-09
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java?rev=417761&r1=417760&r2=417761&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java Wed Jun 28 07:14:59 2006
@@ -56,11 +56,13 @@
   private static final String usage = "distcp "+
           "[-fs <namenode:port | local> ] [-jt <jobtracker:port | local>] " +
           "[-conf <config-file.xml>] " + "[-D <property=value>] "+
-          "<srcurl> <desturl>";
+          "[-i] <srcurl> <desturl>";
   
   private static final long MIN_BYTES_PER_MAP = 1L << 28;
   private static final int MAX_NUM_MAPS = 10000;
   private static final int MAX_MAPS_PER_NODE = 10;
+  private static final String readFailuresAttribute = 
+     "distcp.ignore.read.failures";
   
   public void setConf(Configuration conf) {
       if (conf instanceof JobConf) {
@@ -86,6 +88,7 @@
     private long bytesSinceLastReport = 0L;
     private long totalBytesCopied = 0L;
     private static DecimalFormat percentFormat = new DecimalFormat("0.00");
+    private boolean ignoreReadFailures;
     
     private void copy(String src, Reporter reporter) throws IOException {
       // open source file
@@ -144,6 +147,7 @@
       }
       sizeBuf = job.getInt("copy.buf.size", 4096);
       buffer = new byte[sizeBuf];
+      ignoreReadFailures = job.getBoolean(readFailuresAttribute, false);
     }
     
     /** Map method. Copies one file from source file system to destination.
@@ -156,7 +160,21 @@
         OutputCollector out,
         Reporter reporter) throws IOException {
       String src = ((UTF8) key).toString();
-      copy(src, reporter);
+      try {
+        copy(src, reporter);
+      } catch (IOException except) {
+        if (ignoreReadFailures) {
+          reporter.setStatus("Failed to copy " + src + " : " + 
+                             StringUtils.stringifyException(except));
+          try {
+            destFileSys.delete(new Path(destPath, src));
+          } catch (Throwable ex) {
+            // ignore, we are just cleaning up
+          }
+        } else {
+          throw except;
+        }
+      }
     }
     
     public void close() {
@@ -223,9 +241,12 @@
   public int run(String[] args) throws IOException {
     String srcPath = null;
     String destPath = null;
+    boolean ignoreReadFailures = false;
     
     for (int idx = 0; idx < args.length; idx++) {
-        if (srcPath == null) {
+        if ("-i".equals(args[idx])) {
+          ignoreReadFailures = true;
+        } else if (srcPath == null) {
                 srcPath = args[idx];
         } else if (destPath == null) {
                 destPath = args[idx];
@@ -269,11 +290,9 @@
     destPath = desturl.getPath();
     if ("".equals(destPath)) { destPath = "/"; }
     
-    boolean isFile = false;
     Path tmpPath = new Path(srcPath);
     Path rootPath = new Path(srcPath);
     if (srcfs.isFile(tmpPath)) {
-      isFile = true;
       tmpPath = tmpPath.getParent();
       rootPath = rootPath.getParent();
       jobConf.set("copy.src.path", tmpPath.toString());
@@ -302,6 +321,7 @@
     jobConf.setReducerClass(CopyFilesReducer.class);
     
     jobConf.setNumReduceTasks(1);
+    jobConf.setBoolean(readFailuresAttribute, ignoreReadFailures);
 
     Path tmpDir = new Path("copy-files");
     Path inDir = new Path(tmpDir, "in");
@@ -319,7 +339,6 @@
     ArrayList finalPathList = new ArrayList();
     pathList.add(new Path(srcPath));
     long totalBytes = 0;
-    int part = 0;
     while(!pathList.isEmpty()) {
       Path top = (Path) pathList.remove(0);
       if (srcfs.isFile(top)) {