瀏覽代碼

HADOOP-14839. DistCp log output should contain copied and deleted files and directories. Contributed by Yiqun Lin.

Xiaoyu Yao 7 年之前
父節點
當前提交
b669aca518

+ 1 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java

@@ -45,6 +45,7 @@ public class DistCpConstants {
   public static final String CONF_LABEL_ATOMIC_COPY = "distcp.atomic.copy";
   public static final String CONF_LABEL_WORK_PATH = "distcp.work.path";
   public static final String CONF_LABEL_LOG_PATH = "distcp.log.path";
+  public static final String CONF_LABEL_VERBOSE_LOG = "distcp.verbose.log";
   public static final String CONF_LABEL_IGNORE_FAILURES = "distcp.ignore.failures";
   public static final String CONF_LABEL_PRESERVE_STATUS = "distcp.preserve.status";
   public static final String CONF_LABEL_PRESERVE_RAWXATTRS =

+ 7 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java

@@ -127,6 +127,13 @@ public enum DistCpOptionSwitch {
   LOG_PATH(DistCpConstants.CONF_LABEL_LOG_PATH,
       new Option("log", true, "Folder on DFS where distcp execution logs are saved")),
 
+  /**
+   * Log additional info (path, size) in the SKIP/COPY log.
+   */
+  VERBOSE_LOG(DistCpConstants.CONF_LABEL_VERBOSE_LOG,
+      new Option("v", false,
+          "Log additional info (path, size) in the SKIP/COPY log")),
+
   /**
    * Copy strategy is use. This could be dynamic or uniform size etc.
    * DistCp would use an appropriate input format based on this.

+ 22 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java

@@ -56,6 +56,9 @@ public class DistCpOptions {
   // content at their s1, if src is not the same as tgt.
   private boolean useRdiff = false;
 
+  /** Whether to log additional info (path, size) in the SKIP/COPY log. */
+  private boolean verboseLog = false;
+
   // For both -diff and -rdiff, given the example command line switches, two
   // steps are taken:
   //   1. Sync Step. This step does renaming/deletion ops in the snapshot diff,
@@ -180,6 +183,7 @@ public class DistCpOptions {
       this.filtersFile = that.getFiltersFile();
       this.blocksPerChunk = that.blocksPerChunk;
       this.copyBufferSize = that.copyBufferSize;
+      this.verboseLog = that.verboseLog;
     }
   }
 
@@ -656,6 +660,15 @@ public class DistCpOptions {
     return this.copyBufferSize;
   }
 
+  public void setVerboseLog(boolean newVerboseLog) {
+    validate(DistCpOptionSwitch.VERBOSE_LOG, newVerboseLog);
+    this.verboseLog = newVerboseLog;
+  }
+
+  public boolean shouldVerboseLog() {
+    return verboseLog;
+  }
+
   public void validate(DistCpOptionSwitch option, boolean value) {
 
     boolean syncFolder = (option == DistCpOptionSwitch.SYNC_FOLDERS ?
@@ -671,6 +684,8 @@ public class DistCpOptions {
     boolean append = (option == DistCpOptionSwitch.APPEND ? value : this.append);
     boolean useDiff = (option == DistCpOptionSwitch.DIFF ? value : this.useDiff);
     boolean useRdiff = (option == DistCpOptionSwitch.RDIFF ? value : this.useRdiff);
+    boolean shouldVerboseLog = (option == DistCpOptionSwitch.VERBOSE_LOG ?
+        value : this.verboseLog);
 
     if (syncFolder && atomicCommit) {
       throw new IllegalArgumentException("Atomic commit can't be used with " +
@@ -716,6 +731,10 @@ public class DistCpOptions {
       throw new IllegalArgumentException(
           "-diff and -rdiff are mutually exclusive");
     }
+
+    if (shouldVerboseLog && logPath == null) {
+      throw new IllegalArgumentException("-v is valid only with -log option");
+    }
   }
 
   /**
@@ -754,6 +773,8 @@ public class DistCpOptions {
         String.valueOf(blocksPerChunk));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.COPY_BUFFER_SIZE,
         String.valueOf(copyBufferSize));
+    DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.VERBOSE_LOG,
+        String.valueOf(verboseLog));
   }
 
   /**
@@ -792,6 +813,7 @@ public class DistCpOptions {
         ", filtersFile='" + filtersFile + '\'' +
         ", blocksPerChunk=" + blocksPerChunk +
         ", copyBufferSize=" + copyBufferSize +
+        ", verboseLog=" + verboseLog +
         '}';
   }
 

+ 4 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java

@@ -188,6 +188,10 @@ public class OptionsParser {
 
     parseCopyBufferSize(command, option);
 
+    if (command.hasOption(DistCpOptionSwitch.VERBOSE_LOG.getSwitch())) {
+      option.setVerboseLog(true);
+    }
+
     return option;
   }
 

+ 26 - 6
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java

@@ -58,6 +58,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
    */
   public static enum Counter {
     COPY,         // Number of files received by the mapper for copy.
+    DIR_COPY,     // Number of directories received by the mapper for copy.
     SKIP,         // Number of files skipped.
     FAIL,         // Number of files that failed to be copied.
     BYTESCOPIED,  // Number of bytes actually copied by the copy-mapper, total.
@@ -84,6 +85,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private boolean skipCrc = false;
   private boolean overWrite = false;
   private boolean append = false;
+  private boolean verboseLog = false;
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
@@ -105,6 +107,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
     overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
     append = conf.getBoolean(DistCpOptionSwitch.APPEND.getConfigLabel(), false);
+    verboseLog = conf.getBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), false);
     preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
         PRESERVE_STATUS.getConfigLabel()));
 
@@ -259,6 +263,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         updateSkipCounters(context, sourceCurrStatus);
         context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
 
+        if (verboseLog) {
+          context.write(null,
+              new Text("FILE_SKIPPED: source=" + sourceFileStatus.getPath()
+              + ", size=" + sourceFileStatus.getLen() + " --> "
+              + "target=" + target + ", size=" + (targetStatus == null ?
+                  0 : targetStatus.getLen())));
+        }
       } else {
         if (sourceCurrStatus.isSplit()) {
           tmpTarget = DistCpUtils.getSplitChunkPath(target, sourceCurrStatus);
@@ -266,8 +277,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
         if (LOG.isDebugEnabled()) {
           LOG.debug("copying " + sourceCurrStatus + " " + tmpTarget);
         }
-        copyFileWithRetry(description, sourceCurrStatus, tmpTarget, context,
-            action, fileAttributes);
+        copyFileWithRetry(description, sourceCurrStatus, tmpTarget,
+            targetStatus, context, action, fileAttributes);
       }
       DistCpUtils.preserve(target.getFileSystem(conf), tmpTarget,
           sourceCurrStatus, fileAttributes, preserveRawXattrs);
@@ -298,9 +309,10 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   }
 
   private void copyFileWithRetry(String description,
-      CopyListingFileStatus sourceFileStatus, Path target, Context context,
-      FileAction action, EnumSet<DistCpOptions.FileAttribute> fileAttributes)
-      throws IOException {
+      CopyListingFileStatus sourceFileStatus, Path target,
+      FileStatus targrtFileStatus, Context context, FileAction action,
+      EnumSet<DistCpOptions.FileAttribute> fileAttributes)
+      throws IOException, InterruptedException {
     long bytesCopied;
     try {
       bytesCopied = (Long) new RetriableFileCopyCommand(skipCrc, description,
@@ -313,6 +325,14 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESEXPECTED, sourceFileStatus.getLen());
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
+
+    if (verboseLog) {
+      context.write(null,
+          new Text("FILE_COPIED: source=" + sourceFileStatus.getPath() + ","
+          + " size=" + sourceFileStatus.getLen() + " --> " + "target="
+          + target + ", size=" + (targrtFileStatus == null ?
+              0 : targrtFileStatus.getLen())));
+    }
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -322,7 +342,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     } catch (Exception e) {
       throw new IOException("mkdir failed for " + target, e);
     }
-    incrementCounter(context, Counter.COPY, 1);
+    incrementCounter(context, Counter.DIR_COPY, 1);
   }
 
   private static void updateSkipCounters(Context context,

+ 1 - 0
hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm

@@ -221,6 +221,7 @@ Flag              | Description                          | Notes
 `-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions.
 `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted.
 `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed.
+`-v` | Log additional info (path, size) in the SKIP/COPY log | This option can only be used with -log option.
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.
 `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
 `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.

+ 21 - 1
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java

@@ -313,7 +313,7 @@ public class TestDistCpOptions {
         + "preserveRawXattrs=false, atomicWorkPath=null, logPath=null, "
         + "sourceFileListing=abc, sourcePaths=null, targetPath=xyz, "
         + "targetPathExists=true, filtersFile='null', blocksPerChunk=0, "
-        + "copyBufferSize=8192}";
+        + "copyBufferSize=8192, verboseLog=false}";
     String optionString = option.toString();
     Assert.assertEquals(val, optionString);
     Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
@@ -510,4 +510,24 @@ public class TestDistCpOptions {
     options.setFiltersFile("/tmp/filters.txt");
     Assert.assertEquals("/tmp/filters.txt", options.getFiltersFile());
   }
+
+  @Test
+  public void testVerboseLog() {
+    final DistCpOptions options = new DistCpOptions(
+        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+        new Path("hdfs://localhost:8020/target/"));
+    Assert.assertFalse(options.shouldVerboseLog());
+
+    try {
+      options.setVerboseLog(true);
+      fail("-v should fail if -log option is not specified");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("-v is valid only with -log option", e);
+    }
+
+    final Path logPath = new Path("hdfs://localhost:8020/logs");
+    options.setLogPath(logPath);
+    options.setVerboseLog(true);
+    Assert.assertTrue(options.shouldVerboseLog());
+  }
 }

+ 25 - 1
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.tools;
 
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
@@ -408,7 +409,7 @@ public class TestOptionsParser {
         + "preserveRawXattrs=false, atomicWorkPath=null, logPath=null, "
         + "sourceFileListing=abc, sourcePaths=null, targetPath=xyz, "
         + "targetPathExists=true, filtersFile='null', blocksPerChunk=0, "
-        + "copyBufferSize=8192}";
+        + "copyBufferSize=8192, verboseLog=false}";
     String optionString = option.toString();
     Assert.assertEquals(val, optionString);
     Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
@@ -813,4 +814,27 @@ public class TestOptionsParser {
     } catch (IllegalArgumentException ignore) {
     }
   }
+
+  @Test
+  public void testVerboseLog() {
+    DistCpOptions options = OptionsParser
+        .parse(new String[] {"hdfs://localhost:9820/source/first",
+            "hdfs://localhost:9820/target/"});
+    Assert.assertFalse(options.shouldVerboseLog());
+
+    try {
+      OptionsParser
+          .parse(new String[] {"-v", "hdfs://localhost:8020/source/first",
+              "hdfs://localhost:8020/target/"});
+      Assert.fail("-v should fail if -log option is not specified");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("-v is valid only with -log option", e);
+    }
+
+    options = OptionsParser
+        .parse(new String[] {"-log", "hdfs://localhost:8020/logs", "-v",
+            "hdfs://localhost:8020/source/first",
+            "hdfs://localhost:8020/target/"});
+    Assert.assertTrue(options.shouldVerboseLog());
+  }
 }

+ 96 - 3
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java

@@ -255,7 +255,13 @@ public class TestCopyMapper {
     context.getConfiguration().setBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), true);
     copyMapper.setup(context);
+
+    int numFiles = 0;
     for (Path path: pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
       copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
               new CopyListingFileStatus(cluster.getFileSystem().getFileStatus(
                   path)), context);
@@ -266,7 +272,7 @@ public class TestCopyMapper {
     Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
         .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
         .getValue());
-    Assert.assertEquals(pathList.size(), stubContext.getReporter().
+    Assert.assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
   }
 
@@ -295,7 +301,15 @@ public class TestCopyMapper {
 
     copyMapper.setup(context);
 
-    for (Path path: pathList) {
+    int numFiles = 0;
+    int numDirs = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isDirectory()) {
+        numDirs++;
+      } else {
+        numFiles++;
+      }
+
       copyMapper.map(
           new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
           new CopyListingFileStatus(fs.getFileStatus(path)), context);
@@ -303,8 +317,10 @@ public class TestCopyMapper {
 
     // Check that the maps worked.
     verifyCopy(fs, preserveChecksum);
-    Assert.assertEquals(pathList.size(), stubContext.getReporter()
+    Assert.assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
+    Assert.assertEquals(numDirs, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.DIR_COPY).getValue());
     if (!preserveChecksum) {
       Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
@@ -1118,4 +1134,81 @@ public class TestCopyMapper {
       e.printStackTrace();
     }
   }
+
+  @Test
+  public void testVerboseLogging() throws Exception {
+    deleteState();
+    createSourceData();
+
+    FileSystem fs = cluster.getFileSystem();
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(getConfiguration(), null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
+            = stubContext.getContext();
+    copyMapper.setup(context);
+
+    int numFiles = 0;
+    for (Path path : pathList) {
+      if (fs.getFileStatus(path).isFile()) {
+        numFiles++;
+      }
+
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    // Check that the maps worked.
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    testCopyingExistingFiles(fs, copyMapper, context);
+    // verify the verbose log
+    // we shouldn't print verbose log since this option is disabled
+    for (Text value : stubContext.getWriter().values()) {
+      Assert.assertTrue(!value.toString().startsWith("FILE_COPIED:"));
+      Assert.assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
+    }
+
+    // test with verbose logging
+    deleteState();
+    createSourceData();
+
+    stubContext = new StubContext(getConfiguration(), null, 0);
+    context = stubContext.getContext();
+    copyMapper.setup(context);
+
+    // enables verbose logging
+    context.getConfiguration().setBoolean(
+        DistCpOptionSwitch.VERBOSE_LOG.getConfigLabel(), true);
+    copyMapper.setup(context);
+
+    for (Path path : pathList) {
+      copyMapper.map(
+          new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
+          new CopyListingFileStatus(fs.getFileStatus(path)), context);
+    }
+
+    Assert.assertEquals(numFiles, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.COPY).getValue());
+
+    // verify the verbose log of COPY log
+    int numFileCopied = 0;
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_COPIED:")) {
+        numFileCopied++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileCopied);
+
+    // verify the verbose log of SKIP log
+    int numFileSkipped = 0;
+    testCopyingExistingFiles(fs, copyMapper, context);
+    for (Text value : stubContext.getWriter().values()) {
+      if (value.toString().startsWith("FILE_SKIPPED:")) {
+        numFileSkipped++;
+      }
+    }
+    Assert.assertEquals(numFiles, numFileSkipped);
+  }
 }

+ 1 - 0
hadoop-tools/hadoop-extras/src/main/java/org/apache/hadoop/tools/DistCp_Counter.properties

@@ -15,6 +15,7 @@
 CounterGroupName=       distcp
 
 COPY.name=              Files copied
+DIR_COPY.name=          Directories copied
 SKIP.name=              Files skipped
 FAIL.name=              Files failed
 BYTESCOPIED.name=       Bytes copied