Browse Source

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 years ago
parent
commit
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-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
 Release 2.4.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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.io.InputStream;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.util.EnumSet;
+import java.util.Iterator;
 import java.util.LinkedList;
 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.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,7 +50,6 @@ import org.apache.hadoop.io.IOUtils;
 abstract class CommandWithDestination extends FsCommand {  
 abstract class CommandWithDestination extends FsCommand {  
   protected PathData dst;
   protected PathData dst;
   private boolean overwrite = false;
   private boolean overwrite = false;
-  private boolean preserve = false;
   private boolean verifyChecksum = true;
   private boolean verifyChecksum = true;
   private boolean writeChecksum = true;
   private boolean writeChecksum = true;
   
   
@@ -74,7 +78,54 @@ abstract class CommandWithDestination extends FsCommand {
    * implementation allows.
    * implementation allows.
    */
    */
   protected void setPreserve(boolean preserve) {
   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 {
     try {
       in = src.fs.open(src.path);
       in = src.fs.open(src.path);
       copyStreamToTarget(in, target);
       copyStreamToTarget(in, target);
-      if(preserve) {
+      if (shouldPreserve(FileAttribute.TIMESTAMPS)) {
         target.fs.setTimes(
         target.fs.setTimes(
           target.path,
           target.path,
           src.stat.getModificationTime(),
           src.stat.getModificationTime(),
           src.stat.getAccessTime());
           src.stat.getAccessTime());
+      }
+      if (shouldPreserve(FileAttribute.OWNERSHIP)) {
         target.fs.setOwner(
         target.fs.setOwner(
           target.path,
           target.path,
           src.stat.getOwner(),
           src.stat.getOwner(),
           src.stat.getGroup());
           src.stat.getGroup());
+      }
+      if (shouldPreserve(FileAttribute.PERMISSION)) {
         target.fs.setPermission(
         target.fs.setPermission(
           target.path,
           target.path,
           src.stat.getPermission());
           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 {
     } finally {
       IOUtils.closeStream(in);
       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.io.InputStream;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
@@ -132,24 +133,47 @@ class CopyCommands {
 
 
   static class Cp extends CommandWithDestination {
   static class Cp extends CommandWithDestination {
     public static final String NAME = "cp";
     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 =
     public static final String DESCRIPTION =
       "Copy files that match the file pattern <src> to a\n" +
       "Copy files that match the file pattern <src> to a\n" +
       "destination.  When copying multiple files, the destination\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";
       "overwrites the destination if it already exists.\n";
     
     
     @Override
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
     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);
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
       setOverwrite(cf.getOpt("f"));
-      setPreserve(cf.getOpt("p"));
       // should have a -r option
       // should have a -r option
       setRecursive(true);
       setRecursive(true);
       getRemoteDestination(args);
       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
 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
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
    as well in which case the destination must be a directory.
@@ -167,6 +167,10 @@ cp
     Options:
     Options:
 
 
       * The -f option will overwrite the destination if it already exists.
       * 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:
    Example:
 
 

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

@@ -276,7 +276,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
@@ -284,11 +284,19 @@
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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"});
     int res = admin.run(new String[] {"-refreshNodes"});
     assertEquals("expected to fail -1", res , -1);
     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
   // force Copy Option is -f
   @Test (timeout = 30000)
   @Test (timeout = 30000)