You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by wa...@apache.org on 2014/08/20 20:39:08 UTC

svn commit: r1619197 [4/4] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/bin/ src/main/java/ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/client/ src/main/java/o...

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java Wed Aug 20 18:39:03 2014
@@ -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());

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java Wed Aug 20 18:39:03 2014
@@ -104,7 +104,7 @@ public class TestDFSUtil {
     LocatedBlock l2 = new LocatedBlock(b2, ds, 0, true);
 
     List<LocatedBlock> ls = Arrays.asList(l1, l2);
-    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true);
+    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null);
 
     BlockLocation[] bs = DFSUtil.locatedBlocks2Locations(lbs);
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java Wed Aug 20 18:39:03 2014
@@ -38,7 +38,6 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Random;
-import java.util.concurrent.CancellationException;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.logging.impl.Log4JLogger;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java Wed Aug 20 18:39:03 2014
@@ -1142,7 +1142,7 @@ public class TestFileCreation {
           try {
             nnrpc.create(pathStr, new FsPermission((short)0755), "client",
                 new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)),
-                true, (short)1, 128*1024*1024L);
+                true, (short)1, 128*1024*1024L, null);
             fail("Should have thrown exception when creating '"
                 + pathStr + "'" + " by " + method);
           } catch (InvalidPathException ipe) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java Wed Aug 20 18:39:03 2014
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.anyShort;
 import static org.mockito.Matchers.anyLong;
@@ -29,10 +30,12 @@ import static org.mockito.Mockito.spy;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -339,16 +342,16 @@ public class TestLease {
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
-            1010, 0)).when(mcp).getFileInfo(anyString());
+            1010, 0, null)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0))
+                1010, 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),
-            anyShort(), anyLong());
+            anyShort(), anyLong(), (List<CipherSuite>) anyList());
 
     final Configuration conf = new Configuration();
     final DFSClient c1 = createDFSClientAs(ugi[0], conf);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java Wed Aug 20 18:39:03 2014
@@ -69,6 +69,7 @@ public class FSXAttrBaseTest {
   protected static Configuration conf;
   private static int pathCount = 0;
   protected static Path path;
+  protected static Path rawPath;
   
   // XAttrs
   protected static final String name1 = "user.a1";
@@ -78,6 +79,8 @@ public class FSXAttrBaseTest {
   protected static final byte[] value2 = {0x37, 0x38, 0x39};
   protected static final String name3 = "user.a3";
   protected static final String name4 = "user.a4";
+  protected static final String raw1 = "raw.a1";
+  protected static final String raw2 = "raw.a2";
 
   protected FileSystem fs;
 
@@ -107,6 +110,7 @@ public class FSXAttrBaseTest {
   public void setUp() throws Exception {
     pathCount += 1;
     path = new Path("/p" + pathCount);
+    rawPath = new Path("/.reserved/raw/p" + pathCount);
     initFileSystem();
   }
 
@@ -395,7 +399,8 @@ public class FSXAttrBaseTest {
       Assert.fail("expected IOException");
     } catch (Exception e) {
       GenericTestUtils.assertExceptionContains
-          ("An XAttr name must be prefixed with user/trusted/security/system, " +
+          ("An XAttr name must be prefixed with " +
+           "user/trusted/security/system/raw, " +
            "followed by a '.'",
           e);
     }
@@ -582,7 +587,7 @@ public class FSXAttrBaseTest {
 
     /* Unknown namespace should throw an exception. */
     final String expectedExceptionString = "An XAttr name must be prefixed " +
-        "with user/trusted/security/system, followed by a '.'";
+        "with user/trusted/security/system/raw, followed by a '.'";
     try {
       fs.removeXAttr(path, "wackynamespace.foo");
       Assert.fail("expected IOException");
@@ -918,6 +923,176 @@ public class FSXAttrBaseTest {
     fsAsDiana.removeXAttr(path, name2);
   }
   
+  @Test(timeout = 120000)
+  public void testRawXAttrs() throws Exception {
+    final UserGroupInformation user = UserGroupInformation.
+      createUserForTesting("user", new String[] {"mygroup"});
+
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750));
+    fs.setXAttr(rawPath, raw1, value1, EnumSet.of(XAttrSetFlag.CREATE,
+        XAttrSetFlag.REPLACE));
+
+    {
+      // getXAttr
+      final byte[] value = fs.getXAttr(rawPath, raw1);
+      Assert.assertArrayEquals(value, value1);
+    }
+
+    {
+      // getXAttrs
+      final Map<String, byte[]> xattrs = fs.getXAttrs(rawPath);
+      Assert.assertEquals(xattrs.size(), 1);
+      Assert.assertArrayEquals(value1, xattrs.get(raw1));
+      fs.removeXAttr(rawPath, raw1);
+    }
+
+    {
+      // replace and re-get
+      fs.setXAttr(rawPath, raw1, value1, EnumSet.of(XAttrSetFlag.CREATE));
+      fs.setXAttr(rawPath, raw1, newValue1, EnumSet.of(XAttrSetFlag.CREATE,
+          XAttrSetFlag.REPLACE));
+
+      final Map<String,byte[]> xattrs = fs.getXAttrs(rawPath);
+      Assert.assertEquals(xattrs.size(), 1);
+      Assert.assertArrayEquals(newValue1, xattrs.get(raw1));
+
+      fs.removeXAttr(rawPath, raw1);
+    }
+
+    {
+      // listXAttrs on rawPath ensuring raw.* xattrs are returned
+      fs.setXAttr(rawPath, raw1, value1, EnumSet.of(XAttrSetFlag.CREATE));
+      fs.setXAttr(rawPath, raw2, value2, EnumSet.of(XAttrSetFlag.CREATE));
+
+      final List<String> xattrNames = fs.listXAttrs(rawPath);
+      assertTrue(xattrNames.contains(raw1));
+      assertTrue(xattrNames.contains(raw2));
+      assertTrue(xattrNames.size() == 2);
+      fs.removeXAttr(rawPath, raw1);
+      fs.removeXAttr(rawPath, raw2);
+    }
+
+    {
+      // listXAttrs on non-rawPath ensuring no raw.* xattrs returned
+      fs.setXAttr(rawPath, raw1, value1, EnumSet.of(XAttrSetFlag.CREATE));
+      fs.setXAttr(rawPath, raw2, value2, EnumSet.of(XAttrSetFlag.CREATE));
+
+      final List<String> xattrNames = fs.listXAttrs(path);
+      assertTrue(xattrNames.size() == 0);
+      fs.removeXAttr(rawPath, raw1);
+      fs.removeXAttr(rawPath, raw2);
+    }
+
+    {
+      /*
+       * Test non-root user operations in the "raw.*" namespace.
+       */
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          final FileSystem userFs = dfsCluster.getFileSystem();
+          // Test that non-root can not set xattrs in the "raw.*" namespace
+          try {
+            // non-raw path
+            userFs.setXAttr(path, raw1, value1);
+            fail("setXAttr should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+
+          try {
+            // raw path
+            userFs.setXAttr(rawPath, raw1, value1);
+            fail("setXAttr should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+
+          // Test that non-root can not do getXAttrs in the "raw.*" namespace
+          try {
+            // non-raw path
+            userFs.getXAttrs(rawPath);
+            fail("getXAttrs should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+
+          try {
+            // raw path
+            userFs.getXAttrs(path);
+            fail("getXAttrs should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+
+          // Test that non-root can not do getXAttr in the "raw.*" namespace
+          try {
+            // non-raw path
+            userFs.getXAttr(rawPath, raw1);
+            fail("getXAttr should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+
+          try {
+            // raw path
+            userFs.getXAttr(path, raw1);
+            fail("getXAttr should have thrown");
+          } catch (AccessControlException e) {
+            // ignore
+          }
+          return null;
+        }
+        });
+    }
+
+    {
+      /*
+       * Test that non-root can not do getXAttr in the "raw.*" namespace
+       */
+      fs.setXAttr(rawPath, raw1, value1);
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            try {
+              // non-raw path
+              userFs.getXAttr(rawPath, raw1);
+              fail("getXAttr should have thrown");
+            } catch (AccessControlException e) {
+              // ignore
+            }
+
+            try {
+              // raw path
+              userFs.getXAttr(path, raw1);
+              fail("getXAttr should have thrown");
+            } catch (AccessControlException e) {
+              // ignore
+            }
+
+            /*
+             * Test that only root can see raw.* xattrs returned from listXAttr
+             * and non-root can't do listXAttrs on /.reserved/raw.
+             */
+            // non-raw path
+            final List<String> xattrNames = userFs.listXAttrs(path);
+            assertTrue(xattrNames.size() == 0);
+            try {
+              // raw path
+              userFs.listXAttrs(rawPath);
+              fail("listXAttrs on raw path should have thrown");
+            } catch (AccessControlException e) {
+              // ignore
+            }
+
+            return null;
+          }
+        });
+      fs.removeXAttr(rawPath, raw1);
+    }
+  }
+
   /**
    * Creates a FileSystem for the super-user.
    *

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java Wed Aug 20 18:39:03 2014
@@ -587,7 +587,8 @@ public class NNThroughputBenchmark imple
       // dummyActionNoSynch(fileIdx);
       nameNodeProto.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
                       clientName, new EnumSetWritable<CreateFlag>(EnumSet
-              .of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication, BLOCK_SIZE);
+              .of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, 
+          replication, BLOCK_SIZE, null);
       long end = Time.now();
       for(boolean written = !closeUponCreate; !written; 
         written = nameNodeProto.complete(fileNames[daemonId][inputIdx],
@@ -1133,7 +1134,7 @@ public class NNThroughputBenchmark imple
         String fileName = nameGenerator.getNextFileName("ThroughputBench");
         nameNodeProto.create(fileName, FsPermission.getDefault(), clientName,
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
-            BLOCK_SIZE);
+            BLOCK_SIZE, null);
         ExtendedBlock lastBlock = addBlocks(fileName, clientName);
         nameNodeProto.complete(fileName, clientName, lastBlock, INodeId.GRANDFATHER_INODE_ID);
       }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java Wed Aug 20 18:39:03 2014
@@ -128,7 +128,7 @@ public class TestAddBlockRetry {
     nn.create(src, FsPermission.getFileDefault(),
         "clientName",
         new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)),
-        true, (short)3, 1024);
+        true, (short)3, 1024, null);
 
     // start first addBlock()
     LOG.info("Starting first addBlock for " + src);
@@ -155,7 +155,7 @@ public class TestAddBlockRetry {
     // create file
     nameNodeRpc.create(src, FsPermission.getFileDefault(), "clientName",
         new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
-        (short) 3, 1024);
+        (short) 3, 1024, null);
     // start first addBlock()
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock lb1 = nameNodeRpc.addBlock(src, "clientName", null, null,

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java Wed Aug 20 18:39:03 2014
@@ -191,14 +191,19 @@ public class TestFSDirectory {
     existingXAttrs.add(xAttr1);
     existingXAttrs.add(xAttr2);
     
-    // Adding a system namespace xAttr, isn't affected by inode xAttrs limit.
-    XAttr newXAttr = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.SYSTEM).
+    // Adding system and raw namespace xAttrs aren't affected by inode
+    // xAttrs limit.
+    XAttr newSystemXAttr = (new XAttr.Builder()).
+        setNameSpace(XAttr.NameSpace.SYSTEM).setName("a3").
+        setValue(new byte[]{0x33, 0x33, 0x33}).build();
+    XAttr newRawXAttr = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.RAW).
         setName("a3").setValue(new byte[]{0x33, 0x33, 0x33}).build();
-    List<XAttr> newXAttrs = Lists.newArrayListWithCapacity(1);
-    newXAttrs.add(newXAttr);
+    List<XAttr> newXAttrs = Lists.newArrayListWithCapacity(2);
+    newXAttrs.add(newSystemXAttr);
+    newXAttrs.add(newRawXAttr);
     List<XAttr> xAttrs = fsdir.setINodeXAttrs(existingXAttrs, newXAttrs,
         EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
-    assertEquals(xAttrs.size(), 3);
+    assertEquals(xAttrs.size(), 4);
     
     // Adding a trusted namespace xAttr, is affected by inode xAttrs limit.
     XAttr newXAttr1 = (new XAttr.Builder()).setNameSpace(

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java Wed Aug 20 18:39:03 2014
@@ -1018,7 +1018,7 @@ public class TestFsck {
 
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
         blockSize, modTime, accessTime, perms, owner, group, symlink, path,
-        fileId, numChildren);
+        fileId, numChildren, null);
     Result res = new Result(conf);
 
     try {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java Wed Aug 20 18:39:03 2014
@@ -209,19 +209,20 @@ public class TestNamenodeRetryCache {
     // Two retried calls succeed
     newCall();
     HdfsFileStatus status = namesystem.startFile(src, perm, "holder",
-        "clientmachine", EnumSet.of(CreateFlag.CREATE), true, (short) 1, BlockSize);
+        "clientmachine", EnumSet.of(CreateFlag.CREATE), true, (short) 1, 
+        BlockSize, null);
     Assert.assertEquals(status, namesystem.startFile(src, perm, 
         "holder", "clientmachine", EnumSet.of(CreateFlag.CREATE), 
-        true, (short) 1, BlockSize));
+        true, (short) 1, BlockSize, null));
     Assert.assertEquals(status, namesystem.startFile(src, perm, 
         "holder", "clientmachine", EnumSet.of(CreateFlag.CREATE), 
-        true, (short) 1, BlockSize));
+        true, (short) 1, BlockSize, null));
     
     // A non-retried call fails
     newCall();
     try {
       namesystem.startFile(src, perm, "holder", "clientmachine",
-          EnumSet.of(CreateFlag.CREATE), true, (short) 1, BlockSize);
+          EnumSet.of(CreateFlag.CREATE), true, (short) 1, BlockSize, null);
       Assert.fail("testCreate - expected exception is not thrown");
     } catch (IOException e) {
       // expected

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java Wed Aug 20 18:39:03 2014
@@ -395,7 +395,7 @@ public class TestRetryCacheWithHA {
       this.status = client.getNamenode().create(fileName,
           FsPermission.getFileDefault(), client.getClientName(),
           new EnumSetWritable<CreateFlag>(createFlag), false, DataNodes,
-          BlockSize);
+          BlockSize, null);
     }
 
     @Override

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java Wed Aug 20 18:39:03 2014
@@ -64,7 +64,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        INodeId.GRANDFATHER_INODE_ID, 0);
+        INodeId.GRANDFATHER_INODE_ID, 0, null);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testXAttrConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testXAttrConf.xml?rev=1619197&r1=1619196&r2=1619197&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testXAttrConf.xml (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testXAttrConf.xml Wed Aug 20 18:39:03 2014
@@ -64,7 +64,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>name must be prefixed with user/trusted/security/system, followed by a '.'</expected-output>
+          <expected-output>name must be prefixed with user/trusted/security/system/raw, followed by a '.'</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -126,6 +126,42 @@
     </test>
     
     <test>
+      <description>setfattr : Add an xattr of raw namespace</description>
+      <test-commands>
+          <command>-fs NAMENODE -touchz /file1</command>
+          <command>-fs NAMENODE -setfattr -n raw.a1 -v 123456 /file1</command>
+      </test-commands>
+      <cleanup-commands>
+          <command>-fs NAMENODE -rm /file1</command>
+      </cleanup-commands>
+      <comparators>
+          <comparator>
+              <type>SubstringComparator</type>
+              <expected-output>setfattr: User doesn't have permission for xattr: raw.a1</expected-output>
+          </comparator>
+      </comparators>
+
+    </test>
+
+    <test>
+        <description>setfattr : Add an xattr of raw namespace</description>
+        <test-commands>
+            <command>-fs NAMENODE -touchz /file1</command>
+            <command>-fs NAMENODE -setfattr -n raw.a1 -v 123456 /.reserved/raw/file1</command>
+            <command>-fs NAMENODE -getfattr -n raw.a1 /.reserved/raw/file1</command>
+        </test-commands>
+        <cleanup-commands>
+            <command>-fs NAMENODE -rm /file1</command>
+        </cleanup-commands>
+        <comparators>
+            <comparator>
+                <type>SubstringComparator</type>
+                <expected-output>raw.a1="123456"</expected-output>
+            </comparator>
+        </comparators>
+    </test>
+
+    <test>
       <description>setfattr : Add an xattr, and encode is text</description>
       <test-commands>
         <command>-fs NAMENODE -touchz /file1</command>
@@ -256,6 +292,26 @@
         </comparator>
       </comparators>
     </test>
+
+    <test>
+        <description>setfattr : Remove an xattr of raw namespace</description>
+        <test-commands>
+            <command>-fs NAMENODE -touchz /file1</command>
+            <command>-fs NAMENODE -setfattr -n raw.a1 -v 123456 /.reserved/raw/file1</command>
+            <command>-fs NAMENODE -setfattr -n raw.a2 -v 123456 /.reserved/raw/file1</command>
+            <command>-fs NAMENODE -setfattr -x raw.a2 /.reserved/raw/file1</command>
+            <command>-fs NAMENODE -getfattr -d /.reserved/raw/file1</command>
+        </test-commands>
+        <cleanup-commands>
+            <command>-fs NAMENODE -rm /file1</command>
+        </cleanup-commands>
+        <comparators>
+            <comparator>
+                <type>SubstringComparator</type>
+		<expected-output># file: /.reserved/raw/file1#LF#raw.a1="123456"#LF#</expected-output>
+            </comparator>
+        </comparators>
+    </test>
     
     <test>
       <description>getfattr : Get an xattr</description>