Browse Source

HADOOP-2645. The Metrics initialization code does not throw
exceptions when servers are restarted by MiniDFSCluster.
(Sanjay Radia via dhruba)



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@615093 13f79535-47bb-0310-9956-ffa450edef68

Dhruba Borthakur 17 years ago
parent
commit
0b4231f5bc
25 changed files with 1251 additions and 212 deletions
  1. 4 0
      CHANGES.txt
  2. 2 2
      src/java/org/apache/hadoop/dfs/DataBlockScanner.java
  3. 49 108
      src/java/org/apache/hadoop/dfs/DataNode.java
  4. 42 1
      src/java/org/apache/hadoop/dfs/FSDataset.java
  5. 7 21
      src/java/org/apache/hadoop/dfs/FSDatasetInterface.java
  6. 2 2
      src/java/org/apache/hadoop/dfs/FSEditLog.java
  7. 89 7
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  8. 1 1
      src/java/org/apache/hadoop/dfs/JspHelper.java
  9. 11 4
      src/java/org/apache/hadoop/dfs/NameNode.java
  10. 16 24
      src/java/org/apache/hadoop/dfs/NameNodeMetrics.java
  11. 145 0
      src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeMetrics.java
  12. 319 0
      src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatistics.java
  13. 291 0
      src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatisticsMBean.java
  14. 61 0
      src/java/org/apache/hadoop/dfs/datanode/metrics/FSDatasetMBean.java
  15. 84 0
      src/java/org/apache/hadoop/dfs/namenode/metrics/FSNamesystemMBean.java
  16. 20 15
      src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeStatistics.java
  17. 9 10
      src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeStatisticsMBean.java
  18. 6 1
      src/java/org/apache/hadoop/ipc/Server.java
  19. 7 1
      src/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java
  20. 10 1
      src/java/org/apache/hadoop/ipc/metrics/RpcMgt.java
  21. 18 1
      src/java/org/apache/hadoop/metrics/util/MBeanUtil.java
  22. 6 6
      src/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingInt.java
  23. 46 1
      src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java
  24. 1 1
      src/test/org/apache/hadoop/dfs/TestFileLimit.java
  25. 5 5
      src/webapps/dfs/dfshealth.jsp

+ 4 - 0
CHANGES.txt

@@ -550,6 +550,10 @@ Trunk (unreleased changes)
     sending it down to the compressor so that each write call doesn't
     compress. (Chris Douglas via acmurthy) 
 
+    HADOOP-2645. The Metrics initialization code does not throw
+    exceptions when servers are restarted by MiniDFSCluster.
+    (Sanjay Radia via dhruba)
+
 Release 0.15.3 - 2008-01-18
 
   BUG FIXES

+ 2 - 2
src/java/org/apache/hadoop/dfs/DataBlockScanner.java

@@ -412,13 +412,13 @@ public class DataBlockScanner implements Runnable {
                  StringUtils.stringifyException(e));
         
         if (second) {
-          datanode.getMetrics().verificationFailures(1);
+          datanode.getMetrics().blockVerificationFailures.inc(); 
           handleScanFailure(block);
           return;
         } 
       } finally {
         IOUtils.closeStream(blockSender);
-        datanode.getMetrics().verifiedBlocks(1);
+        datanode.getMetrics().blocksVerified.inc();
         totalScans++;
         totalVerifications++;
       }

+ 49 - 108
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
@@ -36,6 +35,7 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.dfs.BlockCommand;
 import org.apache.hadoop.dfs.DatanodeProtocol;
 import org.apache.hadoop.dfs.FSDatasetInterface.MetaDataInputStream;
+import org.apache.hadoop.dfs.datanode.metrics.DataNodeMetrics;
 
 import java.io.*;
 import java.net.*;
@@ -44,11 +44,6 @@ import java.util.concurrent.Semaphore;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
 
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.jvm.JvmMetrics;
-
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
  * blocks for a DFS deployment.  A single deployment can
@@ -150,88 +145,9 @@ public class DataNode implements FSConstants, Runnable {
     return System.currentTimeMillis();
   }
 
-  static class DataNodeMetrics implements Updater {
-    private final MetricsRecord metricsRecord;
-    private int bytesWritten = 0;
-    private int bytesRead = 0;
-    private int blocksWritten = 0;
-    private int blocksRead = 0;
-    private int blocksReplicated = 0;
-    private int blocksRemoved = 0;
-    private int blocksVerified = 0;
-    private int blockVerificationFailures = 0;
-      
-    DataNodeMetrics(Configuration conf) {
-      String sessionId = conf.get("session.id"); 
-      // Initiate reporting of Java VM metrics
-      JvmMetrics.init("DataNode", sessionId);
-      // Create record for DataNode metrics
-      MetricsContext context = MetricsUtil.getContext("dfs");
-      metricsRecord = MetricsUtil.createRecord(context, "datanode");
-      metricsRecord.setTag("sessionId", sessionId);
-      context.registerUpdater(this);
-    }
-      
-    /**
-     * Since this object is a registered updater, this method will be called
-     * periodically, e.g. every 5 seconds.
-     */
-    public void doUpdates(MetricsContext unused) {
-      synchronized (this) {
-        metricsRecord.incrMetric("bytes_read", bytesRead);
-        metricsRecord.incrMetric("bytes_written", bytesWritten);
-        metricsRecord.incrMetric("blocks_read", blocksRead);
-        metricsRecord.incrMetric("blocks_written", blocksWritten);
-        metricsRecord.incrMetric("blocks_replicated", blocksReplicated);
-        metricsRecord.incrMetric("blocks_removed", blocksRemoved);
-        metricsRecord.incrMetric("blocks_verified", blocksVerified);
-        metricsRecord.incrMetric("block_verification_failures", 
-                                                  blockVerificationFailures);        
-              
-        bytesWritten = 0;
-        bytesRead = 0;
-        blocksWritten = 0;
-        blocksRead = 0;
-        blocksReplicated = 0;
-        blocksRemoved = 0;
-        blocksVerified = 0;
-        blockVerificationFailures = 0;
-      }
-      metricsRecord.update();
-    }
-
-    synchronized void readBytes(int nbytes) {
-      bytesRead += nbytes;
-    }
-      
-    synchronized void wroteBytes(int nbytes) {
-      bytesWritten += nbytes;
-    }
-      
-    synchronized void readBlocks(int nblocks) {
-      blocksRead += nblocks;
-    }
-      
-    synchronized void wroteBlocks(int nblocks) {
-      blocksWritten += nblocks;
-    }
-      
-    synchronized void replicatedBlocks(int nblocks) {
-      blocksReplicated += nblocks;
-    }
-      
-    synchronized void removedBlocks(int nblocks) {
-      blocksRemoved += nblocks;
-    }
-    
-    synchronized void verifiedBlocks(int nblocks) {
-      blocksVerified += nblocks;
-    }
-    
-    synchronized void verificationFailures(int failures) {
-      blockVerificationFailures += failures;
-    }    
-  }
+
+
+
     
   /**
    * Create the DataNode given a configuration and an array of dataDirs.
@@ -240,7 +156,6 @@ public class DataNode implements FSConstants, Runnable {
   DataNode(Configuration conf, 
            AbstractList<File> dataDirs) throws IOException {
       
-    myMetrics = new DataNodeMetrics(conf);
     datanodeObject = this;
 
     try {
@@ -301,7 +216,9 @@ public class DataNode implements FSConstants, Runnable {
         setNewStorageID(dnRegistration);
         dnRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION;
         dnRegistration.storageInfo.namespaceID = nsInfo.namespaceID;
- 
+        // it would have been better to pass storage as a parameter to
+        // constructor below - need to augment ReflectionUtils used below.
+        conf.set("StorageId", dnRegistration.getStorageID());
         try {
           //Equivalent of following (can't do because Simulated is in test dir)
           //  this.data = new SimulatedFSDataset(conf);
@@ -318,6 +235,7 @@ public class DataNode implements FSConstants, Runnable {
       // initialize data node internal structure
       this.data = new FSDataset(storage, conf);
     }
+
       
     // find free port
     ServerSocket ss = new ServerSocket(tmpPort, 0, socAddr.getAddress());
@@ -384,6 +302,7 @@ public class DataNode implements FSConstants, Runnable {
       networkLoc = getNetworkLoc(conf);
     // register datanode
     register();
+    myMetrics = new DataNodeMetrics(conf, dnRegistration.getStorageID());
   }
 
   private NamespaceInfo handshake() throws IOException {
@@ -578,6 +497,12 @@ public class DataNode implements FSConstants, Runnable {
       } catch (InterruptedException ie) {
       }
     }
+    if (data != null) {
+      data.shutdown();
+    }
+    if (myMetrics != null) {
+      myMetrics.shutdown();
+    }
   }
   
   
@@ -638,12 +563,13 @@ public class DataNode implements FSConstants, Runnable {
 
     while (shouldRun) {
       try {
-        long now = System.currentTimeMillis();
+        long startTime = now();
 
         //
         // Every so often, send heartbeat or block-report
         //
-        if (now - lastHeartbeat > heartBeatInterval) {
+        
+        if (startTime - lastHeartbeat > heartBeatInterval) {
           //
           // All heartbeat messages include following info:
           // -- Datanode name
@@ -657,8 +583,9 @@ public class DataNode implements FSConstants, Runnable {
                                                        data.getRemaining(),
                                                        xmitsInProgress,
                                                        xceiverCount.getValue());
+          myMetrics.heartbeats.inc(now() - startTime);
           //LOG.info("Just sent heartbeat, with name " + localName);
-          lastHeartbeat = now;
+          lastHeartbeat = startTime;
           if (!processCommand(cmd))
             continue;
         }
@@ -697,7 +624,7 @@ public class DataNode implements FSConstants, Runnable {
         }
 
         // send block report
-        if (now - lastBlockReport > blockReportInterval) {
+        if (startTime - lastBlockReport > blockReportInterval) {
           //
           // Send latest blockinfo report if timer has expired.
           // Get back a list of local block(s) that are obsolete
@@ -708,6 +635,7 @@ public class DataNode implements FSConstants, Runnable {
           DatanodeCommand cmd = namenode.blockReport(dnRegistration,
                   BlockListAsLongs.convertToArrayLongs(bReport));
           long brTime = now() - brStartTime;
+          myMetrics.blockReports.inc(brTime);
           LOG.info("BlockReport of " + bReport.length +
               " blocks got processed in " + brTime + " msecs");
           //
@@ -715,10 +643,10 @@ public class DataNode implements FSConstants, Runnable {
           // time before we start the periodic block reports.
           //
           if (resetBlockReportTime) {
-            lastBlockReport = now - new Random().nextInt((int)(blockReportInterval));
+            lastBlockReport = startTime - new Random().nextInt((int)(blockReportInterval));
             resetBlockReportTime = false;
           } else {
-            lastBlockReport = now;
+            lastBlockReport = startTime;
           }
           processCommand(cmd);
         }
@@ -784,7 +712,7 @@ public class DataNode implements FSConstants, Runnable {
         checkDiskError();
         throw e;
       }
-      myMetrics.removedBlocks(toDelete.length);
+      myMetrics.blocksRemoved.inc(toDelete.length);
       break;
     case DatanodeProtocol.DNA_SHUTDOWN:
       // shut down the data node
@@ -981,24 +909,37 @@ public class DataNode implements FSConstants, Runnable {
         if ( version != DATA_TRANFER_VERSION ) {
           throw new IOException( "Version Mismatch" );
         }
-
+        boolean local = s.getInetAddress().equals(s.getLocalAddress());
         byte op = in.readByte();
-
+        long startTime = now();
         switch ( op ) {
         case OP_READ_BLOCK:
           readBlock( in );
+          myMetrics.readBlockOp.inc(now() - startTime);
+          if (local)
+            myMetrics.readsFromLocalClient.inc();
+          else
+            myMetrics.readsFromRemoteClient.inc();
           break;
         case OP_WRITE_BLOCK:
           writeBlock( in );
+          myMetrics.writeBlockOp.inc(now() - startTime);
+          if (local)
+            myMetrics.writesFromLocalClient.inc();
+          else
+            myMetrics.writesFromRemoteClient.inc();
           break;
         case OP_READ_METADATA:
           readMetadata( in );
+          myMetrics.readMetadataOp.inc(now() - startTime);
           break;
         case OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
           replaceBlock(in);
+          myMetrics.replaceBlockOp.inc(now() - startTime);
           break;
         case OP_COPY_BLOCK: // for balancing purpose; send to a proxy source
           copyBlock(in);
+          myMetrics.copyBlockOp.inc(now() - startTime);
           break;
         default:
           throw new IOException("Unknown opcode " + op + " in data stream");
@@ -1056,12 +997,12 @@ public class DataNode implements FSConstants, Runnable {
           } catch (IOException ignored) {}
         }
         
-        myMetrics.readBytes((int) read);
-        myMetrics.readBlocks(1);
+        myMetrics.bytesRead.inc((int) read);
+        myMetrics.blocksRead.inc();
         LOG.info(dnRegistration + " Served block " + block + " to " + s.getInetAddress());
       } catch ( SocketException ignored ) {
         // Its ok for remote side to close the connection anytime.
-        myMetrics.readBlocks(1);
+        myMetrics.blocksRead.inc();
       } catch ( IOException ioe ) {
         /* What exactly should we do here?
          * Earlier version shutdown() datanode if there is disk error.
@@ -1301,8 +1242,8 @@ public class DataNode implements FSConstants, Runnable {
         // then send data
         long read = blockSender.sendBlock(targetOut, balancingThrottler);
 
-        myMetrics.readBytes((int) read);
-        myMetrics.readBlocks(1);
+        myMetrics.bytesRead.inc((int) read);
+        myMetrics.blocksRead.inc();
         
         // check the response from target
         receiveResponse(targetSock, 1);
@@ -1807,7 +1748,7 @@ public class DataNode implements FSConstants, Runnable {
               receiver.close();
               block.setNumBytes(receiver.offsetInBlock);
               data.finalizeBlock(block);
-              myMetrics.wroteBlocks(1);
+              myMetrics.blocksWritten.inc();
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               LOG.info("Received block " + block + 
                        " of size " + block.getNumBytes() + 
@@ -1891,7 +1832,7 @@ public class DataNode implements FSConstants, Runnable {
               receiver.close();
               block.setNumBytes(receiver.offsetInBlock);
               data.finalizeBlock(block);
-              myMetrics.wroteBlocks(1);
+              myMetrics.blocksWritten.inc();
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               LOG.info("Received block " + block + 
                        " of size " + block.getNumBytes() + 
@@ -2129,7 +2070,7 @@ public class DataNode implements FSConstants, Runnable {
           out.write(buf, 0, len);
           // Write checksum
           checksumOut.write(buf, len, checksumSize);
-          myMetrics.wroteBytes(len);
+          myMetrics.bytesWritten.inc(len);
         }
       } catch (IOException iex) {
         checkDiskError(iex);
@@ -2295,7 +2236,7 @@ public class DataNode implements FSConstants, Runnable {
           // Finalize the block. Does this fsync()?
           block.setNumBytes(offsetInBlock);
           data.finalizeBlock(block);
-          myMetrics.wroteBlocks(1);
+          myMetrics.blocksWritten.inc();
         }
 
       } catch (IOException ioe) {

+ 42 - 1
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -20,11 +20,17 @@ package org.apache.hadoop.dfs;
 import java.io.*;
 import java.util.*;
 
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
 
 /**************************************************
  * FSDataset manages a set of data blocks.  Each block
@@ -521,6 +527,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     volumes.getBlockMap(blockMap);
     blockWriteTimeout = Math.max(
          conf.getInt("dfs.datanode.block.write.timeout.sec", 3600), 1) * 1000;
+    registerMBean(storage.getStorageID());
   }
 
   /**
@@ -854,8 +861,42 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     return "FSDataset{dirpath='"+volumes+"'}";
   }
 
+  private ObjectName mbeanName;
+  private Random rand = new Random();
+  /**
+   * Register the FSDataset MBean
+   */
+  void registerMBean(final String storageId) {
+    // We wrap to bypass standard mbean naming convetion.
+    // This wraping can be removed in java 6 as it is more flexible in 
+    // package naming for mbeans and their impl.
+    StandardMBean bean;
+    String serverName;
+    if (storageId.equals("")) {// Temp fix for the uninitialized storage
+      serverName = "DataNode-UndefinedStorageId" + rand.nextInt();
+    } else {
+      serverName = "DataNode-" + storageId;
+    }
+    try {
+      bean = new StandardMBean(this,FSDatasetMBean.class);
+      mbeanName = MBeanUtil.registerMBean(serverName, "FSDatasetStatus", bean);
+    } catch (NotCompliantMBeanException e) {
+      e.printStackTrace();
+    }
+ 
+    DataNode.LOG.info("Registered FSDatasetStatusMBean");
+  }
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  public String getStorageInfo() {
+    return toString();
+  }
+  
   public long getBlockSize(Block b) {
     return blockMap.get(b).length();
   }
-
 }

+ 7 - 21
src/java/org/apache/hadoop/dfs/FSDatasetInterface.java

@@ -26,6 +26,7 @@ import java.io.OutputStream;
 
 
 
+import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 /**
@@ -35,7 +36,7 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
  * SimulatedFSDataset (which simulates data).
  *
  */
-public interface FSDatasetInterface {
+public interface FSDatasetInterface extends FSDatasetMBean {
   
   
   /**
@@ -79,27 +80,7 @@ public interface FSDatasetInterface {
    * @throws IOException
    */
   public boolean metaFileExists(Block b) throws IOException;
-    
-  /**
-   * Returns the total space (in bytes) used by dfs datanode
-   * @return  the total space used by dfs datanode
-   * @throws IOException
-   */  
-  public long getDfsUsed() throws IOException;
-    
-  /**
-   * Returns total capacity (in bytes) of storage (used and unused)
-   * @return  total capacity of storage (used and unused)
-   * @throws IOException
-   */
-  public long getCapacity() throws IOException;
 
-  /**
-   * Returns the amount of free storage space (in bytes)
-   * @return The amount of free storage space
-   * @throws IOException
-   */
-  public long getRemaining() throws IOException;
 
   /**
    * Returns the specified block's on-disk length (excluding metadata)
@@ -201,6 +182,11 @@ public interface FSDatasetInterface {
      * Stringifies the name of the storage
      */
   public String toString();
+  
+  /**
+   * Shutdown the FSDataset
+   */
+  public void shutdown();
 
   /**
    * Returns the current offset in the data stream.

+ 2 - 2
src/java/org/apache/hadoop/dfs/FSEditLog.java

@@ -671,7 +671,7 @@ class FSEditLog {
     long end = FSNamesystem.now();
     numTransactions++;
     totalTimeTransactions += (end-start);
-    metrics.incrNumTransactions(1, (int)(end-start));
+    metrics.transactions.inc((end-start));
   }
 
   //
@@ -742,7 +742,7 @@ class FSEditLog {
        this.notifyAll();
     }
 
-    metrics.incrSyncs(1, (int)elapsed);
+    metrics.syncs.inc(elapsed);
   }
 
   //

+ 89 - 7
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -21,10 +21,12 @@ import org.apache.commons.logging.*;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.dfs.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.dfs.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.mapred.StatusHttpServer;
+import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.fs.Path;
@@ -41,6 +43,9 @@ import java.util.*;
 import java.util.Map.Entry;
 import java.text.SimpleDateFormat;
 
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
 import javax.security.auth.login.LoginException;
 
 /***************************************************
@@ -55,7 +60,7 @@ import javax.security.auth.login.LoginException;
  * 4)  machine --> blocklist (inverted #2)
  * 5)  LRU cache of updated-heartbeat machines
  ***************************************************/
-class FSNamesystem implements FSConstants {
+class FSNamesystem implements FSConstants, FSNamesystemMBean {
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.FSNamesystem");
 
   private boolean isPermissionEnabled;
@@ -259,6 +264,8 @@ class FSNamesystem implements FSConstants {
     hbthread.start();
     lmthread.start();
     replthread.start();
+    
+    this.registerMBean(); // register the MBean for the FSNamesystemStutus
 
 
     this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
@@ -2743,14 +2750,15 @@ class FSNamesystem implements FSConstants {
   long[] getStats() throws IOException {
     checkSuperuserPrivilege();
     synchronized(heartbeats) {
-      return new long[]{totalCapacity(), totalDfsUsed(), totalRemaining()};
+      return new long[]
+            {getCapacityTotal(), getCapacityUsed(), getCapacityRemaining()};
     }
   }
 
   /**
    * Total raw bytes including non-dfs used space.
    */
-  public long totalCapacity() {
+  public long getCapacityTotal() {
     synchronized (heartbeats) {
       return totalCapacity;
     }
@@ -2759,7 +2767,7 @@ class FSNamesystem implements FSConstants {
   /**
    * Total used space by data nodes
    */
-  public long totalDfsUsed() {
+  public long getCapacityUsed() {
     synchronized(heartbeats){
       return totalUsed;
     }
@@ -2767,7 +2775,7 @@ class FSNamesystem implements FSConstants {
   /**
    * Total non-used raw bytes.
    */
-  public long totalRemaining() {
+  public long getCapacityRemaining() {
     synchronized (heartbeats) {
       return totalRemaining;
     }
@@ -3800,7 +3808,7 @@ class FSNamesystem implements FSConstants {
   /**
    * Get the total number of blocks in the system. 
    */
-  long getBlockTotal() {
+  public long getBlocksTotal() {
     return blocksMap.size();
   }
 
@@ -4001,7 +4009,7 @@ class FSNamesystem implements FSConstants {
    */
   void checkFsObjectLimit() throws IOException {
     if (maxFsObjects != 0 &&
-        maxFsObjects <= dir.totalInodes() + getBlockTotal()) {
+        maxFsObjects <= dir.totalInodes() + getBlocksTotal()) {
       throw new IOException("Exceeded the configured number of objects " +
                              maxFsObjects + " in the filesystem.");
     }
@@ -4022,4 +4030,78 @@ class FSNamesystem implements FSConstants {
     this.hardLimit = hardLimit; 
     this.lmthread.interrupt();
   }
+
+
+  public long getFilesTotal() {
+    return this.dir.totalInodes();
+  }
+
+  public String getFSState() {
+    return isInSafeMode() ? "safeMode" : "Operational";
+  }
+  
+  private ObjectName mbeanName;
+  /**
+   * Register the FSNamesystem MBean
+   */
+  void registerMBean() {
+    // We wrap to bypass standard mbean naming convetion.
+    // This wraping can be removed in java 6 as it is more flexible in 
+    // package naming for mbeans and their impl.
+    StandardMBean bean;
+    try {
+      bean = new StandardMBean(this,FSNamesystemMBean.class);
+      mbeanName = MBeanUtil.registerMBean("NameNode", "FSNamesystemStatus", bean);
+    } catch (NotCompliantMBeanException e) {
+      e.printStackTrace();
+    }
+
+    LOG.info("Registered FSNamesystemStatusMBean");
+  }
+  
+  /**
+   * shutdown FSNamesystem
+   */
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+  
+
+  /**
+   * Number of live data nodes
+   * @returns Number of live data nodes
+   */
+  public int numLiveDataNodes() {
+    int numLive = 0;
+    synchronized (datanodeMap) {   
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); 
+                                                               it.hasNext();) {
+        DatanodeDescriptor dn = it.next();
+        if (!isDatanodeDead(dn) ) {
+          numLive++;
+        }
+      }
+    }
+    return numLive;
+  }
+  
+
+  /**
+   * Number of dead data nodes
+   * @returns Number of dead data nodes
+   */
+  public int numDeadDataNodes() {
+    int numDead = 0;
+    synchronized (datanodeMap) {   
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); 
+                                                               it.hasNext();) {
+        DatanodeDescriptor dn = it.next();
+        if (isDatanodeDead(dn) ) {
+          numDead++;
+        }
+      }
+    }
+    return numDead;
+  }
 }

+ 1 - 1
src/java/org/apache/hadoop/dfs/JspHelper.java

@@ -183,7 +183,7 @@ public class JspHelper {
 
   public String getInodeLimitText() {
     long inodes = fsn.dir.totalInodes();
-    long blocks = fsn.getBlockTotal();
+    long blocks = fsn.getBlocksTotal();
     long maxobjects = fsn.getMaxObjects();
     long totalMemory = Runtime.getRuntime().totalMemory();   
     long maxMemory = Runtime.getRuntime().maxMemory();   

+ 11 - 4
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -106,6 +106,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public static NameNodeMetrics getNameNodeMetrics() {
     return myMetrics;
   }
+  
     
   /**
    * Initialize the server
@@ -199,6 +200,12 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     if(namesystem != null) namesystem.close();
     if(emptier != null) emptier.interrupt();
     if(server != null) server.stop();
+    if (myMetrics != null) {
+      myMetrics.shutdown();
+    }
+    if (namesystem != null) {
+      namesystem.shutdown();
+    }
   }
   
   /////////////////////////////////////////////////////
@@ -234,7 +241,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     if (result == null) {
       throw new IOException("Cannot open filename " + src);
     }
-    myMetrics.openFile();
+    myMetrics.numFilesOpened.inc();
     return result;
   }
 
@@ -273,7 +280,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     namesystem.startFile(src,
         new PermissionStatus(Server.getUserInfo().getUserName(), null, masked),
         clientName, clientMachine, overwrite, replication, blockSize);
-    myMetrics.createFile();
+    myMetrics.numFilesCreated.inc();
   }
 
   public boolean setReplication(String src, 
@@ -367,7 +374,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     }
     boolean ret = namesystem.renameTo(src, dst);
     if (ret) {
-      myMetrics.renameFile();
+      myMetrics.numFilesRenamed.inc();
     }
     return ret;
   }
@@ -425,7 +432,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public DFSFileInfo[] getListing(String src) throws IOException {
     DFSFileInfo[] files = namesystem.getListing(src);
     if (files != null) {
-      myMetrics.listFile(files.length);
+      myMetrics.numFilesListed.inc(files.length);
     }
     return files;
   }

+ 16 - 24
src/java/org/apache/hadoop/dfs/NameNodeMetrics.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.dfs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.namenode.metrics.NameNodeMgt;
+import org.apache.hadoop.dfs.namenode.metrics.NameNodeStatistics;
 import org.apache.hadoop.metrics.*;
 import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.metrics.util.MetricsIntValue;
@@ -43,6 +43,8 @@ public class NameNodeMetrics implements Updater {
     private static Log log = LogFactory.getLog(NameNodeMetrics.class);
     private final MetricsRecord metricsRecord;
     
+    private NameNodeStatistics namenodeStats;
+    
     public MetricsTimeVaryingInt numFilesCreated = new MetricsTimeVaryingInt("FilesCreated");
     public MetricsTimeVaryingInt numFilesOpened = new MetricsTimeVaryingInt("FilesOpened");
     public MetricsTimeVaryingInt numFilesRenamed = new MetricsTimeVaryingInt("FilesRenamed");
@@ -63,8 +65,8 @@ public class NameNodeMetrics implements Updater {
       JvmMetrics.init("NameNode", sessionId);
 
       
-      // Now the Mbean for the name node
-      new NameNodeMgt(sessionId, this, nameNode);
+      // Now the Mbean for the name node - this alos registers the MBean
+      namenodeStats = new NameNodeStatistics(this);
       
       // Create a record for NameNode metrics
       MetricsContext metricsContext = MetricsUtil.getContext("dfs");
@@ -74,6 +76,13 @@ public class NameNodeMetrics implements Updater {
       log.info("Initializing NameNodeMeterics using context object:" +
                 metricsContext.getClass().getName());
     }
+    
+
+    
+    public void shutdown() {
+      if (namenodeStats != null) 
+        namenodeStats.shutdown();
+    }
       
     /**
      * Since this object is a registered updater, this method will be called
@@ -95,28 +104,11 @@ public class NameNodeMetrics implements Updater {
       }
       metricsRecord.update();
     }
-      
-    void createFile() {
-      numFilesCreated.inc();
-    }
-      
-    void openFile() {
-      numFilesOpened.inc();
-    }
-      
-    void renameFile() {
-      numFilesRenamed.inc();
-    }
-      
-    void listFile(int nfiles) {
-      numFilesListed.inc(nfiles);
-    }
 
-    void incrNumTransactions(int count, int time) {
-      transactions.inc(count, time);
+    public void resetAllMinMax() {
+      transactions.resetMinMax();
+      syncs.resetMinMax();
+      blockReport.resetMinMax();
     }
 
-    void incrSyncs(int count, int time) {
-      syncs.inc(count, time);
-    }
 }

+ 145 - 0
src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeMetrics.java

@@ -0,0 +1,145 @@
+/**
+ * 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.dfs.datanode.metrics;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.Updater;
+import org.apache.hadoop.metrics.jvm.JvmMetrics;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingInt;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
+
+
+/**
+ * 
+ * This class is for maintaining  the various DataNode statistics
+ * and publishing them through the metrics interfaces.
+ * This also registers the JMX MBean for RPC.
+ * <p>
+ * This class has a number of metrics variables that are publicly accessible;
+ * these variables (objects) have methods to update their values;
+ *  for example:
+ *  <p> {@link #blocksRead}.inc()
+ *
+ */
+public class DataNodeMetrics implements Updater {
+  private final MetricsRecord metricsRecord;
+  private DataNodeStatistics datanodeStats;
+  
+  
+  public MetricsTimeVaryingInt bytesWritten = 
+                      new MetricsTimeVaryingInt("bytes_written");
+  public MetricsTimeVaryingInt bytesRead = 
+                      new MetricsTimeVaryingInt("bytes_read");
+  public MetricsTimeVaryingInt blocksWritten = 
+                      new MetricsTimeVaryingInt("blocks_written");
+  public MetricsTimeVaryingInt blocksRead = 
+                      new MetricsTimeVaryingInt("blocks_read");
+  public MetricsTimeVaryingInt blocksReplicated =
+                      new MetricsTimeVaryingInt("blocks_replicated");
+  public MetricsTimeVaryingInt blocksRemoved =
+                       new MetricsTimeVaryingInt("blocks_removed");
+  public MetricsTimeVaryingInt blocksVerified = 
+                        new MetricsTimeVaryingInt("blocks_verified");
+  public MetricsTimeVaryingInt blockVerificationFailures =
+                       new MetricsTimeVaryingInt("block_verification_failures");
+  
+  public MetricsTimeVaryingInt readsFromLocalClient = 
+                new MetricsTimeVaryingInt("reads_from_local_client");
+  public MetricsTimeVaryingInt readsFromRemoteClient = 
+                new MetricsTimeVaryingInt("reads_from_remote_client");
+  public MetricsTimeVaryingInt writesFromLocalClient = 
+              new MetricsTimeVaryingInt("writes_from_local_client");
+  public MetricsTimeVaryingInt writesFromRemoteClient = 
+              new MetricsTimeVaryingInt("writes_from_remote_client");
+  
+  public MetricsTimeVaryingRate readBlockOp = 
+                new MetricsTimeVaryingRate("readBlockOp");
+  public MetricsTimeVaryingRate writeBlockOp = 
+                new MetricsTimeVaryingRate("writeBlockOp");
+  public MetricsTimeVaryingRate readMetadataOp = 
+                new MetricsTimeVaryingRate("readMetadataOp");
+  public MetricsTimeVaryingRate copyBlockOp = 
+                new MetricsTimeVaryingRate("copyBlockOp");
+  public MetricsTimeVaryingRate replaceBlockOp = 
+                new MetricsTimeVaryingRate("replaceBlockOp");
+  public MetricsTimeVaryingRate heartbeats = 
+                    new MetricsTimeVaryingRate("heartBeats");
+  public MetricsTimeVaryingRate blockReports = 
+                    new MetricsTimeVaryingRate("blockReports");
+
+    
+  public DataNodeMetrics(Configuration conf, String storageId) {
+    String sessionId = conf.get("session.id"); 
+    // Initiate reporting of Java VM metrics
+    JvmMetrics.init("DataNode", sessionId);
+    
+
+    // Now the MBean for the data node
+    datanodeStats = new DataNodeStatistics(this, storageId);
+    
+    // Create record for DataNode metrics
+    MetricsContext context = MetricsUtil.getContext("dfs");
+    metricsRecord = MetricsUtil.createRecord(context, "datanode");
+    metricsRecord.setTag("sessionId", sessionId);
+    context.registerUpdater(this);
+  }
+  
+  public void shutdown() {
+    if (datanodeStats != null) 
+      datanodeStats.shutdown();
+  }
+    
+  /**
+   * Since this object is a registered updater, this method will be called
+   * periodically, e.g. every 5 seconds.
+   */
+  public void doUpdates(MetricsContext unused) {
+    synchronized (this) {
+            
+      bytesWritten.pushMetric(metricsRecord);
+      bytesRead.pushMetric(metricsRecord);
+      blocksWritten.pushMetric(metricsRecord);
+      blocksRead.pushMetric(metricsRecord);
+      blocksReplicated.pushMetric(metricsRecord);
+      blocksRemoved.pushMetric(metricsRecord);
+      blocksVerified.pushMetric(metricsRecord);
+      blockVerificationFailures.pushMetric(metricsRecord);
+      readsFromLocalClient.pushMetric(metricsRecord);
+      writesFromLocalClient.pushMetric(metricsRecord);
+      readsFromRemoteClient.pushMetric(metricsRecord);
+      writesFromRemoteClient.pushMetric(metricsRecord);
+      
+      readBlockOp.pushMetric(metricsRecord);
+      writeBlockOp.pushMetric(metricsRecord);
+      readMetadataOp.pushMetric(metricsRecord);
+      copyBlockOp.pushMetric(metricsRecord);
+      replaceBlockOp.pushMetric(metricsRecord);
+    }
+    metricsRecord.update();
+  }
+  public void resetAllMinMax() {
+    readBlockOp.resetMinMax();
+    writeBlockOp.resetMinMax();
+    readMetadataOp.resetMinMax();
+    copyBlockOp.resetMinMax();
+    replaceBlockOp.resetMinMax();
+  }
+}

+ 319 - 0
src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatistics.java

@@ -0,0 +1,319 @@
+package org.apache.hadoop.dfs.datanode.metrics;
+
+import java.util.Random;
+
+import javax.management.ObjectName;
+
+import org.apache.hadoop.metrics.util.MBeanUtil;
+
+public class DataNodeStatistics implements DataNodeStatisticsMBean {
+  private DataNodeMetrics myMetrics;
+  private ObjectName mbeanName;
+  private Random rand = new Random(); 
+  
+  
+
+  /**
+   * This constructs and registers the DataNodeStatisticsMBean
+   * @param dataNodeMetrics - the metrics from which the mbean gets its info
+   */
+  DataNodeStatistics(DataNodeMetrics dataNodeMetrics, String storageId) {
+    myMetrics = dataNodeMetrics;
+    String serverName;
+    if (storageId.equals("")) {// Temp fix for the uninitialized storage
+      serverName = "DataNode-UndefinedStorageId" + rand.nextInt();
+    } else {
+      serverName = "DataNode-" + storageId;
+    }
+    mbeanName = MBeanUtil.registerMBean(serverName, "DataNodeStatistics", this);
+  }
+  
+  /**
+   * Shuts down the statistics
+   *   - unregisters the mbean
+   */
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public void resetAllMinMax() {
+    myMetrics.resetAllMinMax();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlocksRead() {
+    return myMetrics.blocksRead.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlocksRemoved() {
+    return myMetrics.blocksRemoved.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlocksReplicated() {
+    return myMetrics.blocksReplicated.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlocksWritten() {
+    return myMetrics.blocksWritten.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBytesRead() {
+    return myMetrics.bytesRead.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlockVerificationFailures() {
+    return myMetrics.blockVerificationFailures.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlocksVerified() {
+    return myMetrics.blocksVerified.getPreviousIntervalValue();
+  }
+  
+  /**
+   * @inheritDoc
+   */
+  public int getReadsFromLocalClient() {
+    return myMetrics.readsFromLocalClient.getPreviousIntervalValue();
+  }
+  
+  /**
+   * @inheritDoc
+   */
+  public int getReadsFromRemoteClient() {
+    return myMetrics.readsFromRemoteClient.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getWritesFromLocalClient() {
+    return myMetrics.writesFromLocalClient.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getWritesFromRemoteClient() {
+    return myMetrics.writesFromRemoteClient.getPreviousIntervalValue();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadBlockOpAverageTime() {
+    return myMetrics.readBlockOp.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadBlockOpMaxTime() {
+    return myMetrics.readBlockOp.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadBlockOpMinTime() {
+    return myMetrics.readBlockOp.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getReadBlockOpNum() {
+    return myMetrics.readBlockOp.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadMetadataOpAverageTime() {
+    return myMetrics.readMetadataOp.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadMetadataOpMaxTime() {
+    return myMetrics.readMetadataOp.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReadMetadataOpMinTime() {
+    return myMetrics.readMetadataOp.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getReadMetadataOpNum() {
+    return myMetrics.readMetadataOp.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReplaceBlockOpAverageTime() {
+    return myMetrics.replaceBlockOp.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReplaceBlockOpMaxTime() {
+    return myMetrics.replaceBlockOp.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getReplaceBlockOpMinTime() {
+    return myMetrics.replaceBlockOp.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getReplaceBlockOpNum() {
+    return myMetrics.replaceBlockOp.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getWriteBlockOpAverageTime() {
+    return myMetrics.writeBlockOp.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getWriteBlockOpMaxTime() {
+    return myMetrics.writeBlockOp.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getWriteBlockOpMinTime() {
+    return myMetrics.writeBlockOp.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getWriteBlockOpNum() {
+    return myMetrics.writeBlockOp.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getCopyBlockOpAverageTime() {
+    return myMetrics.copyBlockOp.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getCopyBlockOpMaxTime() {
+    return myMetrics.copyBlockOp.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getCopyBlockOpMinTime() {
+    return myMetrics.copyBlockOp.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getCopyBlockOpNum() {
+    return myMetrics.copyBlockOp.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getBlockReportsAverageTime() {
+    return myMetrics.blockReports.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getBlockReportsMaxTime() {
+    return myMetrics.blockReports.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getBlockReportsMinTime() {
+    return myMetrics.blockReports.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getBlockReportsNum() {
+    return myMetrics.blockReports.getPreviousIntervalNumOps();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getHeartbeatsAverageTime() {
+    return myMetrics.heartbeats.getPreviousIntervalAverageTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getHeartbeatsMaxTime() {
+    return myMetrics.heartbeats.getMaxTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public long getHeartbeatsMinTime() {
+    return myMetrics.heartbeats.getMinTime();
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public int getHeartbeatsNum() {
+    return myMetrics.heartbeats.getPreviousIntervalNumOps();
+  }
+}

+ 291 - 0
src/java/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatisticsMBean.java

@@ -0,0 +1,291 @@
+/**
+ * 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.dfs.datanode.metrics;
+
+/**
+ * 
+ * This is the JMX  interface for the runtime statistics for the data node.
+ * Many of the statistics are sampled and averaged on an interval 
+ * which can be specified in the config file.
+ * <p>
+ * For the statistics that are sampled and averaged, one must specify 
+ * a metrics context that does periodic update calls. Most do.
+ * The default Null metrics context however does NOT. So if you aren't
+ * using any other metrics context then you can turn on the viewing and averaging
+ * of sampled metrics by  specifying the following two lines
+ *  in the hadoop-meterics.properties file:
+ *  <pre>
+ *        dfs.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+ *        dfs.period=10
+ *  </pre>
+ *<p>
+ * Note that the metrics are collected regardless of the context used.
+ * The context with the update thread is used to average the data periodically.
+ * <p>
+ * Name Node Status info is reported in another MBean
+ * @see org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean
+ *
+ */
+public interface DataNodeStatisticsMBean {
+  
+  /**
+   *   Number of bytes read in the last interval
+   * @return number of bytes read
+   */
+  int getBytesRead();
+  
+  /**
+   *   Number of blocks written in the last interval
+   * @return number of blocks written
+   */
+  int getBlocksWritten(); 
+  
+  /**
+   *   Number of blocks read in the last interval
+   * @return number of blocks read
+   */
+  int getBlocksRead(); 
+  
+  /**
+   *   Number of blocks replicated in the last interval
+   * @return number of blocks replicated
+   */
+  int getBlocksReplicated();
+  
+  /**
+   *   Number of blocks removed in the last interval
+   * @return number of blocks removed
+   */
+  int getBlocksRemoved();
+  
+  /**
+   *   Number of blocks verified in the last interval
+   * @return number of blocks verified
+   */
+  int getBlocksVerified();
+  
+  /**
+   *   Number of block verification failures in the last interval
+   * @return number of block verification failures
+   */
+  int getBlockVerificationFailures();
+  
+  /**
+   * Number of reads from local clients in the last interval
+   * @return number of reads from local clients
+   */
+  int getReadsFromLocalClient();
+  
+  
+  /**
+   * Number of reads from remote clients in the last interval
+   * @return number of reads from remote clients
+   */
+  int getReadsFromRemoteClient();
+  
+  
+  /**
+   * Number of writes from local clients in the last interval
+   * @return number of writes from local clients
+   */
+  int getWritesFromLocalClient();
+  
+  
+  /**
+   * Number of writes from remote clients in the last interval
+   * @return number of writes from remote clients
+   */
+  int getWritesFromRemoteClient();
+
+  /**
+   * Number of ReadBlock Operation in last interval
+   * @return number of operations
+   */
+  int getReadBlockOpNum();  
+
+  /**
+   * Average time for ReadBlock Operation in last interval
+   * @return time in msec
+   */
+  long getReadBlockOpAverageTime();
+  
+  /**
+   *   The Minimum ReadBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReadBlockOpMinTime();
+  
+  /**
+   *   The Maximum ReadBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReadBlockOpMaxTime();
+  
+  /**
+   * Number of WriteBlock Operation in last interval
+   * @return number of operations
+   */
+  int getWriteBlockOpNum();
+
+  /**
+   * Average time for WriteBlock Operation in last interval
+   * @return time in msec
+   */
+  long getWriteBlockOpAverageTime();
+  
+  /**
+   *   The Minimum WriteBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getWriteBlockOpMinTime();
+  
+  /**
+   *   The Maximum WriteBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getWriteBlockOpMaxTime(); 
+  
+  /**
+   * Number of ReadMetadata Operation in last interval
+   * @return number of operations
+   */
+  int getReadMetadataOpNum(); 
+
+  /**
+   * Average time for ReadMetadata Operation in last interval
+   * @return time in msec
+   */
+  long getReadMetadataOpAverageTime();
+  
+  /**
+   *   The Minimum ReadMetadata Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReadMetadataOpMinTime();
+  
+  /**
+   *   The Maximum ReadMetadata Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReadMetadataOpMaxTime();
+  
+  
+  /**
+   * Number of CopyBlock Operation in last interval
+   * @return number of operations
+   */
+  int getCopyBlockOpNum();
+
+  /**
+   * Average time for CopyBlock Operation in last interval
+   * @return time in msec
+   */
+  long getCopyBlockOpAverageTime();
+  
+  /**
+   *   The Minimum CopyBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getCopyBlockOpMinTime();
+  
+  /**
+   *   The Maximum CopyBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getCopyBlockOpMaxTime();
+
+  /**
+   * Number of ReplaceBlock Operation in last interval
+   * @return number of operations
+   */
+  int getReplaceBlockOpNum();
+  
+
+  /**
+   * Average time for ReplaceBlock Operation in last interval
+   * @return time in msec
+   */
+  long getReplaceBlockOpAverageTime();
+  
+  /**
+   *   The Minimum ReplaceBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReplaceBlockOpMinTime();
+  
+  /**
+   *   The Maximum ReplaceBlock Operation Time since reset was called
+   * @return time in msec
+   */
+  long getReplaceBlockOpMaxTime();
+  
+  /**
+   * Number of Block Reports sent in last interval
+   * @return number of operations
+   */
+  int getBlockReportsNum();
+
+  /**
+   * Average time for Block Reports Operation in last interval
+   * @return time in msec
+   */
+  long getBlockReportsAverageTime();
+  
+  /**
+   *   The Minimum Block Reports Operation Time since reset was called
+   * @return time in msec
+   */
+  long getBlockReportsMinTime();
+  
+  /**
+   *   The Maximum Block Reports Operation Time since reset was called
+   * @return time in msec
+   */
+  long getBlockReportsMaxTime();
+
+  /**
+   * Number of Heartbeat Operation in last interval
+   * @return number of operations
+   */
+  int getHeartbeatsNum();
+
+  /**
+   * Average time for Heartbeat Operation in last interval
+   * @return time in msec
+   */
+  long getHeartbeatsAverageTime();
+  
+  /**
+   *   The Minimum Heartbeat Operation Time since reset was called
+   * @return time in msec
+   */
+  long getHeartbeatsMinTime();
+  
+  /**
+   *   The Maximum Heartbeat Operation Time since reset was called
+   * @return time in msec
+   */
+  long getHeartbeatsMaxTime();
+  
+  
+  /**
+   * Reset all min max times
+   */
+  public void resetAllMinMax();
+}

+ 61 - 0
src/java/org/apache/hadoop/dfs/datanode/metrics/FSDatasetMBean.java

@@ -0,0 +1,61 @@
+/**
+ * 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.dfs.datanode.metrics;
+
+import java.io.IOException;
+
+/**
+ * 
+ * This Interface defines the methods to get the status of a the FSDataset of
+ * a data node.
+ * It is also used for publishing via JMX (hence we follow the JMX naming
+ * convention.) 
+ * <p>
+ * Data Node runtime statistic  info is report in another MBean
+ * @see org.apache.hadoop.dfs.datanode.metrics.DataNodeStatisticsMBean
+ *
+ */
+public interface FSDatasetMBean {
+  
+  /**
+   * Returns the total space (in bytes) used by dfs datanode
+   * @return  the total space used by dfs datanode
+   * @throws IOException
+   */  
+  public long getDfsUsed() throws IOException;
+    
+  /**
+   * Returns total capacity (in bytes) of storage (used and unused)
+   * @return  total capacity of storage (used and unused)
+   * @throws IOException
+   */
+  public long getCapacity() throws IOException;
+
+  /**
+   * Returns the amount of free storage space (in bytes)
+   * @return The amount of free storage space
+   * @throws IOException
+   */
+  public long getRemaining() throws IOException;
+  
+  /**
+   * Returns the storage id of the underlying storage
+   */
+  public String getStorageInfo();
+
+}

+ 84 - 0
src/java/org/apache/hadoop/dfs/namenode/metrics/FSNamesystemMBean.java

@@ -0,0 +1,84 @@
+/**
+ * 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.dfs.namenode.metrics;
+
+/**
+ * 
+ * This Interface defines the methods to get the status of a the FSNamesystem of
+ * a name node.
+ * It is also used for publishing via JMX (hence we follow the JMX naming
+ * convention.)
+ * 
+ * <p>
+ * Name Node runtime statistic  info is report in another MBean
+ * @see org.apache.hadoop.dfs.namenode.metrics.NameNodeStatisticsMBean
+ *
+ */
+public interface FSNamesystemMBean {
+
+  /**
+   * The state of the file system: Safemode or Operational
+   * @return the state
+   */
+  public String getFSState();
+  
+  
+  /**
+   * Number of allocated blocks in the system
+   * @return -  number of allocated blocks
+   */
+  public long getBlocksTotal();
+
+  /**
+   * Total storage capacity
+   * @return -  total capacity in bytes
+   */
+  public long getCapacityTotal();
+
+
+  /**
+   * Free (unused) storage capacity
+   * @return -  free capacity in bytes
+   */
+  public long getCapacityRemaining();
+ 
+  /**
+   * Used storage capacity
+   * @return -  used capacity in bytes
+   */
+  public long getCapacityUsed();
+ 
+
+  /**
+   * Total number of files and directories
+   * @return -  num of files and directories
+   */
+  public long getFilesTotal();
+ 
+  /**
+   * Number of Live data nodes
+   * @return number of live data nodes
+   */
+  public int numLiveDataNodes();
+  
+  /**
+   * Number of dead data nodes
+   * @return number of dead data nodes
+   */
+  public int numDeadDataNodes();
+}

+ 20 - 15
src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeMgt.java → src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeStatistics.java

@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.dfs.namenode.metrics;
 
-import org.apache.hadoop.dfs.NameNode;
+import javax.management.ObjectName;
+
 import org.apache.hadoop.dfs.NameNodeMetrics;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 
@@ -26,14 +27,26 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
  * This is the implementation of the Name Node JMX MBean
  *
  */
-public class NameNodeMgt implements NameNodeMgtMBean {
+public class NameNodeStatistics implements NameNodeStatisticsMBean {
   private NameNodeMetrics myMetrics;
-  private NameNode myNameNode;
+  private ObjectName mbeanName;
 
-  public NameNodeMgt(String sessionId, NameNodeMetrics nameNodeMetrics, NameNode nameNode) {
+  /**
+   * This constructs and registers the NameNodeStatisticsMBean
+   * @param nameNodeMetrics - the metrics from which the mbean gets its info
+   */
+  public NameNodeStatistics(NameNodeMetrics nameNodeMetrics) {
     myMetrics = nameNodeMetrics;
-    myNameNode = nameNode;
-    MBeanUtil.registerMBean("NameNode", "NameNodeStatistics", this);
+    mbeanName = MBeanUtil.registerMBean("NameNode", "NameNodeStatistics", this);
+  }
+  
+  /**
+   * Shuts down the statistics
+   *   - unregisters the mbean
+   */
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
   }
 
   /**
@@ -121,12 +134,6 @@ public class NameNodeMgt implements NameNodeMgtMBean {
     return myMetrics.syncs.getPreviousIntervalNumOps();
   }
 
-  /**
-   * @inheritDoc
-   */
-  public String getNameNodeState() {
-    return myNameNode.isInSafeMode() ? "safeMode" : "Operational";
-  }
  
   /**
    * @inheritDoc
@@ -146,9 +153,7 @@ public class NameNodeMgt implements NameNodeMgtMBean {
    * @inheritDoc
    */
   public void resetAllMinMax() {
-    myMetrics.syncs.resetMinMax();
-    myMetrics.transactions.resetMinMax();
-    myMetrics.blockReport.resetMinMax();
+    myMetrics.resetAllMinMax();
   }
   
   /**

+ 9 - 10
src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeMgtMBean.java → src/java/org/apache/hadoop/dfs/namenode/metrics/NameNodeStatisticsMBean.java

@@ -20,7 +20,8 @@ package org.apache.hadoop.dfs.namenode.metrics;
 
 /**
  * 
- * This is the JMX management interface for the name node.
+ * This is the JMX management interface for getting runtime statistics of
+ * the name node.
  * Many of the statistics are sampled and averaged on an interval 
  * which can be specified in the config file.
  * <p>
@@ -31,20 +32,18 @@ package org.apache.hadoop.dfs.namenode.metrics;
  * of sampled metrics by  specifying the following two lines
  *  in the hadoop-meterics.properties file:
  *  <pre>
- *        rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
- *        rpc.period=10
+ *        dfs.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+ *        dfs.period=10
  *  </pre>
  *<p>
  * Note that the metrics are collected regardless of the context used.
- * The context with the update thread is used to average the data periodically
+ * The context with the update thread is used to average the data periodically.
+ * <p>
+ * Name Node Status info is report in another MBean
+ * @see org.apache.hadoop.dfs.namenode.metrics.FSNamesystemMBean
  *
  */
-public interface NameNodeMgtMBean {
-  /**
-   * The state of the name node: Safemode or Operational
-   * @return the state
-   */
-  String getNameNodeState();
+public interface NameNodeStatisticsMBean {
   
   /**
    * The time spent in the Safemode at startup

+ 6 - 1
src/java/org/apache/hadoop/ipc/Server.java

@@ -966,11 +966,13 @@ public abstract class Server {
     this.maxIdleTime = conf.getInt("ipc.client.maxidletime", 120000);
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
-    this.rpcMetrics = new RpcMetrics(serverName, Integer.toString(port), this);
     
     // Start the listener here and let it bind to the port
     listener = new Listener();
     this.port = listener.getAddress().getPort();    
+    this.rpcMetrics = new RpcMetrics(serverName,
+                          Integer.toString(this.port), this);
+
 
     // Create the responder here
     responder = new Responder();
@@ -1009,6 +1011,9 @@ public abstract class Server {
     listener.doStop();
     responder.interrupt();
     notifyAll();
+    if (this.rpcMetrics != null) {
+      this.rpcMetrics.shutdown();
+    }
   }
 
   /** Wait for the server to be stopped.

+ 7 - 1
src/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
 public class RpcMetrics implements Updater {
   private MetricsRecord metricsRecord;
   private static Log LOG = LogFactory.getLog(JvmMetrics.class);
+  RpcMgt rpcMgt;
   
   private void setTags(String serverName, String port) {
     metricsRecord.setTag("serverName", serverName);
@@ -57,7 +58,7 @@ public class RpcMetrics implements Updater {
     context.registerUpdater(this);
     
     // Need to clean up the interface to RpcMgt - don't need both metrics and server params
-    new RpcMgt(serverName, port, this, server);
+    rpcMgt = new RpcMgt(serverName, port, this, server);
   }
   
   
@@ -81,4 +82,9 @@ public class RpcMetrics implements Updater {
     rpcDiscardedOps.pushMetric(metricsRecord);
     
   }
+
+  public void shutdown() {
+    if (rpcMgt != null) 
+      rpcMgt.shutdown();
+  }
 }

+ 10 - 1
src/java/org/apache/hadoop/ipc/metrics/RpcMgt.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.ipc.metrics;
 
 
+import javax.management.ObjectName;
+
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 
@@ -29,14 +31,21 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
 class RpcMgt implements RpcMgtMBean {
   private RpcMetrics myMetrics;
   private Server myServer;
+  private ObjectName mbeanName;
   
   RpcMgt(final String serviceName, final String port,
                 final RpcMetrics metrics, Server server) {
     myMetrics = metrics;
     myServer = server;
-    MBeanUtil.registerMBean(serviceName, "RpcStatistics", this);
+    mbeanName = MBeanUtil.registerMBean(serviceName,
+                    "RpcStatisticsForPort" + port, this);
   }
 
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+  
   /**
    * @inheritDoc
    */

+ 18 - 1
src/java/org/apache/hadoop/metrics/util/MBeanUtil.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.metrics.util;
 
 import java.lang.management.ManagementFactory;
 
+import javax.management.InstanceNotFoundException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -40,17 +41,33 @@ public class MBeanUtil {
    * @param serviceName
    * @param nameName
    * @param theMbean - the MBean to register
+   * @return the named used to register the MBean
    */	
-  static public void registerMBean(final String serviceName, 
+  static public ObjectName registerMBean(final String serviceName, 
 		  							final String nameName,
 		  							final Object theMbean) {
     final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
     ObjectName name = getMBeanName(serviceName, nameName);
     try {
       mbs.registerMBean(theMbean, name);
+      return name;
     } catch (Exception e) {
       e.printStackTrace();
     }
+    return null;
+  }
+  
+  static public void unregisterMBean(ObjectName mbeanName) {
+    final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    if (mbeanName == null) 
+        return;
+    try {
+      mbs.unregisterMBean(mbeanName);
+    } catch (InstanceNotFoundException e ) {
+      // ignore
+    } catch (Exception e) {
+      e.printStackTrace();
+    } 
   }
   
   static private ObjectName getMBeanName(final String serviceName,

+ 6 - 6
src/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingInt.java

@@ -46,15 +46,15 @@ public class MetricsTimeVaryingInt {
   }
   
   /**
-   * Inc metrics for numOps operations
-   * @param numOps - number of operations
+   * Inc metrics for incr vlaue
+   * @param incr - number of operations
    */
-  public synchronized void inc(final int numOps) {
-    currentValue += numOps;
+  public synchronized void inc(final int incr) {
+    currentValue += incr;
   }
   
   /**
-   * Inc metrics for one operation
+   * Inc metrics by one
    */
   public synchronized void inc() {
     currentValue++;
@@ -76,7 +76,7 @@ public class MetricsTimeVaryingInt {
    */
   public synchronized void pushMetric(final MetricsRecord mr) {
     intervalHeartBeat();
-    mr.incrMetric(name + "_num_ops", getPreviousIntervalValue());
+    mr.incrMetric(name, getPreviousIntervalValue());
   }
   
   

+ 46 - 1
src/test/org/apache/hadoop/dfs/SimulatedFSDataset.java

@@ -21,8 +21,16 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.HashMap;
+import java.util.Random;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 /**
@@ -196,6 +204,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   
   private HashMap<Block, BInfo> blockMap = null;
   private SimulatedStorage storage = null;
+  private String storageId;
   
   public SimulatedFSDataset(Configuration conf) throws IOException {
     setConf(conf);
@@ -203,12 +212,16 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   
   private SimulatedFSDataset() { // real construction when setConf called.. Uggg
   }
+  
   public Configuration getConf() {
     return conf;
   }
 
   public void setConf(Configuration iconf)  {
     conf = iconf;
+    storageId = conf.get("StorageId", "unknownStorageId" +
+                                        new Random().nextInt());
+    registerMBean(storageId);
     storage = new SimulatedStorage(
         conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY));
     //DataNode.LOG.info("Starting Simulated storage; Capacity = " + getCapacity() + 
@@ -312,7 +325,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   }
 
   public String toString() {
-    return "Simulated FSDataset";
+    return getStorageInfo();
   }
 
   public synchronized BlockWriteStreams writeToBlock(Block b, 
@@ -540,6 +553,38 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       length += len;
     }
   }
+  
+  private ObjectName mbeanName;
+  /**
+   * Register the FSDataset MBean
+   * @param storageId 
+   * 
+   * We use storage id for MBean name since a minicluster within a single
+   * Java VM may have multiple Simulated Datanodes.
+   */
+  void registerMBean(String storageId) {
+    // We wrap to bypass standard mbean naming convention.
+    // This wrapping can be removed in java 6 as Java6 is more flexible in 
+    // package naming for mbeans and their impl.
+    StandardMBean bean;
+    try {
+      bean = new StandardMBean(this,FSDatasetMBean.class);
+      mbeanName = MBeanUtil.registerMBean("DataNode-"+ storageId,
+                                          "FSDatasetStatus", bean);
+    } catch (NotCompliantMBeanException e) {
+      e.printStackTrace();
+    }
+
+    DataNode.LOG.info("Registered FSDatasetStatusMBean");
+  }
 
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+
+  public String getStorageInfo() {
+    return "Simulated FSDataset-" + storageId;
+  }
 
 }

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestFileLimit.java

@@ -68,7 +68,7 @@ public class TestFileLimit extends TestCase {
   {
     // wait for number of blocks to decrease
     while (true) {
-      long total = namesys.getBlockTotal() + namesys.dir.totalInodes();
+      long total = namesys.getBlocksTotal() + namesys.dir.totalInodes();
       System.out.println("Comparing current nodes " + total +
                          " to become " + num);
       if (total == num) {

+ 5 - 5
src/webapps/dfs/dfshealth.jsp

@@ -147,14 +147,14 @@
     
     out.print( "<div id=\"dfstable\"> <table>\n" +
 	       rowTxt() + colTxt() + "Capacity" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( fsn.totalCapacity() ) +
+	       FsShell.byteDesc( fsn.getCapacityTotal() ) +
 	       rowTxt() + colTxt() + "DFS Remaining" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( fsn.totalRemaining() ) +
+	       FsShell.byteDesc( fsn.getCapacityRemaining() ) +
 	       rowTxt() + colTxt() + "DFS Used" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( fsn.totalDfsUsed() ) +
+	       FsShell.byteDesc( fsn.getCapacityUsed() ) +
 	       rowTxt() + colTxt() + "DFS Used%" + colTxt() + ":" + colTxt() +
-	       FsShell.limitDecimalTo2((fsn.totalDfsUsed())*100.0/
-				       (fsn.totalCapacity() + 1e-10)) + " %" +
+	       FsShell.limitDecimalTo2((fsn.getCapacityUsed())*100.0/
+				       (fsn.getCapacityTotal() + 1e-10)) + " %" +
 	       rowTxt() + colTxt() +
                "<a href=\"#LiveNodes\">Live Nodes</a> " +
                colTxt() + ":" + colTxt() + live.size() +