Просмотр исходного кода

HDFS-6377. Unify xattr name and value limits into a single limit. Contributed by Andrew Wang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2006@1594404 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 лет назад
Родитель
Сommit
427ad33f72

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt

@@ -29,6 +29,8 @@ HDFS-2006 (Unreleased)
 
     HDFS-6344. Maximum limit on the size of an xattr. (Yi Liu via umamahesh)
 
+    HDFS-6377. Unify xattr name and value limits into a single limit. (wang)
+
   OPTIMIZATIONS
 
     HDFS-6346. Optimize OP_SET_XATTRS by persisting single Xattr entry per setXattr/removeXattr api call

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -194,12 +194,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_NAMENODE_ACLS_ENABLED_DEFAULT = false;
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
-  public static final String  DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_KEY = "dfs.namenode.inode.xattrs.max-limit";
-  public static final int     DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_DEFAULT = 32;
-  public static final String  DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_KEY = "dfs.namenode.xattr.name.max-length";
-  public static final int     DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_DEFAULT = 64;
-  public static final String  DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_KEY = "dfs.namenode.xattr.value.max-length";
-  public static final int     DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_DEFAULT = 16384;
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY = "dfs.https.server.keystore.resource";
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT = "ssl-server.xml";
@@ -299,6 +293,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT = 1024*1024;
   public static final String  DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY = "dfs.namenode.fs-limits.max-blocks-per-file";
   public static final long    DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT = 1024*1024;
+  public static final String  DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY = "dfs.namenode.fs-limits.max-xattrs-per-inode";
+  public static final int     DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT = 32;
+  public static final String  DFS_NAMENODE_MAX_XATTR_SIZE_KEY = "dfs.namenode.fs-limits.max-xattr-size";
+  public static final int     DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT = 16384;
+
 
   //Following keys have no defaults
   public static final String  DFS_DATANODE_DATA_DIR_KEY = "dfs.datanode.data.dir";

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java

@@ -44,14 +44,14 @@ public class XAttrHelper {
    * Both name and namespace are case sensitive.
    */
   public static XAttr buildXAttr(String name, byte[] value) {
-    Preconditions.checkNotNull(name, "XAttr name can not be null.");
+    Preconditions.checkNotNull(name, "XAttr name cannot be null.");
     
     final int prefixIndex = name.indexOf(".");
     if (prefixIndex < 4) {// Prefix length is at least 4.
       throw new HadoopIllegalArgumentException("An XAttr name must be " +
           "prefixed with user/trusted/security/system, followed by a '.'");
     } else if (prefixIndex == name.length() - 1) {
-      throw new HadoopIllegalArgumentException("XAttr name can not be empty.");
+      throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
     }
     
     NameSpace ns;

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -196,8 +196,11 @@ public class FSDirectory implements Closeable {
         DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY,
         DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT);
     this.inodeXAttrsLimit = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_KEY,
-        DFSConfigKeys.DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_DEFAULT);
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY,
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT);
+    Preconditions.checkArgument(this.inodeXAttrsLimit >= 0,
+        "Cannot set a negative limit on the number of xattrs per inode (%s).",
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY);
     // We need a maximum maximum because by default, PB limits message sizes
     // to 64MB. This means we can only store approximately 6.7 million entries
     // per directory, but let's use 6.4 million for some safety.
@@ -2938,8 +2941,8 @@ public class FSDirectory implements Closeable {
     xAttrs.add(xAttr);
     
     if (xAttrs.size() > inodeXAttrsLimit) {
-      throw new IOException("Operation fails, XAttrs of " +
-          "inode exceeds maximum limit of " + inodeXAttrsLimit);
+      throw new IOException("Cannot add additional XAttr to inode, "
+          + "would exceed limit of " + inodeXAttrsLimit);
     }
     
     return xAttrs;

+ 20 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -96,6 +96,7 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.URI;
@@ -1123,8 +1124,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // so that the tailer starts from the right spot.
         dir.fsImage.updateLastAppliedTxIdFromWritten();
       }
-      cacheManager.stopMonitorThread();
-      cacheManager.clearDirectiveStats();
+      if (cacheManager != null) {
+        cacheManager.stopMonitorThread();
+        cacheManager.clearDirectiveStats();
+      }
       blockManager.getDatanodeManager().clearPendingCachingCommands();
       blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
       // Don't want to keep replication queues when not in Active.
@@ -7820,20 +7823,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     logAuditEvent(true, "setXAttr", src, null, resultingStat);
   }
-  
+
+  /**
+   * Verifies that the combined size of the name and value of an xattr is within
+   * the configured limit. Setting a limit of zero disables this check.
+   */
   private void checkXAttrSize(XAttr xAttr) {
-    if (xAttr.getName().length() > nnConf.xattrNameMaxLength) {
-      throw new HadoopIllegalArgumentException(
-          "XAttr name is too long, maximum length = "
-              + nnConf.xattrNameMaxLength + ", but now the length = "
-              + xAttr.getName().length());
+    if (nnConf.xattrMaxSize == 0) {
+      return;
+    }
+    int size = xAttr.getName().getBytes(Charsets.UTF_8).length;
+    if (xAttr.getValue() != null) {
+      size += xAttr.getValue().length;
     }
-    if (xAttr.getValue() != null
-        && xAttr.getValue().length > nnConf.xattrValueMaxLength) {
+    if (size > nnConf.xattrMaxSize) {
       throw new HadoopIllegalArgumentException(
-          "XAttr value is too long, maximum length = "
-              + nnConf.xattrValueMaxLength + ", but now the length = "
-              + xAttr.getValue().length);
+          "The XAttr is too big. The maximum combined size of the"
+          + " name and value is " + nnConf.xattrMaxSize
+          + ", but the total size is " + size);
     }
   }
   

+ 15 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java

@@ -24,6 +24,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.AclException;
 
+import com.google.common.base.Preconditions;
+
 /**
  * This class is a common place for NN configuration.
  */
@@ -42,8 +44,10 @@ final class NNConf {
    */
   private final boolean xattrsEnabled;
   
-  final int xattrNameMaxLength;
-  final int xattrValueMaxLength;
+  /**
+   * Maximum size of a single name-value extended attribute.
+   */
+  final int xattrMaxSize;
 
   /**
    * Creates a new NNConf from configuration.
@@ -58,12 +62,15 @@ final class NNConf {
         DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
         DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
     LogFactory.getLog(NNConf.class).info("XAttrs enabled? " + xattrsEnabled);
-    xattrNameMaxLength = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_KEY, 
-        DFSConfigKeys.DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_DEFAULT);
-    xattrValueMaxLength = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_KEY, 
-        DFSConfigKeys.DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_DEFAULT);
+    xattrMaxSize = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
+    Preconditions.checkArgument(xattrMaxSize >= 0,
+        "Cannot set a negative value for the maximum size of an xattr (%s).",
+        DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
+    final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
+    LogFactory.getLog(NNConf.class).info(
+        "Maximum size of an xattr: " + xattrMaxSize + unlimited);
   }
 
   /**

+ 40 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java

@@ -28,12 +28,16 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -43,7 +47,9 @@ import com.google.common.collect.Lists;
  * This test suite covers restarting the NN, saving a new checkpoint. 
  */
 public class FSXAttrBaseTest {
-  
+
+  private static final int MAX_SIZE = 16;
+
   protected static MiniDFSCluster dfsCluster;
   protected static Configuration conf;
   private static int pathCount = 0;
@@ -60,6 +66,15 @@ public class FSXAttrBaseTest {
 
   protected FileSystem fs;
 
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 3);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY, MAX_SIZE);
+    initCluster(true);
+  }
+
   @AfterClass
   public static void shutdown() {
     if (dfsCluster != null) {
@@ -87,7 +102,7 @@ public class FSXAttrBaseTest {
    * 3. Create multiple xattrs.
    * 4. Restart NN and save checkpoint scenarios.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testCreateXAttr() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
@@ -143,7 +158,7 @@ public class FSXAttrBaseTest {
    * 3. Create multiple xattrs and replace some.
    * 4. Restart NN and save checkpoint scenarios.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testReplaceXAttr() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
@@ -198,7 +213,7 @@ public class FSXAttrBaseTest {
    * 5. Set xattr and name is too long.
    * 6. Set xattr and value is too long.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testSetXAttr() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE, 
@@ -215,6 +230,7 @@ public class FSXAttrBaseTest {
           XAttrSetFlag.REPLACE));
       Assert.fail("Setting xattr with null name should fail.");
     } catch (NullPointerException e) {
+      GenericTestUtils.assertExceptionContains("XAttr name cannot be null", e);
     }
     
     // Set xattr with empty name: "user."
@@ -223,6 +239,7 @@ public class FSXAttrBaseTest {
           XAttrSetFlag.REPLACE));
       Assert.fail("Setting xattr with empty name should fail.");
     } catch (HadoopIllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains("XAttr name cannot be empty", e);
     }
     
     // Set xattr with invalid name: "a1"
@@ -232,6 +249,7 @@ public class FSXAttrBaseTest {
       Assert.fail("Setting xattr with invalid name prefix or without " +
           "name prefix should fail.");
     } catch (HadoopIllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains("XAttr name must be prefixed", e);
     }
     
     // Set xattr without XAttrSetFlag
@@ -261,25 +279,36 @@ public class FSXAttrBaseTest {
       Assert.fail("Setting xattr should fail if total number of xattrs " +
           "for inode exceeds max limit.");
     } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Cannot add additional XAttr", e);
     }
     fs.removeXAttr(path, name1);
     fs.removeXAttr(path, name2);
     fs.removeXAttr(path, name3);
     
     // Name length exceeds max limit
-    String longName = "user.abcdefg123456789000";
+    String longName = "user.0123456789abcdefX";
     try {
-      fs.setXAttr(path, longName, value1);
+      fs.setXAttr(path, longName, null);
       Assert.fail("Setting xattr should fail if name is too long.");
     } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("XAttr is too big", e);
+      GenericTestUtils.assertExceptionContains("total size is 17", e);
     }
+
     // Value length exceeds max limit
-    byte[] longValue = new byte[40];
+    byte[] longValue = new byte[MAX_SIZE];
     try {
-      fs.setXAttr(path, name1, longValue);
+      fs.setXAttr(path, "user.a", longValue);
       Assert.fail("Setting xattr should fail if value is too long.");
     } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("XAttr is too big", e);
+      GenericTestUtils.assertExceptionContains("total size is 17", e);
     }
+
+    // Name + value exactly equal the limit
+    String name = "user.111";
+    byte[] value = new byte[MAX_SIZE-3];
+    fs.setXAttr(path, name, value);
   }
   
   /**
@@ -287,7 +316,7 @@ public class FSXAttrBaseTest {
    * 1. To get xattr which does not exist.
    * 2. To get multiple xattrs.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testGetXAttrs() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
@@ -315,7 +344,7 @@ public class FSXAttrBaseTest {
    * 1. Remove xattr.
    * 2. Restart NN and save checkpoint scenarios.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testRemoveXAttr() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
@@ -354,7 +383,7 @@ public class FSXAttrBaseTest {
    * 6) Restart NN without saving a checkpoint.
    * 7) Set xattrs again on the same file.
    */
-  @Test
+  @Test(timeout = 120000)
   public void testCleanupXAttrs() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));

+ 0 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java

@@ -37,15 +37,6 @@ import org.junit.BeforeClass;
  * Tests of XAttr operations using FileContext APIs.
  */
 public class TestFileContextXAttr extends FSXAttrBaseTest  {
-  @BeforeClass
-  public static void init() throws Exception {
-    conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_KEY, 3);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_KEY, 16);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_KEY, 32);
-    initCluster(true);
-  }
 
   @Override
   protected FileSystem createFileSystem() throws Exception {

+ 1 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeXAttr.java

@@ -19,12 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -39,17 +36,7 @@ public class TestNameNodeXAttr extends FSXAttrBaseTest {
   private static final Path link = new Path(linkParent, "link");
   private static final Path target = new Path(targetParent, "target");
 
-  @BeforeClass
-  public static void init() throws Exception {
-    conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_KEY, 3);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_XATTR_NAME_MAX_LENGTH_KEY, 16);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_XATTR_VALUE_MAX_LENGTH_KEY, 32);
-    initCluster(true);
-  }
-
-  @Test
+  @Test(timeout = 120000)
   public void testXAttrSymlinks() throws Exception {
     fs.mkdirs(linkParent);
     fs.mkdirs(targetParent);

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -620,4 +620,68 @@ public class TestStartup {
     fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
   }
+
+
+  @Test(timeout = 120000)
+  public void testXattrConfiguration() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+
+    try {
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY, -1);
+      cluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(0).format(true).build();
+      fail("Expected exception with negative xattr size");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot set a negative value for the maximum size of an xattr", e);
+    } finally {
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+
+    try {
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, -1);
+      cluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(0).format(true).build();
+      fail("Expected exception with negative # xattrs per inode");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot set a negative limit on the number of xattrs per inode", e);
+    } finally {
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+
+    try {
+      // Set up a logger to check log message
+      final LogVerificationAppender appender = new LogVerificationAppender();
+      final Logger logger = Logger.getRootLogger();
+      logger.addAppender(appender);
+      int count = appender.countLinesWithMessage(
+          "Maximum size of an xattr: 0 (unlimited)");
+      assertEquals("Expected no messages about unlimited xattr size", 0, count);
+
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY, 0);
+      cluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(0).format(true).build();
+
+      count = appender.countLinesWithMessage(
+          "Maximum size of an xattr: 0 (unlimited)");
+      // happens twice because we format then run
+      assertEquals("Expected unlimited xattr size", 2, count);
+    } finally {
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }