瀏覽代碼

HDFS-2264. NamenodeProtocol has the wrong value for clientPrincipal in KerberosInfo annotation. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1560511 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 年之前
父節點
當前提交
0abf9f1bcf

+ 3 - 0
CHANGES.txt

@@ -193,6 +193,9 @@ Release 1.3.0 - unreleased
     MAPREDUCE-5710. Backport MAPREDUCE-1305 to branch-1 (Yongjun Zhang via
     Sandy Ryza)
 
+    HDFS-2264. NamenodeProtocol has the wrong value for clientPrincipal in 
+    KerberosInfo annotation. (Aaron T. Myers, backported by jing9)
+
 Release 1.2.2 - unreleased
 
   INCOMPATIBLE CHANGES

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -5743,7 +5743,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     checkPermission(pc, path, false, null, null, null, null);
   }
 
-  private void checkSuperuserPrivilege() throws AccessControlException {
+  void checkSuperuserPrivilege() throws AccessControlException {
     if (isPermissionEnabled) {
       FSPermissionChecker pc = getPermissionChecker();
       pc.checkSuperuserPrivilege();

+ 20 - 24
src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 
 import java.io.File;
 import java.io.IOException;
@@ -26,8 +26,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -88,15 +86,15 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.security.authorize.PolicyProvider;
-import org.apache.hadoop.util.ReflectionUtils;
 
 /**********************************************************
  * NameNode serves as both directory namespace manager and
@@ -141,6 +139,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     Configuration.addDefaultResource("hdfs-site.xml");
   }
   
+  @Override
   public long getProtocolVersion(String protocol, 
                                  long clientVersion) throws IOException {
     if (protocol.equals(ClientProtocol.class.getName())) {
@@ -661,20 +660,14 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /////////////////////////////////////////////////////
   // NamenodeProtocol
   /////////////////////////////////////////////////////
-  /**
-   * return a list of blocks & their locations on <code>datanode</code> whose
-   * total size is <code>size</code>
-   * 
-   * @param datanode on which blocks are located
-   * @param size total size of blocks
-   */
+  @Override // NameNodeProtocol
   public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
   throws IOException {
     if(size <= 0) {
       throw new IllegalArgumentException(
         "Unexpected not positive size: "+size);
     }
-
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlocks(datanode, size); 
   }
   
@@ -1029,24 +1022,21 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     namesystem.refreshNodes(new Configuration());
   }
 
-  /**
-   * Returns the size of the current edit log.
-   */
+  @Override // NameNodeProtocol
   public long getEditLogSize() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getEditLogSize();
   }
 
-  /**
-   * Roll the edit log.
-   */
+  @Override // NameNodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.rollEditLog();
   }
 
-  /**
-   * Roll the image 
-   */
+  @Override // NameNodeProtocol
   public void rollFsImage() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     namesystem.rollFSImage();
   }
     
@@ -1165,8 +1155,9 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     }
   }
 
-  /** {@inheritDoc} */
+  @Override // NameNodeProtocol
   public ExportedBlockKeys getBlockKeys() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockKeys();
   }
 
@@ -1188,8 +1179,13 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
       namesystem.removeDatanode(nodeReg);            
     }
   }
-    
+  
+  @Override // VersionedProtocol
   public NamespaceInfo versionRequest() throws IOException {
+    // currently this method is only called by DN and SNN (although this method
+    // is also included in ClientProtocol), thus we can/should check the super
+    // user privilege.
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getNamespaceInfo();
   }
 

+ 1 - 2
src/hdfs/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java

@@ -32,8 +32,7 @@ import org.apache.hadoop.security.KerberosInfo;
  * It's used to get part of the name node state
  *****************************************************************************/
 @KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 public interface NamenodeProtocol extends VersionedProtocol {
   /**
    * 3: new method added: getAccessKeys()