Browse Source

Revert HDFS-2922 via svn merge -c -1242572

The patch broke a lot of unit tests in the nightly build. Will recommit after it is fixed.



git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1242874 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 years ago
parent
commit
a626fa04f9

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

@@ -113,6 +113,8 @@ 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()) {

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

@@ -193,6 +193,4 @@ HDFS-2915. HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race conditio
 
 HDFS-2912. Namenode not shutting down when shared edits dir is inaccessible. (Bikas Saha via atm)
 
-HDFS-2922. HA: close out operation categories. (eli)
-
 HDFS-2917. HA: haadmin should not work if run by regular user (eli)

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

@@ -646,7 +646,7 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /*
-   * Requests the namenode to dump data structures into specified 
+   * Requests the namenode to dump data strcutures into specified 
    * file.
    */
   public void metaSave(String pathname) throws IOException {
@@ -664,7 +664,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 corruption.
+  // want to rely on user to report block corruptions.
   @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

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