浏览代码

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/trunk@1419949 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 12 年之前
父节点
当前提交
47ae6831e9

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

@@ -600,6 +600,9 @@ Release 2.0.3-alpha - Unreleased
     HDFS-4294. Backwards compatibility is not maintained for TestVolumeId.
     HDFS-4294. Backwards compatibility is not maintained for TestVolumeId.
     (Ivan A. Veselovsky and Robert Parker via atm)
     (Ivan A. Veselovsky and Robert Parker via atm)
 
 
+    HDFS-2264. NamenodeProtocol has the wrong value for clientPrincipal in
+    KerberosInfo annotation. (atm)
+
   BREAKDOWN OF HDFS-3077 SUBTASKS
   BREAKDOWN OF HDFS-3077 SUBTASKS
 
 
     HDFS-3077. Quorum-based protocol for reading and writing edit logs.
     HDFS-3077. Quorum-based protocol for reading and writing edit logs.

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

@@ -338,11 +338,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
         "Unexpected not positive size: "+size);
         "Unexpected not positive size: "+size);
     }
     }
     namesystem.checkOperation(OperationCategory.READ);
     namesystem.checkOperation(OperationCategory.READ);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlocks(datanode, size); 
     return namesystem.getBlockManager().getBlocks(datanode, size); 
   }
   }
 
 
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public ExportedBlockKeys getBlockKeys() throws IOException {
   public ExportedBlockKeys getBlockKeys() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getBlockManager().getBlockKeys();
     return namesystem.getBlockManager().getBlockKeys();
   }
   }
 
 
@@ -351,6 +353,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
                           int errorCode, 
                           int errorCode, 
                           String msg) throws IOException {
                           String msg) throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
     verifyRequest(registration);
     LOG.info("Error report from " + registration + ": " + msg);
     LOG.info("Error report from " + registration + ": " + msg);
     if (errorCode == FATAL) {
     if (errorCode == FATAL) {
@@ -361,6 +364,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public NamenodeRegistration register(NamenodeRegistration registration)
   public NamenodeRegistration register(NamenodeRegistration registration)
   throws IOException {
   throws IOException {
+    namesystem.checkSuperuserPrivilege();
     verifyLayoutVersion(registration.getVersion());
     verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();
     NamenodeRegistration myRegistration = nn.setRegistration();
     namesystem.registerBackupNode(registration, myRegistration);
     namesystem.registerBackupNode(registration, myRegistration);
@@ -370,6 +374,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
   public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
   throws IOException {
   throws IOException {
+    namesystem.checkSuperuserPrivilege();
     verifyRequest(registration);
     verifyRequest(registration);
     if(!nn.isRole(NamenodeRole.NAMENODE))
     if(!nn.isRole(NamenodeRole.NAMENODE))
       throw new IOException("Only an ACTIVE node can invoke startCheckpoint.");
       throw new IOException("Only an ACTIVE node can invoke startCheckpoint.");
@@ -379,6 +384,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public void endCheckpoint(NamenodeRegistration registration,
   public void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
                             CheckpointSignature sig) throws IOException {
+    namesystem.checkSuperuserPrivilege();
     namesystem.endCheckpoint(registration, sig);
     namesystem.endCheckpoint(registration, sig);
   }
   }
 
 
@@ -755,17 +761,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public long getTransactionID() throws IOException {
   public long getTransactionID() throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
     return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
   }
   }
   
   
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public long getMostRecentCheckpointTxId() throws IOException {
   public long getMostRecentCheckpointTxId() throws IOException {
     namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.checkOperation(OperationCategory.UNCHECKED);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getFSImage().getMostRecentCheckpointTxId();
     return namesystem.getFSImage().getMostRecentCheckpointTxId();
   }
   }
   
   
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
   public CheckpointSignature rollEditLog() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.rollEditLog();
     return namesystem.rollEditLog();
   }
   }
   
   
@@ -773,6 +782,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   throws IOException {
   throws IOException {
     namesystem.checkOperation(OperationCategory.READ);
     namesystem.checkOperation(OperationCategory.READ);
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
   }
   }
     
     
@@ -949,6 +959,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     
     
   @Override // DatanodeProtocol, NamenodeProtocol
   @Override // DatanodeProtocol, NamenodeProtocol
   public NamespaceInfo versionRequest() throws IOException {
   public NamespaceInfo versionRequest() throws IOException {
+    namesystem.checkSuperuserPrivilege();
     return namesystem.getNamespaceInfo();
     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
  * It's used to get part of the name node state
  *****************************************************************************/
  *****************************************************************************/
 @KerberosInfo(
 @KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public interface NamenodeProtocol {
 public interface NamenodeProtocol {
   /**
   /**