Jelajahi Sumber

HADOOP-10561. Copy command with preserve option should handle Xattrs. Contributed by Yi Liu.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1602225 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 tahun lalu
induk
melakukan
2368332138

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -558,6 +558,9 @@ Release 2.5.0 - UNRELEASED
 
     HADOOP-10575. Small fixes for XAttrCommands and test. (Yi Liu via umamahesh)
 
+    HADOOP-10561. Copy command with preserve option should handle Xattrs.
+    (Yi Liu via cnauroth)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 68 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -22,7 +22,12 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.EnumSet;
+import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,7 +50,6 @@ import org.apache.hadoop.io.IOUtils;
 abstract class CommandWithDestination extends FsCommand {  
   protected PathData dst;
   private boolean overwrite = false;
-  private boolean preserve = false;
   private boolean verifyChecksum = true;
   private boolean writeChecksum = true;
   
@@ -74,7 +78,54 @@ abstract class CommandWithDestination extends FsCommand {
    * implementation allows.
    */
   protected void setPreserve(boolean preserve) {
-    this.preserve = preserve;
+    if (preserve) {
+      preserve(FileAttribute.TIMESTAMPS);
+      preserve(FileAttribute.OWNERSHIP);
+      preserve(FileAttribute.PERMISSION);
+    } else {
+      preserveStatus.clear();
+    }
+  }
+  
+  protected static enum FileAttribute {
+    TIMESTAMPS, OWNERSHIP, PERMISSION, XATTR;
+
+    public static FileAttribute getAttribute(char symbol) {
+      for (FileAttribute attribute : values()) {
+        if (attribute.name().charAt(0) == Character.toUpperCase(symbol)) {
+          return attribute;
+        }
+      }
+      throw new NoSuchElementException("No attribute for " + symbol);
+    }
+  }
+  
+  private EnumSet<FileAttribute> preserveStatus = 
+      EnumSet.noneOf(FileAttribute.class);
+  
+  /**
+   * Checks if the input attribute should be preserved or not
+   *
+   * @param attribute - Attribute to check
+   * @return boolean true if attribute should be preserved, false otherwise
+   */
+  private boolean shouldPreserve(FileAttribute attribute) {
+    return preserveStatus.contains(attribute);
+  }
+  
+  /**
+   * Add file attributes that need to be preserved. This method may be
+   * called multiple times to add attributes.
+   *
+   * @param fileAttribute - Attribute to add, one at a time
+   */
+  protected void preserve(FileAttribute fileAttribute) {
+    for (FileAttribute attribute : preserveStatus) {
+      if (attribute.equals(fileAttribute)) {
+        return;
+      }
+    }
+    preserveStatus.add(fileAttribute);
   }
 
   /**
@@ -243,19 +294,33 @@ abstract class CommandWithDestination extends FsCommand {
     try {
       in = src.fs.open(src.path);
       copyStreamToTarget(in, target);
-      if(preserve) {
+      if (shouldPreserve(FileAttribute.TIMESTAMPS)) {
         target.fs.setTimes(
           target.path,
           src.stat.getModificationTime(),
           src.stat.getAccessTime());
+      }
+      if (shouldPreserve(FileAttribute.OWNERSHIP)) {
         target.fs.setOwner(
           target.path,
           src.stat.getOwner(),
           src.stat.getGroup());
+      }
+      if (shouldPreserve(FileAttribute.PERMISSION)) {
         target.fs.setPermission(
           target.path,
           src.stat.getPermission());
       }
+      if (shouldPreserve(FileAttribute.XATTR)) {
+        Map<String, byte[]> srcXAttrs = src.fs.getXAttrs(src.path);
+        if (srcXAttrs != null) {
+          Iterator<Entry<String, byte[]>> iter = srcXAttrs.entrySet().iterator();
+          while (iter.hasNext()) {
+            Entry<String, byte[]> entry = iter.next();
+            target.fs.setXAttr(target.path, entry.getKey(), entry.getValue());
+          }
+        }
+      }
     } finally {
       IOUtils.closeStream(in);
     }

+ 29 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -132,24 +133,47 @@ class CopyCommands {
 
   static class Cp extends CommandWithDestination {
     public static final String NAME = "cp";
-    public static final String USAGE = "[-f] [-p] <src> ... <dst>";
+    public static final String USAGE = "[-f] [-p | -p[topx]] <src> ... <dst>";
     public static final String DESCRIPTION =
       "Copy files that match the file pattern <src> to a\n" +
       "destination.  When copying multiple files, the destination\n" +
-      "must be a directory. Passing -p preserves access and\n" +
-      "modification times, ownership and the mode. Passing -f\n" +
+      "must be a directory. Passing -p preserves status\n" +
+      "[topx] (timestamps, ownership, permission, XAttr).\n" +
+      "If -p is specified with no <arg>, then preserves\n" +
+      "timestamps, ownership, permission. Passing -f\n" +
       "overwrites the destination if it already exists.\n";
     
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "f", "p");
+      popPreserveOption(args);
+      CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "f");
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
-      setPreserve(cf.getOpt("p"));
       // should have a -r option
       setRecursive(true);
       getRemoteDestination(args);
     }
+    
+    private void popPreserveOption(List<String> args) {
+      for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
+        String cur = iter.next();
+        if (cur.equals("--")) {
+          // stop parsing arguments when you see --
+          break;
+        } else if (cur.startsWith("-p")) {
+          iter.remove();
+          if (cur.length() == 2) {
+            setPreserve(true);
+          } else {
+            String attributes = cur.substring(2);
+            for (int index = 0; index < attributes.length(); index++) {
+              preserve(FileAttribute.getAttribute(attributes.charAt(index)));
+            }
+          }
+          return;
+        }
+      }
+    }
   }
   
   /** 

+ 5 - 1
hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm

@@ -159,7 +159,7 @@ count
 
 cp
 
-   Usage: <<<hdfs dfs -cp [-f] URI [URI ...] <dest> >>>
+   Usage: <<<hdfs dfs -cp [-f] [-p | -p[topx]] URI [URI ...] <dest> >>>
 
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
@@ -167,6 +167,10 @@ cp
     Options:
 
       * The -f option will overwrite the destination if it already exists.
+      
+      * The -p option will preserve file attributes [topx] (timestamps, 
+        ownership, permission, XAttr). If -p is specified with no <arg>, 
+        then preserves timestamps, ownership, permission.
 
    Example:
 

+ 11 - 3
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -276,7 +276,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-cp \[-f\] \[-p\] &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
+          <expected-output>^-cp \[-f\] \[-p \| -p\[topx\]\] &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -284,11 +284,19 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*must be a directory.( )*Passing -p preserves access and( )*</expected-output>
+          <expected-output>^( |\t)*must be a directory.( )*Passing -p preserves status( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*modification times, ownership and the mode. Passing -f( )*</expected-output>
+          <expected-output>^( |\t)*\[topx\] \(timestamps, ownership, permission, XAttr\).( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*If -p is specified with no &lt;arg&gt;, then preserves( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*timestamps, ownership, permission. Passing -f( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

+ 88 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -1620,6 +1620,94 @@ public class TestDFSShell {
     int res = admin.run(new String[] {"-refreshNodes"});
     assertEquals("expected to fail -1", res , -1);
   }
+  
+  // Preserve Copy Option is -ptopx (timestamps, ownership, permission, XATTR)
+  @Test (timeout = 120000)
+  public void testCopyCommandsWithPreserveOption() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .format(true).build();
+    FsShell shell = null;
+    FileSystem fs = null;
+    final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithPreserveOption-"
+        + counter.getAndIncrement();
+    final Path hdfsTestDir = new Path(testdir);
+    try {
+      fs = cluster.getFileSystem();
+      fs.mkdirs(hdfsTestDir);
+      Path src = new Path(hdfsTestDir, "srcfile");
+      fs.create(src).close();
+      FileStatus status = fs.getFileStatus(src);
+      final long mtime = status.getModificationTime();
+      final long atime = status.getAccessTime();
+      final String owner = status.getOwner();
+      final String group = status.getGroup();
+      final FsPermission perm = status.getPermission();
+      
+      fs.setXAttr(src, "user.a1", new byte[]{0x31, 0x32, 0x33});
+      fs.setXAttr(src, "trusted.a1", new byte[]{0x31, 0x31, 0x31});
+      
+      shell = new FsShell(conf);
+      
+      // -p
+      Path target1 = new Path(hdfsTestDir, "targetfile1");
+      String[] argv = new String[] { "-cp", "-p", src.toUri().toString(), 
+          target1.toUri().toString() };
+      int ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -p is not working", SUCCESS, ret);
+      FileStatus targetStatus = fs.getFileStatus(target1);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      assertTrue(perm.equals(targetStatus.getPermission()));
+      Map<String, byte[]> xattrs = fs.getXAttrs(target1);
+      assertTrue(xattrs.isEmpty());
+
+      // -ptop
+      Path target2 = new Path(hdfsTestDir, "targetfile2");
+      argv = new String[] { "-cp", "-ptop", src.toUri().toString(), 
+          target2.toUri().toString() };
+      ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -p is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target1);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      assertTrue(perm.equals(targetStatus.getPermission()));
+      xattrs = fs.getXAttrs(target2);
+      assertTrue(xattrs.isEmpty());
+      
+      // -ptopx
+      Path target3 = new Path(hdfsTestDir, "targetfile3");
+      argv = new String[] { "-cp", "-ptopx", src.toUri().toString(), 
+          target3.toUri().toString() };
+      ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -p is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target1);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      assertTrue(perm.equals(targetStatus.getPermission()));
+      xattrs = fs.getXAttrs(target3);
+      assertEquals(xattrs.size(), 2);
+      assertArrayEquals(new byte[]{0x31, 0x32, 0x33}, xattrs.get("user.a1"));
+      assertArrayEquals(new byte[]{0x31, 0x31, 0x31}, xattrs.get("trusted.a1"));
+    } finally {
+      if (null != shell) {
+        shell.close();
+      }
+
+      if (null != fs) {
+        fs.delete(hdfsTestDir, true);
+        fs.close();
+      }
+      cluster.shutdown();
+    }
+  }
 
   // force Copy Option is -f
   @Test (timeout = 30000)