Bläddra i källkod

HADOOP-10919. Copy command should preserve raw.* namespace extended attributes. (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1616840 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 år sedan
förälder
incheckning
e6bdb33784

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

@@ -46,6 +46,9 @@ fs-encryption (Unreleased)
     HADOOP-10853. Refactor get instance of CryptoCodec and support create via
     algorithm/mode/padding. (Yi Liu)
 
+    HADOOP-10919. Copy command should preserve raw.* namespace
+    extended attributes. (clamb)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -57,6 +57,17 @@ abstract class CommandWithDestination extends FsCommand {
   private boolean verifyChecksum = true;
   private boolean writeChecksum = true;
   
+  /**
+   * The name of the raw xattr namespace. It would be nice to use
+   * XAttr.RAW.name() but we can't reference the hadoop-hdfs project.
+   */
+  private final String RAW = "raw.";
+
+  /**
+   * The name of the reserved raw directory.
+   */
+  private final String RESERVED_RAW = "/.reserved/raw";
+
   /**
    * 
    * This method is used to enable the force(-f)  option while copying the files.
@@ -231,7 +242,7 @@ abstract class CommandWithDestination extends FsCommand {
   /**
    * Called with a source and target destination pair
    * @param src for the operation
-   * @param target for the operation
+   * @param dst for the operation
    * @throws IOException if anything goes wrong
    */
   protected void processPath(PathData src, PathData dst) throws IOException {
@@ -253,6 +264,8 @@ abstract class CommandWithDestination extends FsCommand {
       // modify dst as we descend to append the basename of the
       // current directory being processed
       dst = getTargetPath(src);
+      final boolean preserveRawXattrs =
+          checkPathsForReservedRaw(src.path, dst.path);
       if (dst.exists) {
         if (!dst.stat.isDirectory()) {
           throw new PathIsNotDirectoryException(dst.toString());
@@ -268,7 +281,7 @@ abstract class CommandWithDestination extends FsCommand {
       }      
       super.recursePath(src);
       if (dst.stat.isDirectory()) {
-        preserveAttributes(src, dst);
+        preserveAttributes(src, dst, preserveRawXattrs);
       }
     } finally {
       dst = savedDst;
@@ -295,18 +308,60 @@ abstract class CommandWithDestination extends FsCommand {
    * @param target where to copy the item
    * @throws IOException if copy fails
    */ 
-  protected void copyFileToTarget(PathData src, PathData target) throws IOException {
+  protected void copyFileToTarget(PathData src, PathData target)
+      throws IOException {
+    final boolean preserveRawXattrs =
+        checkPathsForReservedRaw(src.path, target.path);
     src.fs.setVerifyChecksum(verifyChecksum);
     InputStream in = null;
     try {
       in = src.fs.open(src.path);
       copyStreamToTarget(in, target);
-      preserveAttributes(src, target);
+      preserveAttributes(src, target, preserveRawXattrs);
     } finally {
       IOUtils.closeStream(in);
     }
   }
   
+  /**
+   * Check the source and target paths to ensure that they are either both in
+   * /.reserved/raw or neither in /.reserved/raw. If neither src nor target are
+   * in /.reserved/raw, then return false, indicating not to preserve raw.*
+   * xattrs. If both src/target are in /.reserved/raw, then return true,
+   * indicating raw.* xattrs should be preserved. If only one of src/target is
+   * in /.reserved/raw then throw an exception.
+   *
+   * @param src The source path to check. This should be a fully-qualified
+   *            path, not relative.
+   * @param target The target path to check. This should be a fully-qualified
+   *               path, not relative.
+   * @return true if raw.* xattrs should be preserved.
+   * @throws PathOperationException is only one of src/target are in
+   * /.reserved/raw.
+   */
+  private boolean checkPathsForReservedRaw(Path src, Path target)
+      throws PathOperationException {
+    final boolean srcIsRR = Path.getPathWithoutSchemeAndAuthority(src).
+        toString().startsWith(RESERVED_RAW);
+    final boolean dstIsRR = Path.getPathWithoutSchemeAndAuthority(target).
+        toString().startsWith(RESERVED_RAW);
+    boolean preserveRawXattrs = false;
+    if (srcIsRR && !dstIsRR) {
+      final String s = "' copy from '" + RESERVED_RAW + "' to non '" +
+          RESERVED_RAW + "'. Either both source and target must be in '" +
+          RESERVED_RAW + "' or neither.";
+      throw new PathOperationException("'" + src.toString() + s);
+    } else if (!srcIsRR && dstIsRR) {
+      final String s = "' copy from non '" + RESERVED_RAW +"' to '" +
+          RESERVED_RAW + "'. Either both source and target must be in '" +
+          RESERVED_RAW + "' or neither.";
+      throw new PathOperationException("'" + dst.toString() + s);
+    } else if (srcIsRR && dstIsRR) {
+      preserveRawXattrs = true;
+    }
+    return preserveRawXattrs;
+  }
+
   /**
    * Copies the stream contents to a temporary file.  If the copy is
    * successful, the temporary file will be renamed to the real path,
@@ -337,9 +392,11 @@ abstract class CommandWithDestination extends FsCommand {
    * attribute to preserve.
    * @param src source to preserve
    * @param target where to preserve attributes
+   * @param preserveRawXAttrs true if raw.* xattrs should be preserved
    * @throws IOException if fails to preserve attributes
    */
-  protected void preserveAttributes(PathData src, PathData target)
+  protected void preserveAttributes(PathData src, PathData target,
+      boolean preserveRawXAttrs)
       throws IOException {
     if (shouldPreserve(FileAttribute.TIMESTAMPS)) {
       target.fs.setTimes(
@@ -369,13 +426,17 @@ abstract class CommandWithDestination extends FsCommand {
         target.fs.setAcl(target.path, srcFullEntries);
       }
     }
-    if (shouldPreserve(FileAttribute.XATTR)) {
+    final boolean preserveXAttrs = shouldPreserve(FileAttribute.XATTR);
+    if (preserveXAttrs || preserveRawXAttrs) {
       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());
+          final String xattrName = entry.getKey();
+          if (xattrName.startsWith(RAW) || preserveXAttrs) {
+            target.fs.setXAttr(target.path, entry.getKey(), entry.getValue());
+          }
         }
       }
     }

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

@@ -143,7 +143,11 @@ class CopyCommands {
       "timestamps, ownership, permission. If -pa is specified, " +
       "then preserves permission also because ACL is a super-set of " +
       "permission. Passing -f overwrites the destination if it " +
-      "already exists.\n";
+      "already exists. raw namespace extended attributes are preserved " +
+      "if (1) they are supported (HDFS only) and, (2) all of the source and " +
+      "target pathnames are in the /.reserved/raw hierarchy. raw namespace " +
+      "xattr preservation is determined solely by the presence (or absence) " +
+      "of the /.reserved/raw prefix and not by the -p option.\n";
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {

+ 9 - 2
hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm

@@ -164,15 +164,22 @@ cp
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
 
+   'raw.*' namespace extended attributes are preserved if (1) the source and
+   destination filesystems support them (HDFS only), and (2) all source and
+   destination pathnames are in the /.reserved/raw hierarchy. Determination of
+   whether raw.* namespace xattrs are preserved is independent of the
+   -p (preserve) flag.
+
     Options:
 
       * The -f option will overwrite the destination if it already exists.
       
-      * The -p option will preserve file attributes [topx] (timestamps, 
+      * The -p option will preserve file attributes [topx] (timestamps,
         ownership, permission, ACL, XAttr). If -p is specified with no <arg>,
         then preserves timestamps, ownership, permission. If -pa is specified,
         then preserves permission also because ACL is a super-set of
-        permission.
+        permission. Determination of whether raw namespace extended attributes
+        are preserved is independent of the -p flag.
 
    Example:
 

+ 169 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -77,6 +77,13 @@ public class TestDFSShell {
 
   static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class);
 
+  private static final String RAW_A1 = "raw.a1";
+  private static final String TRUSTED_A1 = "trusted.a1";
+  private static final String USER_A1 = "user.a1";
+  private static final byte[] RAW_A1_VALUE = new byte[]{0x32, 0x32, 0x32};
+  private static final byte[] TRUSTED_A1_VALUE = new byte[]{0x31, 0x31, 0x31};
+  private static final byte[] USER_A1_VALUE = new byte[]{0x31, 0x32, 0x33};
+
   static Path writeFile(FileSystem fs, Path f) throws IOException {
     DataOutputStream out = fs.create(f);
     out.writeBytes("dhruba: " + f);
@@ -1664,8 +1671,8 @@ public class TestDFSShell {
       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});
+      fs.setXAttr(src, USER_A1, USER_A1_VALUE);
+      fs.setXAttr(src, TRUSTED_A1, TRUSTED_A1_VALUE);
       
       shell = new FsShell(conf);
       
@@ -1722,8 +1729,8 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       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"));
+      assertArrayEquals(USER_A1_VALUE, xattrs.get(USER_A1));
+      assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = fs.getAclStatus(target3).getEntries();
       assertTrue(acls.isEmpty());
       assertFalse(targetPerm.getAclBit());
@@ -1780,6 +1787,160 @@ public class TestDFSShell {
     }
   }
 
+  @Test (timeout = 120000)
+  public void testCopyCommandsWithRawXAttrs() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
+      numDataNodes(1).format(true).build();
+    FsShell shell = null;
+    FileSystem fs = null;
+    final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithRawXAttrs-"
+      + counter.getAndIncrement();
+    final Path hdfsTestDir = new Path(testdir);
+    final Path rawHdfsTestDir = new Path("/.reserved/raw" + testdir);
+    try {
+      fs = cluster.getFileSystem();
+      fs.mkdirs(hdfsTestDir);
+      final Path src = new Path(hdfsTestDir, "srcfile");
+      final String rawSrcBase = "/.reserved/raw" + testdir;
+      final Path rawSrc = new Path(rawSrcBase, "srcfile");
+      fs.create(src).close();
+
+      final Path srcDir = new Path(hdfsTestDir, "srcdir");
+      final Path rawSrcDir = new Path("/.reserved/raw" + testdir, "srcdir");
+      fs.mkdirs(srcDir);
+      final Path srcDirFile = new Path(srcDir, "srcfile");
+      final Path rawSrcDirFile =
+              new Path("/.reserved/raw" + srcDirFile);
+      fs.create(srcDirFile).close();
+
+      final Path[] paths = { rawSrc, rawSrcDir, rawSrcDirFile };
+      final String[] xattrNames = { USER_A1, RAW_A1 };
+      final byte[][] xattrVals = { USER_A1_VALUE, RAW_A1_VALUE };
+
+      for (int i = 0; i < paths.length; i++) {
+        for (int j = 0; j < xattrNames.length; j++) {
+          fs.setXAttr(paths[i], xattrNames[j], xattrVals[j]);
+        }
+      }
+
+      shell = new FsShell(conf);
+
+      /* Check that a file as the source path works ok. */
+      doTestCopyCommandsWithRawXAttrs(shell, fs, src, hdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, rawSrc, hdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, src, rawHdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, rawSrc, rawHdfsTestDir, true);
+
+      /* Use a relative /.reserved/raw path. */
+      final Path savedWd = fs.getWorkingDirectory();
+      try {
+        fs.setWorkingDirectory(new Path(rawSrcBase));
+        final Path relRawSrc = new Path("../srcfile");
+        final Path relRawHdfsTestDir = new Path("..");
+        doTestCopyCommandsWithRawXAttrs(shell, fs, relRawSrc, relRawHdfsTestDir,
+                true);
+      } finally {
+        fs.setWorkingDirectory(savedWd);
+      }
+
+      /* Check that a directory as the source path works ok. */
+      doTestCopyCommandsWithRawXAttrs(shell, fs, srcDir, hdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, rawSrcDir, hdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, srcDir, rawHdfsTestDir, false);
+      doTestCopyCommandsWithRawXAttrs(shell, fs, rawSrcDir, rawHdfsTestDir,
+        true);
+
+      /* Use relative in an absolute path. */
+      final String relRawSrcDir = "./.reserved/../.reserved/raw/../raw" +
+          testdir + "/srcdir";
+      final String relRawDstDir = "./.reserved/../.reserved/raw/../raw" +
+          testdir;
+      doTestCopyCommandsWithRawXAttrs(shell, fs, new Path(relRawSrcDir),
+          new Path(relRawDstDir), true);
+    } finally {
+      if (null != shell) {
+        shell.close();
+      }
+
+      if (null != fs) {
+        fs.delete(hdfsTestDir, true);
+        fs.close();
+      }
+      cluster.shutdown();
+    }
+  }
+
+  private void doTestCopyCommandsWithRawXAttrs(FsShell shell, FileSystem fs,
+      Path src, Path hdfsTestDir, boolean expectRaw) throws Exception {
+    Path target;
+    boolean srcIsRaw;
+    if (src.isAbsolute()) {
+      srcIsRaw = src.toString().contains("/.reserved/raw");
+    } else {
+      srcIsRaw = new Path(fs.getWorkingDirectory(), src).
+          toString().contains("/.reserved/raw");
+    }
+    final boolean destIsRaw = hdfsTestDir.toString().contains("/.reserved/raw");
+    final boolean srcDestMismatch = srcIsRaw ^ destIsRaw;
+
+    // -p (possibly preserve raw if src & dst are both /.r/r */
+    if (srcDestMismatch) {
+      doCopyAndTest(shell, hdfsTestDir, src, "-p", ERROR);
+    } else {
+      target = doCopyAndTest(shell, hdfsTestDir, src, "-p", SUCCESS);
+      checkXAttrs(fs, target, expectRaw, false);
+    }
+
+    // -px (possibly preserve raw, always preserve non-raw xattrs. */
+    if (srcDestMismatch) {
+      doCopyAndTest(shell, hdfsTestDir, src, "-px", ERROR);
+    } else {
+      target = doCopyAndTest(shell, hdfsTestDir, src, "-px", SUCCESS);
+      checkXAttrs(fs, target, expectRaw, true);
+    }
+
+    // no args (possibly preserve raw, never preserve non-raw xattrs. */
+    if (srcDestMismatch) {
+      doCopyAndTest(shell, hdfsTestDir, src, null, ERROR);
+    } else {
+      target = doCopyAndTest(shell, hdfsTestDir, src, null, SUCCESS);
+      checkXAttrs(fs, target, expectRaw, false);
+    }
+  }
+
+  private Path doCopyAndTest(FsShell shell, Path dest, Path src,
+      String cpArgs, int expectedExitCode) throws Exception {
+    final Path target = new Path(dest, "targetfile" +
+        counter.getAndIncrement());
+    final String[] argv = cpArgs == null ?
+        new String[] { "-cp",         src.toUri().toString(),
+            target.toUri().toString() } :
+        new String[] { "-cp", cpArgs, src.toUri().toString(),
+            target.toUri().toString() };
+    final int ret = ToolRunner.run(shell, argv);
+    assertEquals("cp -p is not working", expectedExitCode, ret);
+    return target;
+  }
+
+  private void checkXAttrs(FileSystem fs, Path target, boolean expectRaw,
+      boolean expectVanillaXAttrs) throws Exception {
+    final Map<String, byte[]> xattrs = fs.getXAttrs(target);
+    int expectedCount = 0;
+    if (expectRaw) {
+      assertArrayEquals("raw.a1 has incorrect value",
+          RAW_A1_VALUE, xattrs.get(RAW_A1));
+      expectedCount++;
+    }
+    if (expectVanillaXAttrs) {
+      assertArrayEquals("user.a1 has incorrect value",
+          USER_A1_VALUE, xattrs.get(USER_A1));
+      expectedCount++;
+    }
+    assertEquals("xattrs size mismatch", expectedCount, xattrs.size());
+  }
+
   // verify cp -ptopxa option will preserve directory attributes.
   @Test (timeout = 120000)
   public void testCopyCommandsToDirectoryWithPreserveOption()
@@ -1825,8 +1986,8 @@ public class TestDFSShell {
       final String group = status.getGroup();
       final FsPermission perm = status.getPermission();
 
-      fs.setXAttr(srcDir, "user.a1", new byte[]{0x31, 0x32, 0x33});
-      fs.setXAttr(srcDir, "trusted.a1", new byte[]{0x31, 0x31, 0x31});
+      fs.setXAttr(srcDir, USER_A1, USER_A1_VALUE);
+      fs.setXAttr(srcDir, TRUSTED_A1, TRUSTED_A1_VALUE);
 
       shell = new FsShell(conf);
 
@@ -1883,8 +2044,8 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       xattrs = fs.getXAttrs(targetDir3);
       assertEquals(xattrs.size(), 2);
-      assertArrayEquals(new byte[]{0x31, 0x32, 0x33}, xattrs.get("user.a1"));
-      assertArrayEquals(new byte[]{0x31, 0x31, 0x31}, xattrs.get("trusted.a1"));
+      assertArrayEquals(USER_A1_VALUE, xattrs.get(USER_A1));
+      assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = fs.getAclStatus(targetDir3).getEntries();
       assertTrue(acls.isEmpty());
       assertFalse(targetPerm.getAclBit());