Browse Source

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-2@1419951 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 12 năm trước cách đây
mục cha
commit
42f6c0fd63

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

@@ -426,6 +426,9 @@ Release 2.0.3-alpha - Unreleased
     HDFS-4294. Backwards compatibility is not maintained for TestVolumeId.
     (Ivan A. Veselovsky and Robert Parker via atm)
 
+    HDFS-2264. NamenodeProtocol has the wrong value for clientPrincipal in
+    KerberosInfo annotation. (atm)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -326,11 +326,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
         "Unexpected not positive size: "+size);
     }
     namesystem.checkOperation(OperationCategory.READ);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlocks(datanode, size); 
   }
 
   @Override // NamenodeProtocol
   public ExportedBlockKeys getBlockKeys() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlockKeys();
   }
 
@@ -339,6 +341,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
                           int errorCode, 
                           String msg) throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
     LOG.info("Error report from " + registration + ": " + msg);
     if (errorCode == FATAL) {
@@ -349,6 +352,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public NamenodeRegistration register(NamenodeRegistration registration)
   throws IOException {
+    namesystem.checkSuperuserPrivilege();
     verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();
     namesystem.registerBackupNode(registration, myRegistration);
@@ -358,6 +362,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
   throws IOException {
+    namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
     if(!nn.isRole(NamenodeRole.NAMENODE))
       throw new IOException("Only an ACTIVE node can invoke startCheckpoint.");
@@ -367,6 +372,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
+    namesystem.checkSuperuserPrivilege();
     namesystem.endCheckpoint(registration, sig);
   }
 
@@ -743,17 +749,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public long getTransactionID() throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
   }
   
   @Override // NamenodeProtocol
   public long getMostRecentCheckpointTxId() throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getMostRecentCheckpointTxId();
   }
   
   @Override // NamenodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.rollEditLog();
   }
   
@@ -761,6 +770,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   throws IOException {
     namesystem.checkOperation(OperationCategory.READ);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
   }
     
@@ -937,6 +947,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     
   @Override // DatanodeProtocol, NamenodeProtocol
   public NamespaceInfo versionRequest() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getNamespaceInfo();
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/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)
 @InterfaceAudience.Private
 public interface NamenodeProtocol {
   /**