فهرست منبع

HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to FSNamesystem. Contributed by Li Lu.

Haohui Mai 10 سال پیش
والد
کامیت
2caed865e9

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

@@ -173,6 +173,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7468. Moving verify* functions to corresponding classes.
     (Li Lu via wheat9)
 
+    HDFS-7478. Move org.apache.hadoop.hdfs.server.namenode.NNConf to
+    FSNamesystem. (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 51 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -521,7 +521,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   private final RetryCache retryCache;
 
-  private final NNConf nnConf;
+  private final boolean aclsEnabled;
+  private final boolean xattrsEnabled;
+  private final int xattrMaxSize;
 
   private KeyProviderCryptoExtension provider = null;
 
@@ -839,7 +841,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
         auditLoggers.get(0) instanceof DefaultAuditLogger;
       this.retryCache = ignoreRetryCache ? null : initRetryCache(conf);
-      this.nnConf = new NNConf(conf);
+
+      this.aclsEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
+      LOG.info("ACLs enabled? " + aclsEnabled);
+      this.xattrsEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
+      LOG.info("XAttrs enabled? " + xattrsEnabled);
+      this.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)" : "";
+      LOG.info("Maximum size of an xattr: " + xattrMaxSize + unlimited);
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
@@ -7827,7 +7845,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void modifyAclEntries(final String srcArg, List<AclEntry> aclSpec)
       throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7854,7 +7872,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void removeAclEntries(final String srcArg, List<AclEntry> aclSpec)
       throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7880,7 +7898,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void removeDefaultAcl(final String srcArg) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7906,7 +7924,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void removeAcl(final String srcArg) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7932,7 +7950,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   void setAcl(final String srcArg, List<AclEntry> aclSpec) throws IOException {
     String src = srcArg;
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -7957,7 +7975,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   AclStatus getAclStatus(String src) throws IOException {
-    nnConf.checkAclsConfigFlag();
+    checkAclsConfigFlag();
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -8141,7 +8159,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private void setXAttrInt(final String srcArg, XAttr xAttr,
       EnumSet<XAttrSetFlag> flag, boolean logRetryCache) throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     checkXAttrSize(xAttr);
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8172,17 +8190,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * the configured limit. Setting a limit of zero disables this check.
    */
   private void checkXAttrSize(XAttr xAttr) {
-    if (nnConf.xattrMaxSize == 0) {
+    if (xattrMaxSize == 0) {
       return;
     }
     int size = xAttr.getName().getBytes(Charsets.UTF_8).length;
     if (xAttr.getValue() != null) {
       size += xAttr.getValue().length;
     }
-    if (size > nnConf.xattrMaxSize) {
+    if (size > xattrMaxSize) {
       throw new HadoopIllegalArgumentException(
           "The XAttr is too big. The maximum combined size of the"
-          + " name and value is " + nnConf.xattrMaxSize
+          + " name and value is " + xattrMaxSize
           + ", but the total size is " + size);
     }
   }
@@ -8190,7 +8208,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   List<XAttr> getXAttrs(final String srcArg, List<XAttr> xAttrs)
       throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     FSPermissionChecker pc = getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     boolean getAll = xAttrs == null || xAttrs.isEmpty();
@@ -8247,7 +8265,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   List<XAttr> listXAttrs(String src) throws IOException {
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     final FSPermissionChecker pc = getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     checkOperation(OperationCategory.READ);
@@ -8297,7 +8315,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void removeXAttrInt(final String srcArg, XAttr xAttr, boolean logRetryCache)
       throws IOException {
     String src = srcArg;
-    nnConf.checkXAttrsConfigFlag();
+    checkXAttrsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     XAttrPermissionFilter.checkPermissionForApi(pc, xAttr,
@@ -8453,5 +8471,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       logger.addAppender(asyncAppender);        
     }
   }
+
+  private void checkAclsConfigFlag() throws AclException {
+    if (!aclsEnabled) {
+      throw new AclException(String.format(
+          "The ACL operation has been rejected.  "
+              + "Support for ACLs has been disabled by setting %s to false.",
+          DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
+    }
+  }
+
+  private void checkXAttrsConfigFlag() throws IOException {
+    if (!xattrsEnabled) {
+      throw new IOException(String.format(
+          "The XAttr operation has been rejected.  "
+              + "Support for XAttrs has been disabled by setting %s to false.",
+          DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY));
+    }
+  }
 }
 

+ 0 - 104
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNConf.java

@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-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.
- */
-@InterfaceAudience.Private
-final class NNConf {
-  /**
-   * Support for ACLs is controlled by a configuration flag. If the 
-   * configuration flag is false, then the NameNode will reject all 
-   * ACL-related operations.
-   */
-  private final boolean aclsEnabled;
-  
-  /**
-   * Support for XAttrs is controlled by a configuration flag. If the 
-   * configuration flag is false, then the NameNode will reject all 
-   * XAttr-related operations.
-   */
-  private final boolean xattrsEnabled;
-  
-  /**
-   * Maximum size of a single name-value extended attribute.
-   */
-  final int xattrMaxSize;
-
-  /**
-   * Creates a new NNConf from configuration.
-   *
-   * @param conf Configuration to check
-   */
-  public NNConf(Configuration conf) {
-    aclsEnabled = conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
-      DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
-    LogFactory.getLog(NNConf.class).info("ACLs enabled? " + aclsEnabled);
-    xattrsEnabled = conf.getBoolean(
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
-    LogFactory.getLog(NNConf.class).info("XAttrs enabled? " + xattrsEnabled);
-    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);
-  }
-
-  /**
-   * Checks the flag on behalf of an ACL API call.
-   *
-   * @throws AclException if ACLs are disabled
-   */
-  public void checkAclsConfigFlag() throws AclException {
-    if (!aclsEnabled) {
-      throw new AclException(String.format(
-        "The ACL operation has been rejected.  "
-        + "Support for ACLs has been disabled by setting %s to false.",
-        DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY));
-    }
-  }
-  
-  /**
-   * Checks the flag on behalf of an XAttr API call.
-   * @throws IOException if XAttrs are disabled
-   */
-  public void checkXAttrsConfigFlag() throws IOException {
-    if (!xattrsEnabled) {
-      throw new IOException(String.format(
-        "The XAttr operation has been rejected.  "
-        + "Support for XAttrs has been disabled by setting %s to false.",
-        DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY));
-    }
-  }
-}