Browse Source

HDFS-2922. HA: close out operation categories. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1242572 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 năm trước cách đây
mục cha
commit
a63e12c4c8

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -113,8 +113,6 @@ class RetryInvocationHandler implements RpcInvocationHandler {
             msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
             if (LOG.isDebugEnabled()) {
               LOG.debug(msg, e);
-            } else {
-              LOG.warn(msg);
             }
           } else {
             if(LOG.isDebugEnabled()) {

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

@@ -192,3 +192,5 @@ HDFS-2924. Standby checkpointing fails to authenticate in secure cluster. (todd)
 HDFS-2915. HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition. (Bikas Saha via jitendra)
 
 HDFS-2912. Namenode not shutting down when shared edits dir is inaccessible. (Bikas Saha via atm)
+
+HDFS-2922. HA: close out operation categories. (eli)

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

@@ -681,7 +681,7 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /*
-   * Requests the namenode to dump data strcutures into specified 
+   * Requests the namenode to dump data structures into specified 
    * file.
    */
   public void metaSave(String pathname) throws IOException {
@@ -699,7 +699,7 @@ public class DistributedFileSystem extends FileSystem {
    * we can consider figuring out exactly which block is corrupt.
    */
   // We do not see a need for user to report block checksum errors and do not  
-  // want to rely on user to report block corruptions.
+  // want to rely on user to report block corruption.
   @Deprecated
   public boolean reportChecksumFailure(Path f, 
     FSDataInputStream in, long inPos, 

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

@@ -697,7 +697,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
-    // TODO(HA): decide on OperationCategory for this
+    namesystem.checkOperation(OperationCategory.READ);
     DatanodeInfo results[] = namesystem.datanodeReport(type);
     if (results == null ) {
       throw new IOException("Cannot find datanode report");
@@ -707,32 +707,32 @@ class NameNodeRpcServer implements NamenodeProtocols {
     
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action) throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     return namesystem.setSafeMode(action);
   }
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) 
       throws AccessControlException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     return namesystem.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public void saveNamespace() throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     namesystem.saveNamespace();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     namesystem.getBlockManager().getDatanodeManager().refreshNodes(
         new HdfsConfiguration());
   }
 
   @Override // NamenodeProtocol
-  public long getTransactionID() {
-    // TODO:HA decide on OperationCategory for this
+  public long getTransactionID()  throws IOException {
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getEditLog().getSyncTxId();
   }
 
@@ -744,26 +744,26 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
   }
     
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    namesystem.checkOperation(OperationCategory.WRITE);
     namesystem.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
       throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.distributedUpgradeProgress(action);
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     namesystem.metaSave(filename);
   }
   @Override // ClientProtocol
@@ -784,12 +784,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
   /**
    * Tell all datanodes to use a new, non-persistent bandwidth value for
    * dfs.datanode.balance.bandwidthPerSec.
-   * @param bandwidth Blanacer bandwidth in bytes per second for all datanodes.
+   * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
    * @throws IOException
    */
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    // TODO:HA decide on OperationCategory for this
+    // NB: not checking OperationCategory so this works on a standby
     namesystem.getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
   }