You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by su...@apache.org on 2011/08/11 02:44:05 UTC

svn commit: r1156418 - in /hadoop/common/branches/HDFS-1623/hdfs: ./ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/java/org/apache/hadoop/hdfs/server/namenode/ha/ src/java/org/apache/hadoop/hdfs/server/protocol/

Author: suresh
Date: Thu Aug 11 00:44:05 2011
New Revision: 1156418

URL: http://svn.apache.org/viewvc?rev=1156418&view=rev
Log:
HDFS-1974. Introduce active and standy states to the namenode. Contributed by Suresh Srinivas.


Added:
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java
Modified:
    hadoop/common/branches/HDFS-1623/hdfs/CHANGES.HDFS-1623.txt
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/DFSUtil.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java
    hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java

Modified: hadoop/common/branches/HDFS-1623/hdfs/CHANGES.HDFS-1623.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/CHANGES.HDFS-1623.txt?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/CHANGES.HDFS-1623.txt (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/CHANGES.HDFS-1623.txt Thu Aug 11 00:44:05 2011
@@ -5,3 +5,5 @@ branch is merged.
 ------------------------------
 
 HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
+
+HDFS-1974. Introduce active and standy states to the namenode. (suresh)

Modified: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/DFSUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/DFSUtil.java?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/DFSUtil.java (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/DFSUtil.java Thu Aug 11 00:44:05 2011
@@ -696,4 +696,14 @@ public class DFSUtil {
         ClientDatanodeProtocol.versionID, addr, ticket, confWithNoIpcIdle,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
   }
+
+  /**
+   * Returns true if HA for namenode is configured.
+   * @param conf Configuration
+   * @return true if HA is configured in the configuration; else false.
+   */
+  public static boolean isHAEnabled(Configuration conf) {
+    // TODO:HA configuration changes pending
+    return false;
+  }
 }

Modified: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java Thu Aug 11 00:44:05 2011
@@ -189,34 +189,6 @@ public class BackupNode extends NameNode
   }
 
   /////////////////////////////////////////////////////
-  // NamenodeProtocol implementation for backup node.
-  /////////////////////////////////////////////////////
-  @Override // NamenodeProtocol
-  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
-  throws IOException {
-    throw new UnsupportedActionException("getBlocks");
-  }
-
-  // Only active name-node can register other nodes.
-  @Override // NamenodeProtocol
-  public NamenodeRegistration register(NamenodeRegistration registration
-  ) throws IOException {
-    throw new UnsupportedActionException("register");
-  }
-
-  @Override // NamenodeProtocol
-  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
-  throws IOException {
-    throw new UnsupportedActionException("startCheckpoint");
-  }
-
-  @Override // NamenodeProtocol
-  public void endCheckpoint(NamenodeRegistration registration,
-                            CheckpointSignature sig) throws IOException {
-    throw new UnsupportedActionException("endCheckpoint");
-  }  
-
-  /////////////////////////////////////////////////////
   // BackupNodeProtocol implementation for backup node.
   /////////////////////////////////////////////////////
 
@@ -224,6 +196,7 @@ public class BackupNode extends NameNode
   public void journal(NamenodeRegistration nnReg,
       long firstTxId, int numTxns,
       byte[] records) throws IOException {
+    checkOperation(OperationCategory.JOURNAL);
     verifyRequest(nnReg);
     if(!nnRpcAddress.equals(nnReg.getAddress()))
       throw new IOException("Journal request from unexpected name-node: "
@@ -234,6 +207,7 @@ public class BackupNode extends NameNode
   @Override
   public void startLogSegment(NamenodeRegistration registration, long txid)
       throws IOException {
+    checkOperation(OperationCategory.JOURNAL);
     verifyRequest(registration);
   
     getBNImage().namenodeStartedLogSegment(txid);
@@ -369,4 +343,14 @@ public class BackupNode extends NameNode
   String getClusterId() {
     return clusterId;
   }
+  
+  @Override // NameNode
+  protected void checkOperation(OperationCategory op)
+      throws UnsupportedActionException {
+    if (OperationCategory.JOURNAL != op) {
+      String msg = "Operation category " + op
+          + " is not supported at the BackupNode";
+      throw new UnsupportedActionException(msg);
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Thu Aug 11 00:44:05 2011
@@ -31,6 +31,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HealthCheckFailedException;
+import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -67,11 +69,15 @@ import org.apache.hadoop.hdfs.server.com
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
+import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -146,6 +152,20 @@ public class NameNode implements Namenod
   }
   
   /**
+   * Categories of operations supported by the namenode.
+   */
+  public static enum OperationCategory {
+    /** Read operation that does not change the namespace state */
+    READ,
+    /** Write operation that changes the namespace state */
+    WRITE,
+    /** Operations related to checkpointing */
+    CHECKPOINT,
+    /** Operations related to {@link JournalProtocol} */
+    JOURNAL
+  }
+  
+  /**
    * HDFS federation configuration can have two types of parameters:
    * <ol>
    * <li>Parameter that is common for all the name services in the cluster.</li>
@@ -204,9 +224,15 @@ public class NameNode implements Namenod
 
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
   public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
+  public static final HAState ACTIVE_STATE = new ActiveState();
+  public static final HAState STANDBY_STATE = new StandbyState();
   
   protected FSNamesystem namesystem; 
   protected NamenodeRole role;
+  private HAState state;
+  private final boolean haEnabled;
+
+  
   /** RPC server. Package-protected for use in tests. */
   Server server;
   /** RPC server for HDFS Services communication.
@@ -402,6 +428,7 @@ public class NameNode implements Namenod
    * @param conf the configuration
    */
   protected void initialize(Configuration conf) throws IOException {
+    initializeGenericKeys(conf);
     InetSocketAddress socAddr = getRpcServerAddress(conf);
     UserGroupInformation.setConfiguration(conf);
     loginAsNameNodeUser(conf);
@@ -450,10 +477,6 @@ public class NameNode implements Namenod
     }
 
     activate(conf);
-    LOG.info(getRole() + " up at: " + rpcAddress);
-    if (serviceRPCAddress != null) {
-      LOG.info(getRole() + " service server is up at: " + serviceRPCAddress); 
-    }
   }
   
   /**
@@ -503,6 +526,10 @@ public class NameNode implements Namenod
         LOG.warn("ServicePlugin " + p + " could not be started", t);
       }
     }
+    LOG.info(getRole() + " up at: " + rpcAddress);
+    if (serviceRPCAddress != null) {
+      LOG.info(getRole() + " service server is up at: " + serviceRPCAddress); 
+    }
   }
 
   private void startTrashEmptier(Configuration conf) throws IOException {
@@ -556,8 +583,9 @@ public class NameNode implements Namenod
   protected NameNode(Configuration conf, NamenodeRole role) 
       throws IOException { 
     this.role = role;
+    this.haEnabled = DFSUtil.isHAEnabled(conf);
+    this.state = !haEnabled ? ACTIVE_STATE : STANDBY_STATE;
     try {
-      initializeGenericKeys(conf);
       initialize(conf);
     } catch (IOException e) {
       this.stop();
@@ -638,6 +666,7 @@ public class NameNode implements Namenod
   public void errorReport(NamenodeRegistration registration,
                           int errorCode, 
                           String msg) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     verifyRequest(registration);
     LOG.info("Error report from " + registration + ": " + msg);
     if(errorCode == FATAL)
@@ -665,27 +694,28 @@ public class NameNode implements Namenod
   @Override // NamenodeProtocol
   public void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
-    verifyRequest(registration);
-    if(!isRole(NamenodeRole.NAMENODE))
-      throw new IOException("Only an ACTIVE node can invoke endCheckpoint.");
+    checkOperation(OperationCategory.CHECKPOINT);
     namesystem.endCheckpoint(registration, sig);
   }
 
   @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     return namesystem.getDelegationToken(renewer);
   }
 
   @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
+    checkOperation(OperationCategory.WRITE);
     return namesystem.renewDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.cancelDelegationToken(token);
   }
   
@@ -694,6 +724,7 @@ public class NameNode implements Namenod
                                           long offset, 
                                           long length) 
       throws IOException {
+    checkOperation(OperationCategory.READ);
     metrics.incrGetBlockLocations();
     return namesystem.getBlockLocations(getClientMachine(), 
                                         src, offset, length);
@@ -712,6 +743,7 @@ public class NameNode implements Namenod
                      boolean createParent,
                      short replication,
                      long blockSize) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.create: file "
@@ -732,6 +764,7 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public LocatedBlock append(String src, String clientName) 
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.append: file "
@@ -744,6 +777,7 @@ public class NameNode implements Namenod
 
   @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     String clientMachine = getClientMachine();
     return namesystem.recoverLease(src, clientName, clientMachine);
   }
@@ -751,18 +785,21 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public boolean setReplication(String src, short replication) 
     throws IOException {  
+    checkOperation(OperationCategory.WRITE);
     return namesystem.setReplication(src, replication);
   }
     
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.setPermission(src, permissions);
   }
 
   @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.setOwner(src, username, groupname);
   }
 
@@ -772,6 +809,7 @@ public class NameNode implements Namenod
                                ExtendedBlock previous,
                                DatanodeInfo[] excludedNodes)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
           +src+" for "+clientName);
@@ -795,6 +833,7 @@ public class NameNode implements Namenod
       final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAdditionalDatanode: src=" + src
           + ", blk=" + blk
@@ -820,8 +859,10 @@ public class NameNode implements Namenod
   /**
    * The client needs to give up on the block.
    */
+  @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, String src, String holder)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
           +b+" of file "+src);
@@ -834,6 +875,7 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public boolean complete(String src, String clientName, ExtendedBlock last)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.complete: "
           + src + " for " + clientName);
@@ -847,8 +889,9 @@ public class NameNode implements Namenod
    * mark the block as corrupt.  In the future we might 
    * check the blocks are actually corrupt. 
    */
-  @Override
+  @Override // ClientProtocol, DatanodeProtocol
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
     for (int i = 0; i < blocks.length; i++) {
       ExtendedBlock blk = blocks[i].getBlock();
@@ -863,6 +906,7 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     return namesystem.updateBlockForPipeline(block, clientName);
   }
 
@@ -871,6 +915,7 @@ public class NameNode implements Namenod
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
   }
   
@@ -879,6 +924,7 @@ public class NameNode implements Namenod
       long newgenerationstamp, long newlength,
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.commitBlockSynchronization(block,
         newgenerationstamp, newlength, closeFile, deleteblock, newtargets);
   }
@@ -886,12 +932,14 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public long getPreferredBlockSize(String filename) 
       throws IOException {
+    checkOperation(OperationCategory.READ);
     return namesystem.getPreferredBlockSize(filename);
   }
     
   @Deprecated
   @Override // ClientProtocol
   public boolean rename(String src, String dst) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
     }
@@ -908,12 +956,14 @@ public class NameNode implements Namenod
   
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.concat(trg, src);
   }
   
   @Override // ClientProtocol
   public void rename(String src, String dst, Options.Rename... options)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
     }
@@ -928,11 +978,13 @@ public class NameNode implements Namenod
   @Deprecated
   @Override // ClientProtocol
   public boolean delete(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     return delete(src, true);
   }
 
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
           + ", recursive=" + recursive);
@@ -957,6 +1009,7 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
     }
@@ -971,13 +1024,14 @@ public class NameNode implements Namenod
 
   @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.renewLease(clientName);        
   }
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation)
-  throws IOException {
+      boolean needLocation) throws IOException {
+    checkOperation(OperationCategory.READ);
     DirectoryListing files = namesystem.getListing(
         src, startAfter, needLocation);
     if (files != null) {
@@ -989,12 +1043,14 @@ public class NameNode implements Namenod
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src)  throws IOException {
+    checkOperation(OperationCategory.READ);
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, true);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
+    checkOperation(OperationCategory.READ);
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, false);
   }
@@ -1007,6 +1063,7 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
+    checkOperation(OperationCategory.READ);
     DatanodeInfo results[] = namesystem.datanodeReport(type);
     if (results == null ) {
       throw new IOException("Cannot find datanode report");
@@ -1016,6 +1073,7 @@ public class NameNode implements Namenod
     
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action) throws IOException {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.setSafeMode(action);
   }
 
@@ -1029,54 +1087,64 @@ public class NameNode implements Namenod
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) 
       throws AccessControlException {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public void saveNamespace() throws IOException {
+    // TODO:HA decide on OperationCategory for this
     namesystem.saveNamespace();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
+    // TODO:HA decide on OperationCategory for this
     namesystem.refreshNodes(new HdfsConfiguration());
   }
 
   @Override // NamenodeProtocol
   public long getTransactionID() {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.getTransactionID();
   }
 
   @Override // NamenodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.rollEditLog();
   }
   
-  @Override
+  @Override // NamenodeProtocol
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   throws IOException {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.getEditLogManifest(sinceTxId);
   }
     
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
+    // TODO:HA decide on OperationCategory for this
     namesystem.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
       throws IOException {
+    // TODO:HA decide on OperationCategory for this
     return namesystem.distributedUpgradeProgress(action);
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
+    // TODO:HA decide on OperationCategory for this
     namesystem.metaSave(filename);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
+    checkOperation(OperationCategory.READ);
     Collection<FSNamesystem.CorruptFileBlockInfo> fbs =
       namesystem.listCorruptFileBlocks(path, cookie);
     
@@ -1096,35 +1164,42 @@ public class NameNode implements Namenod
    * @param bandwidth Blanacer 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
     namesystem.setBalancerBandwidth(bandwidth);
   }
   
   @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
+    checkOperation(OperationCategory.READ);
     return namesystem.getContentSummary(path);
   }
 
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota) 
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
   }
   
   @Override // ClientProtocol
   public void fsync(String src, String clientName) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.fsync(src, clientName);
   }
 
   @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) 
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     namesystem.setTimes(src, mtime, atime);
   }
 
   @Override // ClientProtocol
   public void createSymlink(String target, String link, FsPermission dirPerms,
       boolean createParent) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     metrics.incrCreateSymlinkOps();
     /* We enforce the MAX_PATH_LENGTH limit even though a symlink target 
      * URI may refer to a non-HDFS file system. 
@@ -1144,6 +1219,7 @@ public class NameNode implements Namenod
 
   @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
+    checkOperation(OperationCategory.READ);
     metrics.incrGetLinkTargetOps();
     /* Resolves the first symlink in the given path, returning a
      * new path consisting of the target of the symlink and any 
@@ -1591,4 +1667,43 @@ public class NameNode implements Namenod
     }
     return clientMachine;
   }
+  
+  @Override // HAServiceProtocol
+  public synchronized void monitorHealth() throws HealthCheckFailedException {
+    if (!haEnabled) {
+      return; // no-op, if HA is not eanbled
+    }
+    // TODO:HA implement health check
+    return;
+  }
+  
+  @Override // HAServiceProtocol
+  public synchronized void transitionToActive() throws ServiceFailedException {
+    if (!haEnabled) {
+      throw new ServiceFailedException("HA for namenode is not enabled");
+    }
+    state.setState(this, ACTIVE_STATE);
+  }
+  
+  @Override // HAServiceProtocol
+  public synchronized void transitionToStandby() throws ServiceFailedException {
+    if (!haEnabled) {
+      throw new ServiceFailedException("HA for namenode is not enabled");
+    }
+    state.setState(this, STANDBY_STATE);
+  }
+  
+  /** Check if an operation of given category is allowed */
+  protected synchronized void checkOperation(final OperationCategory op)
+      throws UnsupportedActionException {
+    state.checkOperation(this, op);
+  }
+  
+  public synchronized HAState getState() {
+    return state;
+  }
+  
+  public synchronized void setState(final HAState s) {
+    state = s;
+  }
 }

Modified: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java Thu Aug 11 00:44:05 2011
@@ -32,8 +32,7 @@ public class UnsupportedActionException 
   /** for java.io.Serializable */
   private static final long serialVersionUID = 1L;
 
-  public UnsupportedActionException(String action) {
-    super("Action " + action + "() is not supported.");
+  public UnsupportedActionException(String msg) {
+    super(msg);
   }
-
 }

Added: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java?rev=1156418&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java (added)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java Thu Aug 11 00:44:05 2011
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+
+/**
+ * Active state of the namenode. In this state, namenode provides the namenode
+ * service and handles operations of type {@link OperationCategory#WRITE} and
+ * {@link OperationCategory#READ}.
+ */
+public class ActiveState extends HAState {
+  public ActiveState() {
+    super("active");
+  }
+
+  @Override
+  public void checkOperation(NameNode nn, OperationCategory op)
+      throws UnsupportedActionException {
+    return; // Other than journal all operations are allowed in active state
+  }
+  
+  @Override
+  public void setState(NameNode nn, HAState s) throws ServiceFailedException {
+    if (s == NameNode.STANDBY_STATE) {
+      setStateInternal(nn, s);
+      return;
+    }
+    super.setState(nn, s);
+  }
+
+  @Override
+  protected void enterState(NameNode nn) throws ServiceFailedException {
+    // TODO:HA
+  }
+
+  @Override
+  protected void exitState(NameNode nn) throws ServiceFailedException {
+    // TODO:HA
+  }
+}

Added: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java?rev=1156418&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java (added)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java Thu Aug 11 00:44:05 2011
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+
+/**
+ * Namenode base state to implement state machine pattern.
+ */
+@InterfaceAudience.Private
+abstract public class HAState {
+  protected final String name;
+
+  /**
+   * Constructor
+   * @param name Name of the state.
+   */
+  public HAState(String name) {
+    this.name = name;
+  }
+
+  /**
+   * Internal method to transition the state of a given namenode to a new state.
+   * @param nn Namenode
+   * @param s new state
+   * @throws ServiceFailedException on failure to transition to new state.
+   */
+  protected final void setStateInternal(final NameNode nn, final HAState s)
+      throws ServiceFailedException {
+    exitState(nn);
+    nn.setState(s);
+    s.enterState(nn);
+  }
+
+  /**
+   * Method to be overridden by subclasses to perform steps necessary for
+   * entering a state.
+   * @param nn Namenode
+   * @throws ServiceFailedException on failure to enter the state.
+   */
+  protected abstract void enterState(final NameNode nn)
+      throws ServiceFailedException;
+
+  /**
+   * Method to be overridden by subclasses to perform steps necessary for
+   * exiting a state.
+   * @param nn Namenode
+   * @throws ServiceFailedException on failure to enter the state.
+   */
+  protected abstract void exitState(final NameNode nn)
+      throws ServiceFailedException;
+
+  /**
+   * Move from the existing state to a new state
+   * @param nn Namenode
+   * @param s new state
+   * @throws ServiceFailedException on failure to transition to new state.
+   */
+  public void setState(NameNode nn, HAState s) throws ServiceFailedException {
+    if (this == s) { // Aleady in the new state
+      return;
+    }
+    throw new ServiceFailedException("Transtion from state " + this + " to "
+        + s + " is not allowed.");
+  }
+  
+  /**
+   * Check if an operation is supported in a given state.
+   * @param nn Namenode
+   * @param op Type of the operation.
+   * @throws UnsupportedActionException if a given type of operation is not
+   *           supported in this state.
+   */
+  public void checkOperation(final NameNode nn, final OperationCategory op)
+      throws UnsupportedActionException {
+    String msg = "Operation category " + op + " is not supported in state "
+        + nn.getState();
+    throw new UnsupportedActionException(msg);
+  }
+  
+  @Override
+  public String toString() {
+    return super.toString();
+  }
+}
\ No newline at end of file

Added: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java?rev=1156418&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java (added)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java Thu Aug 11 00:44:05 2011
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+/**
+ * Namenode standby state. In this state the namenode acts as warm standby and
+ * keeps the following updated:
+ * <ul>
+ * <li>Namespace by getting the edits.</li>
+ * <li>Block location information by receiving block reports and blocks
+ * received from the datanodes.</li>
+ * </ul>
+ * 
+ * It does not handle read/write/checkpoint operations.
+ */
+public class StandbyState extends HAState {
+  public StandbyState() {
+    super("standby");
+  }
+
+  @Override
+  public void setState(NameNode nn, HAState s) throws ServiceFailedException {
+    if (s == NameNode.ACTIVE_STATE) {
+      setStateInternal(nn, s);
+      return;
+    }
+    super.setState(nn, s);
+  }
+
+  @Override
+  protected void enterState(NameNode nn) throws ServiceFailedException {
+    // TODO:HA
+  }
+
+  @Override
+  protected void exitState(NameNode nn) throws ServiceFailedException {
+    // TODO:HA
+  }
+}
+

Modified: hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java?rev=1156418&r1=1156417&r2=1156418&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java (original)
+++ hadoop/common/branches/HDFS-1623/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java Thu Aug 11 00:44:05 2011
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
@@ -32,5 +33,6 @@ public interface NamenodeProtocols
           NamenodeProtocol,
           RefreshAuthorizationPolicyProtocol,
           RefreshUserMappingsProtocol,
-          GetUserMappingsProtocol {
+          GetUserMappingsProtocol,
+          HAServiceProtocol {
 }